From 9a6658cd8605dbd7bfcf0356a33d7d544e7e5471 Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Fri, 9 Sep 2016 13:29:50 -0700 Subject: [PATCH 01/83] [SPARK-17477]: SparkSQL cannot handle schema evolution from Int -> Long when parquet files have Int as its type while hive metastore has Long as its type --- .../expressions/SpecificMutableRow.scala | 25 ++++++++++++++++--- 1 file changed, 21 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 61ca7272dfa61..6b23292d075e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -242,14 +242,31 @@ final class SpecificMutableRow(val values: Array[MutableValue]) } } + /** + * When reading a hive table with schema evolution from Int to Long, + * if hive metastore has Long as its type while parquet files have Int, + * SparkSQL need to differentiate the actual type in the parquet files. + * Otherwise, it will result in java.lang.ClassCastException: + * [[MutableLong]] cannot be cast to [[MutableInt]]. + */ override def setInt(ordinal: Int, value: Int): Unit = { - val currentValue = values(ordinal).asInstanceOf[MutableInt] - currentValue.isNull = false - currentValue.value = value + values(ordinal) match { + case currentValue: MutableInt => + currentValue.isNull = false + currentValue.value = value + case currentValue: MutableLong => + currentValue.isNull = false + currentValue.value = value + } } override def getInt(i: Int): Int = { - values(i).asInstanceOf[MutableInt].value + values(i) match { + case currentValue: MutableInt => + currentValue.value + case currentValue: MutableLong => + currentValue.value.toInt + } } override def setFloat(ordinal: Int, value: Float): Unit = { From d978d4bb194f36ed915869e2092581b49a253a3d Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 9 Sep 2016 14:23:05 -0700 Subject: [PATCH 02/83] [SPARK-17354] [SQL] Partitioning by dates/timestamps should work with Parquet vectorized reader ## What changes were proposed in this pull request? This PR fixes `ColumnVectorUtils.populate` so that Parquet vectorized reader can read partitioned table with dates/timestamps. This works fine with Parquet normal reader. This is being only called within [VectorizedParquetRecordReader.java#L185](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java#L185). When partition column types are explicitly given to `DateType` or `TimestampType` (rather than inferring the type of partition column), this fails with the exception below: ``` 16/09/01 10:30:07 ERROR Executor: Exception in task 0.0 in stage 5.0 (TID 6) java.lang.ClassCastException: java.lang.Integer cannot be cast to java.sql.Date at org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate(ColumnVectorUtils.java:89) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:185) at org.apache.spark.sql.execution.datasources.parquet.VectorizedParquetRecordReader.initBatch(VectorizedParquetRecordReader.java:204) at org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat$$anonfun$buildReader$1.apply(ParquetFileFormat.scala:362) ... ``` ## How was this patch tested? Unit tests in `SQLQuerySuite`. Author: hyukjinkwon Closes #14919 from HyukjinKwon/SPARK-17354. --- .../vectorized/ColumnVectorUtils.java | 5 +- .../execution/vectorized/ColumnarBatch.java | 6 +++ .../datasources/parquet/ParquetIOSuite.scala | 49 ++++++++++++++++++- .../sql/hive/execution/SQLQuerySuite.scala | 21 ++++++++ 4 files changed, 78 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java index 2fa476b9cfb71..900d7c431e723 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnVectorUtils.java @@ -86,8 +86,9 @@ public static void populate(ColumnVector col, InternalRow row, int fieldIdx) { col.getChildColumn(0).putInts(0, capacity, c.months); col.getChildColumn(1).putLongs(0, capacity, c.microseconds); } else if (t instanceof DateType) { - Date date = (Date)row.get(fieldIdx, t); - col.putInts(0, capacity, DateTimeUtils.fromJavaDate(date)); + col.putInts(0, capacity, row.getInt(fieldIdx)); + } else if (t instanceof TimestampType) { + col.putLongs(0, capacity, row.getLong(fieldIdx)); } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java index f3afa8f938f86..62abc2a821a3a 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/ColumnarBatch.java @@ -137,6 +137,10 @@ public InternalRow copy() { DataType dt = columns[i].dataType(); if (dt instanceof BooleanType) { row.setBoolean(i, getBoolean(i)); + } else if (dt instanceof ByteType) { + row.setByte(i, getByte(i)); + } else if (dt instanceof ShortType) { + row.setShort(i, getShort(i)); } else if (dt instanceof IntegerType) { row.setInt(i, getInt(i)); } else if (dt instanceof LongType) { @@ -154,6 +158,8 @@ public InternalRow copy() { row.setDecimal(i, getDecimal(i, t.precision(), t.scale()), t.precision()); } else if (dt instanceof DateType) { row.setInt(i, getInt(i)); + } else if (dt instanceof TimestampType) { + row.setLong(i, getLong(i)); } else { throw new RuntimeException("Not implemented. " + dt); } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 4aa046bd91e02..3161a630af0f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -38,11 +38,12 @@ import org.apache.parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} -import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, UnsafeRow} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String // Write support class for nested groups: ParquetWriter initializes GroupWriteSupport // with an empty configuration (it is after all not intended to be used in this way?) @@ -689,6 +690,52 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { } } } + + test("VectorizedParquetRecordReader - partition column types") { + withTempPath { dir => + Seq(1).toDF().repartition(1).write.parquet(dir.getCanonicalPath) + + val dataTypes = + Seq(StringType, BooleanType, ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType(25, 5), DateType, TimestampType) + + val constantValues = + Seq( + UTF8String.fromString("a string"), + true, + 1.toByte, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75D, + Decimal("1234.23456"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"))) + + dataTypes.zip(constantValues).foreach { case (dt, v) => + val schema = StructType(StructField("pcol", dt) :: Nil) + val vectorizedReader = new VectorizedParquetRecordReader + val partitionValues = new GenericMutableRow(Array(v)) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + try { + vectorizedReader.initialize(file, null) + vectorizedReader.initBatch(schema, partitionValues) + vectorizedReader.nextKeyValue() + val row = vectorizedReader.getCurrentValue.asInstanceOf[InternalRow] + + // Use `GenericMutableRow` by explicitly copying rather than `ColumnarBatch` + // in order to use get(...) method which is not implemented in `ColumnarBatch`. + val actual = row.copy().get(1, dt) + val expected = v + assert(actual == expected) + } finally { + vectorizedReader.close() + } + } + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) 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 05d0687fb7e48..dc4d099f0f666 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 @@ -1787,6 +1787,27 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-17354: Partitioning by dates/timestamps works with Parquet vectorized reader") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + sql( + """CREATE TABLE order(id INT) + |PARTITIONED BY (pd DATE, pt TIMESTAMP) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql( + """INSERT INTO TABLE order PARTITION(pd, pt) + |SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt + """.stripMargin) + val actual = sql("SELECT * FROM order") + val expected = sql( + "SELECT 1 AS id, CAST('1990-02-24' AS DATE) AS pd, CAST('1990-02-24' AS TIMESTAMP) AS pt") + checkAnswer(actual, expected) + sql("DROP TABLE order") + } + } + def testCommandAvailable(command: String): Boolean = { val attempt = Try(Process(command).run(ProcessLogger(_ => ())).exitValue()) attempt.isSuccess && attempt.get == 0 From 965d9663056364afd6f30704ff8d044dfff05143 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Sat, 10 Sep 2016 09:27:22 +0800 Subject: [PATCH 03/83] [SPARK-15453][SQL] FileSourceScanExec to extract `outputOrdering` information ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-15453 Extracting sort ordering information in `FileSourceScanExec` so that planner can make use of it. My motivation to make this change was to get Sort Merge join in par with Hive's Sort-Merge-Bucket join when the source tables are bucketed + sorted. Query: ``` val df = (0 until 16).map(i => (i % 8, i * 2, i.toString)).toDF("i", "j", "k").coalesce(1) df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table8") df.write.bucketBy(8, "j", "k").sortBy("j", "k").saveAsTable("table9") context.sql("SELECT * FROM table8 a JOIN table9 b ON a.j=b.j AND a.k=b.k").explain(true) ``` Before: ``` == Physical Plan == *SortMergeJoin [j#120, k#121], [j#123, k#124], Inner :- *Sort [j#120 ASC, k#121 ASC], false, 0 : +- *Project [i#119, j#120, k#121] : +- *Filter (isnotnull(k#121) && isnotnull(j#120)) : +- *FileScan orc default.table8[i#119,j#120,k#121] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct +- *Sort [j#123 ASC, k#124 ASC], false, 0 +- *Project [i#122, j#123, k#124] +- *Filter (isnotnull(k#124) && isnotnull(j#123)) +- *FileScan orc default.table9[i#122,j#123,k#124] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct ``` After: (note that the `Sort` step is no longer there) ``` == Physical Plan == *SortMergeJoin [j#49, k#50], [j#52, k#53], Inner :- *Project [i#48, j#49, k#50] : +- *Filter (isnotnull(k#50) && isnotnull(j#49)) : +- *FileScan orc default.table8[i#48,j#49,k#50] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table8, PartitionFilters: [], PushedFilters: [IsNotNull(k), IsNotNull(j)], ReadSchema: struct +- *Project [i#51, j#52, k#53] +- *Filter (isnotnull(j#52) && isnotnull(k#53)) +- *FileScan orc default.table9[i#51,j#52,k#53] Batched: false, Format: ORC, InputPaths: file:/Users/tejasp/Desktop/dev/tp-spark/spark-warehouse/table9, PartitionFilters: [], PushedFilters: [IsNotNull(j), IsNotNull(k)], ReadSchema: struct ``` ## How was this patch tested? Added a test case in `JoinSuite`. Ran all other tests in `JoinSuite` Author: Tejas Patil Closes #14864 from tejasapatil/SPARK-15453_smb_optimization. --- .../sql/execution/DataSourceScanExec.scala | 79 +++++++++++++++---- .../spark/sql/sources/BucketedReadSuite.scala | 63 ++++++++++++++- 2 files changed, 123 insertions(+), 19 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 9597bdf34b714..6cdba406937de 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 @@ -23,12 +23,11 @@ import org.apache.commons.lang3.StringUtils import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SparkSession, SQLContext} +import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} @@ -156,24 +155,72 @@ case class FileSourceScanExec( false } - override val outputPartitioning: Partitioning = { + @transient private lazy val selectedPartitions = relation.location.listFiles(partitionFilters) + + override val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { val bucketSpec = if (relation.sparkSession.sessionState.conf.bucketingEnabled) { relation.bucketSpec } else { None } - bucketSpec.map { spec => - val numBuckets = spec.numBuckets - val bucketColumns = spec.bucketColumnNames.flatMap { n => - output.find(_.name == n) - } - if (bucketColumns.size == spec.bucketColumnNames.size) { - HashPartitioning(bucketColumns, numBuckets) - } else { - UnknownPartitioning(0) - } - }.getOrElse { - UnknownPartitioning(0) + bucketSpec match { + case Some(spec) => + // For bucketed columns: + // ----------------------- + // `HashPartitioning` would be used only when: + // 1. ALL the bucketing columns are being read from the table + // + // For sorted columns: + // --------------------- + // Sort ordering should be used when ALL these criteria's match: + // 1. `HashPartitioning` is being used + // 2. A prefix (or all) of the sort columns are being read from the table. + // + // Sort ordering would be over the prefix subset of `sort columns` being read + // from the table. + // eg. + // Assume (col0, col2, col3) are the columns read from the table + // If sort columns are (col0, col1), then sort ordering would be considered as (col0) + // If sort columns are (col1, col0), then sort ordering would be empty as per rule #2 + // above + + def toAttribute(colName: String): Option[Attribute] = + output.find(_.name == colName) + + val bucketColumns = spec.bucketColumnNames.flatMap(n => toAttribute(n)) + if (bucketColumns.size == spec.bucketColumnNames.size) { + val partitioning = HashPartitioning(bucketColumns, spec.numBuckets) + val sortColumns = + spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) + + val sortOrder = if (sortColumns.nonEmpty) { + // In case of bucketing, its possible to have multiple files belonging to the + // same bucket in a given relation. Each of these files are locally sorted + // but those files combined together are not globally sorted. Given that, + // the RDD partition will not be sorted even if the relation has sort columns set + // Current solution is to check if all the buckets have a single file in it + + val files = selectedPartitions.flatMap(partition => partition.files) + val bucketToFilesGrouping = + files.map(_.getPath.getName).groupBy(file => BucketingUtils.getBucketId(file)) + val singleFilePartitions = bucketToFilesGrouping.forall(p => p._2.length <= 1) + + if (singleFilePartitions) { + // TODO Currently Spark does not support writing columns sorting in descending order + // so using Ascending order. This can be fixed in future + sortColumns.map(attribute => SortOrder(attribute, Ascending)) + } else { + Nil + } + } else { + Nil + } + (partitioning, sortOrder) + } else { + (UnknownPartitioning(0), Nil) + } + case _ => + (UnknownPartitioning(0), Nil) } } @@ -187,8 +234,6 @@ case class FileSourceScanExec( "InputPaths" -> relation.location.paths.mkString(", ")) private lazy val inputRDD: RDD[InternalRow] = { - val selectedPartitions = relation.location.listFiles(partitionFilters) - val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala index ca2ec9f6a5ede..3ff85176de10e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/BucketedReadSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.DataSourceScanExec +import org.apache.spark.sql.execution.{DataSourceScanExec, SortExec} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.exchange.ShuffleExchange import org.apache.spark.sql.execution.joins.SortMergeJoinExec @@ -237,7 +237,9 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet bucketSpecRight: Option[BucketSpec], joinColumns: Seq[String], shuffleLeft: Boolean, - shuffleRight: Boolean): Unit = { + shuffleRight: Boolean, + sortLeft: Boolean = true, + sortRight: Boolean = true): Unit = { withTable("bucketed_table1", "bucketed_table2") { def withBucket( writer: DataFrameWriter[Row], @@ -247,6 +249,15 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet spec.numBuckets, spec.bucketColumnNames.head, spec.bucketColumnNames.tail: _*) + + if (spec.sortColumnNames.nonEmpty) { + writer.sortBy( + spec.sortColumnNames.head, + spec.sortColumnNames.tail: _* + ) + } else { + writer + } }.getOrElse(writer) } @@ -267,12 +278,21 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet assert(joined.queryExecution.executedPlan.isInstanceOf[SortMergeJoinExec]) val joinOperator = joined.queryExecution.executedPlan.asInstanceOf[SortMergeJoinExec] + // check existence of shuffle assert( joinOperator.left.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleLeft, s"expected shuffle in plan to be $shuffleLeft but found\n${joinOperator.left}") assert( joinOperator.right.find(_.isInstanceOf[ShuffleExchange]).isDefined == shuffleRight, s"expected shuffle in plan to be $shuffleRight but found\n${joinOperator.right}") + + // check existence of sort + assert( + joinOperator.left.find(_.isInstanceOf[SortExec]).isDefined == sortLeft, + s"expected sort in plan to be $shuffleLeft but found\n${joinOperator.left}") + assert( + joinOperator.right.find(_.isInstanceOf[SortExec]).isDefined == sortRight, + s"expected sort in plan to be $shuffleRight but found\n${joinOperator.right}") } } } @@ -321,6 +341,45 @@ class BucketedReadSuite extends QueryTest with SQLTestUtils with TestHiveSinglet } } + test("avoid shuffle and sort when bucket and sort columns are join keys") { + val bucketSpec = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + testBucketing( + bucketSpec, bucketSpec, Seq("i", "j"), + shuffleLeft = false, shuffleRight = false, + sortLeft = false, sortRight = false + ) + } + + test("avoid shuffle and sort when sort columns are a super set of join keys") { + val bucketSpec1 = Some(BucketSpec(8, Seq("i"), Seq("i", "j"))) + val bucketSpec2 = Some(BucketSpec(8, Seq("i"), Seq("i", "k"))) + testBucketing( + bucketSpec1, bucketSpec2, Seq("i"), + shuffleLeft = false, shuffleRight = false, + sortLeft = false, sortRight = false + ) + } + + test("only sort one side when sort columns are different") { + val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("k"))) + testBucketing( + bucketSpec1, bucketSpec2, Seq("i", "j"), + shuffleLeft = false, shuffleRight = false, + sortLeft = false, sortRight = true + ) + } + + test("only sort one side when sort columns are same but their ordering is different") { + val bucketSpec1 = Some(BucketSpec(8, Seq("i", "j"), Seq("i", "j"))) + val bucketSpec2 = Some(BucketSpec(8, Seq("i", "j"), Seq("j", "i"))) + testBucketing( + bucketSpec1, bucketSpec2, Seq("i", "j"), + shuffleLeft = false, shuffleRight = false, + sortLeft = false, sortRight = true + ) + } + test("avoid shuffle when grouping keys are equal to bucket keys") { withTable("bucketed_table") { df1.write.format("parquet").bucketBy(8, "i", "j").saveAsTable("bucketed_table") From a30257bb3928b69a88db73dd6130c098ce107d13 Mon Sep 17 00:00:00 2001 From: Yves Raimond Date: Sat, 10 Sep 2016 00:15:59 -0700 Subject: [PATCH 04/83] [SPARK-11496][GRAPHX] Parallel implementation of personalized pagerank (Updated version of [PR-9457](https://github.com/apache/spark/pull/9457), rebased on latest Spark master, and using mllib-local). This implements a parallel version of personalized pagerank, which runs all propagations for a list of source vertices in parallel. I ran a few benchmarks on the full [DBpedia](http://dbpedia.org/) graph. When running personalized pagerank for only one source node, the existing implementation is twice as fast as the parallel one (because of the SparseVector overhead). However for 10 source nodes, the parallel implementation is four times as fast. When increasing the number of source nodes, this difference becomes even greater. ![image](https://cloud.githubusercontent.com/assets/2491/10927702/dd82e4fa-8256-11e5-89a8-4799b407f502.png) Author: Yves Raimond Closes #14998 from moustaki/parallel-ppr. --- graphx/pom.xml | 5 ++ .../org/apache/spark/graphx/GraphOps.scala | 12 ++- .../apache/spark/graphx/lib/PageRank.scala | 81 +++++++++++++++++++ .../spark/graphx/lib/PageRankSuite.scala | 24 ++++++ 4 files changed, 121 insertions(+), 1 deletion(-) diff --git a/graphx/pom.xml b/graphx/pom.xml index bd4e53371b86d..10d5ba93ebb88 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -46,6 +46,11 @@ test-jar test + + org.apache.spark + spark-mllib-local_${scala.binary.version} + ${project.version} + org.apache.xbean xbean-asm5-shaded diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 868658dfe55e5..90907300be975 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -20,9 +20,10 @@ package org.apache.spark.graphx import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.SparkException import org.apache.spark.graphx.lib._ +import org.apache.spark.ml.linalg.Vector import org.apache.spark.rdd.RDD +import org.apache.spark.SparkException /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -391,6 +392,15 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali PageRank.runUntilConvergenceWithOptions(graph, tol, resetProb, Some(src)) } + /** + * Run parallel personalized PageRank for a given array of source vertices, such + * that all random walks are started relative to the source vertices + */ + def staticParallelPersonalizedPageRank(sources: Array[VertexId], numIter: Int, + resetProb: Double = 0.15) : Graph[Vector, Double] = { + PageRank.runParallelPersonalizedPageRank(graph, numIter, resetProb, sources) + } + /** * Run Personalized PageRank for a fixed number of iterations with * with all iterations originating at the source node diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 2f5bd4ed4ff6b..f4b00757a8b54 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -19,8 +19,11 @@ package org.apache.spark.graphx.lib import scala.reflect.ClassTag +import breeze.linalg.{Vector => BV} + import org.apache.spark.graphx._ import org.apache.spark.internal.Logging +import org.apache.spark.ml.linalg.{Vector, Vectors} /** * PageRank algorithm implementation. There are two implementations of PageRank implemented. @@ -162,6 +165,84 @@ object PageRank extends Logging { rankGraph } + /** + * Run Personalized PageRank for a fixed number of iterations, for a + * set of starting nodes in parallel. Returns a graph with vertex attributes + * containing the pagerank relative to all starting nodes (as a sparse vector) and + * edge attributes the normalized edge weight + * + * @tparam VD The original vertex attribute (not used) + * @tparam ED The original edge attribute (not used) + * + * @param graph The graph on which to compute personalized pagerank + * @param numIter The number of iterations to run + * @param resetProb The random reset probability + * @param sources The list of sources to compute personalized pagerank from + * @return the graph with vertex attributes + * containing the pagerank relative to all starting nodes (as a sparse vector) and + * edge attributes the normalized edge weight + */ + def runParallelPersonalizedPageRank[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED], + numIter: Int, resetProb: Double = 0.15, + sources: Array[VertexId]): Graph[Vector, Double] = { + // TODO if one sources vertex id is outside of the int range + // we won't be able to store its activations in a sparse vector + val zero = Vectors.sparse(sources.size, List()).asBreeze + val sourcesInitMap = sources.zipWithIndex.map { case (vid, i) => + val v = Vectors.sparse(sources.size, Array(i), Array(resetProb)).asBreeze + (vid, v) + }.toMap + val sc = graph.vertices.sparkContext + val sourcesInitMapBC = sc.broadcast(sourcesInitMap) + // Initialize the PageRank graph with each edge attribute having + // weight 1/outDegree and each source vertex with attribute 1.0. + var rankGraph = graph + // Associate the degree with each vertex + .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } + // Set the weight on the edges based on the degree + .mapTriplets(e => 1.0 / e.srcAttr, TripletFields.Src) + .mapVertices { (vid, attr) => + if (sourcesInitMapBC.value contains vid) { + sourcesInitMapBC.value(vid) + } else { + zero + } + } + + var i = 0 + while (i < numIter) { + val prevRankGraph = rankGraph + // Propagates the message along outbound edges + // and adding start nodes back in with activation resetProb + val rankUpdates = rankGraph.aggregateMessages[BV[Double]]( + ctx => ctx.sendToDst(ctx.srcAttr :* ctx.attr), + (a : BV[Double], b : BV[Double]) => a :+ b, TripletFields.Src) + + rankGraph = rankGraph.joinVertices(rankUpdates) { + (vid, oldRank, msgSum) => + val popActivations: BV[Double] = msgSum :* (1.0 - resetProb) + val resetActivations = if (sourcesInitMapBC.value contains vid) { + sourcesInitMapBC.value(vid) + } else { + zero + } + popActivations :+ resetActivations + }.cache() + + rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist(false) + prevRankGraph.edges.unpersist(false) + + logInfo(s"Parallel Personalized PageRank finished iteration $i.") + + i += 1 + } + + rankGraph.mapVertices { (vid, attr) => + Vectors.fromBreeze(attr) + } + } + /** * Run a dynamic version of PageRank returning a graph with vertex attributes containing the * PageRank and edge attributes containing the normalized edge weight. diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index bdff31446f8ee..b6305c8d00aba 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -118,11 +118,29 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val dynamicRanks = starGraph.personalizedPageRank(0, 0, resetProb).vertices.cache() assert(compareRanks(staticRanks2, dynamicRanks) < errorTol) + val parallelStaticRanks1 = starGraph + .staticParallelPersonalizedPageRank(Array(0), 1, resetProb).mapVertices { + case (vertexId, vector) => vector(0) + }.vertices.cache() + assert(compareRanks(staticRanks1, parallelStaticRanks1) < errorTol) + + val parallelStaticRanks2 = starGraph + .staticParallelPersonalizedPageRank(Array(0, 1), 2, resetProb).mapVertices { + case (vertexId, vector) => vector(0) + }.vertices.cache() + assert(compareRanks(staticRanks2, parallelStaticRanks2) < errorTol) + // We have one outbound edge from 1 to 0 val otherStaticRanks2 = starGraph.staticPersonalizedPageRank(1, numIter = 2, resetProb) .vertices.cache() val otherDynamicRanks = starGraph.personalizedPageRank(1, 0, resetProb).vertices.cache() + val otherParallelStaticRanks2 = starGraph + .staticParallelPersonalizedPageRank(Array(0, 1), 2, resetProb).mapVertices { + case (vertexId, vector) => vector(1) + }.vertices.cache() assert(compareRanks(otherDynamicRanks, otherStaticRanks2) < errorTol) + assert(compareRanks(otherStaticRanks2, otherParallelStaticRanks2) < errorTol) + assert(compareRanks(otherDynamicRanks, otherParallelStaticRanks2) < errorTol) } } // end of test Star PersonalPageRank @@ -177,6 +195,12 @@ class PageRankSuite extends SparkFunSuite with LocalSparkContext { val dynamicRanks = chain.personalizedPageRank(4, tol, resetProb).vertices assert(compareRanks(staticRanks, dynamicRanks) < errorTol) + + val parallelStaticRanks = chain + .staticParallelPersonalizedPageRank(Array(4), numIter, resetProb).mapVertices { + case (vertexId, vector) => vector(0) + }.vertices.cache() + assert(compareRanks(staticRanks, parallelStaticRanks) < errorTol) } } } From bca8f30f5207ae25d1edc4e3e779de728aa0f215 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sat, 10 Sep 2016 00:27:10 -0700 Subject: [PATCH 05/83] [SPARK-15509][FOLLOW-UP][ML][SPARKR] R MLlib algorithms should support input columns "features" and "label" MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? #13584 resolved the issue of features and label columns conflict with ```RFormula``` default ones when loading libsvm data, but it still left some issues should be resolved: 1, It’s not necessary to check and rename label column. Since we have considerations on the design of ```RFormula```, it can handle the case of label column already exists(with restriction of the existing label column should be numeric/boolean type). So it’s not necessary to change the column name to avoid conflict. If the label column is not numeric/boolean type, ```RFormula``` will throw exception. 2, We should rename features column name to new one if there is conflict, but appending a random value is enough since it was used internally only. We done similar work when implementing ```SQLTransformer```. 3, We should set correct new features column for the estimators. Take ```GLM``` as example: ```GLM``` estimator should set features column with the changed one(rFormula.getFeaturesCol) rather than the default “features”. Although it’s same when training model, but it involves problems when predicting. The following is the prediction result of GLM before this PR: ![image](https://cloud.githubusercontent.com/assets/1962026/18308227/84c3c452-74a8-11e6-9caa-9d6d846cc957.png) We should drop the internal used feature column name, otherwise, it will appear on the prediction DataFrame which will confused users. And this behavior is same as other scenarios which does not exist column name conflict. After this PR: ![image](https://cloud.githubusercontent.com/assets/1962026/18308240/92082a04-74a8-11e6-9226-801f52b856d9.png) ## How was this patch tested? Existing unit tests. Author: Yanbo Liang Closes #14993 from yanboliang/spark-15509. --- .../ml/r/AFTSurvivalRegressionWrapper.scala | 1 + .../spark/ml/r/GaussianMixtureWrapper.scala | 1 + .../GeneralizedLinearRegressionWrapper.scala | 1 + .../ml/r/IsotonicRegressionWrapper.scala | 1 + .../org/apache/spark/ml/r/KMeansWrapper.scala | 1 + .../apache/spark/ml/r/NaiveBayesWrapper.scala | 1 + .../org/apache/spark/ml/r/RWrapperUtils.scala | 34 +++---------------- .../spark/ml/r/RWrapperUtilsSuite.scala | 16 +++------ 8 files changed, 14 insertions(+), 42 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala index 67d037ed6e024..bd965acf56944 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -99,6 +99,7 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg val aft = new AFTSurvivalRegression() .setCensorCol(censorCol) .setFitIntercept(rFormula.hasIntercept) + .setFeaturesCol(rFormula.getFeaturesCol) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, aft)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala index b654233a89360..b708702959829 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala @@ -85,6 +85,7 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp .setK(k) .setMaxIter(maxIter) .setTol(tol) + .setFeaturesCol(rFormula.getFeaturesCol) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, gm)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala index 35313258f940e..b1bb577e1ffe4 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala @@ -89,6 +89,7 @@ private[r] object GeneralizedLinearRegressionWrapper .setMaxIter(maxIter) .setWeightCol(weightCol) .setRegParam(regParam) + .setFeaturesCol(rFormula.getFeaturesCol) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, glr)) .fit(data) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala index 2ed7d7b770cc9..48632316f3950 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala @@ -75,6 +75,7 @@ private[r] object IsotonicRegressionWrapper .setIsotonic(isotonic) .setFeatureIndex(featureIndex) .setWeightCol(weightCol) + .setFeaturesCol(rFormula.getFeaturesCol) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, isotonicRegression)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala index 8616a8c01e5ac..ea9458525aa31 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala @@ -86,6 +86,7 @@ private[r] object KMeansWrapper extends MLReadable[KMeansWrapper] { .setK(k) .setMaxIter(maxIter) .setInitMode(initMode) + .setFeaturesCol(rFormula.getFeaturesCol) val pipeline = new Pipeline() .setStages(Array(rFormulaModel, kMeans)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala index f2cb24b96404b..d1a39fea76ef8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -73,6 +73,7 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { val naiveBayes = new NaiveBayes() .setSmoothing(smoothing) .setModelType("bernoulli") + .setFeaturesCol(rFormula.getFeaturesCol) .setPredictionCol(PREDICTED_LABEL_INDEX_COL) val idxToStr = new IndexToString() .setInputCol(PREDICTED_LABEL_INDEX_COL) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala index 6a435992e3b35..379007c4d948d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrapperUtils.scala @@ -19,14 +19,15 @@ package org.apache.spark.ml.r import org.apache.spark.internal.Logging import org.apache.spark.ml.feature.RFormula +import org.apache.spark.ml.util.Identifiable import org.apache.spark.sql.Dataset object RWrapperUtils extends Logging { /** * DataFrame column check. - * When loading data, default columns "features" and "label" will be added. And these two names - * would conflict with RFormula default feature and label column names. + * When loading libsvm data, default columns "features" and "label" will be added. + * And "features" would conflict with RFormula default feature column names. * Here is to change the column name to avoid "column already exists" error. * * @param rFormula RFormula instance @@ -34,38 +35,11 @@ object RWrapperUtils extends Logging { * @return Unit */ def checkDataColumns(rFormula: RFormula, data: Dataset[_]): Unit = { - if (data.schema.fieldNames.contains(rFormula.getLabelCol)) { - val newLabelName = convertToUniqueName(rFormula.getLabelCol, data.schema.fieldNames) - logWarning( - s"data containing ${rFormula.getLabelCol} column, using new name $newLabelName instead") - rFormula.setLabelCol(newLabelName) - } - if (data.schema.fieldNames.contains(rFormula.getFeaturesCol)) { - val newFeaturesName = convertToUniqueName(rFormula.getFeaturesCol, data.schema.fieldNames) + val newFeaturesName = s"${Identifiable.randomUID(rFormula.getFeaturesCol)}" logWarning(s"data containing ${rFormula.getFeaturesCol} column, " + s"using new name $newFeaturesName instead") rFormula.setFeaturesCol(newFeaturesName) } } - - /** - * Convert conflicting name to be an unique name. - * Appending a sequence number, like originalName_output1 - * and incrementing until it is not already there - * - * @param originalName Original name - * @param fieldNames Array of field names in existing schema - * @return String - */ - def convertToUniqueName(originalName: String, fieldNames: Array[String]): String = { - var counter = 1 - var newName = originalName + "_output" - - while (fieldNames.contains(newName)) { - newName = originalName + "_output" + counter - counter += 1 - } - newName - } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala index ddc24cb3a6481..27b03918d951e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/r/RWrapperUtilsSuite.scala @@ -35,22 +35,14 @@ class RWrapperUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { // after checking, model build is ok RWrapperUtils.checkDataColumns(rFormula, data) - assert(rFormula.getLabelCol == "label_output") - assert(rFormula.getFeaturesCol == "features_output") + assert(rFormula.getLabelCol == "label") + assert(rFormula.getFeaturesCol.startsWith("features_")) val model = rFormula.fit(data) assert(model.isInstanceOf[RFormulaModel]) - assert(model.getLabelCol == "label_output") - assert(model.getFeaturesCol == "features_output") - } - - test("generate unique name by appending a sequence number") { - val originalName = "label" - val fieldNames = Array("label_output", "label_output1", "label_output2") - val newName = RWrapperUtils.convertToUniqueName(originalName, fieldNames) - - assert(newName === "label_output3") + assert(model.getLabelCol == "label") + assert(model.getFeaturesCol.startsWith("features_")) } } From 705fbdc605d359ca3c68c14e408b2da71510ebff Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 10 Sep 2016 10:18:53 +0100 Subject: [PATCH 06/83] [SPARK-17396][CORE] Share the task support between UnionRDD instances. ## What changes were proposed in this pull request? Share the ForkJoinTaskSupport between UnionRDD instances to avoid creating a huge number of threads if lots of RDDs are created at the same time. ## How was this patch tested? This uses existing UnionRDD tests. Author: Ryan Blue Closes #14985 from rdblue/SPARK-17396-use-shared-pool. --- .../main/scala/org/apache/spark/rdd/UnionRDD.scala | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 8171dcc046379..ad1fddbde7b00 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.collection.parallel.ForkJoinTaskSupport +import scala.collection.parallel.{ForkJoinTaskSupport, ThreadPoolTaskSupport} import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag @@ -58,6 +58,11 @@ private[spark] class UnionPartition[T: ClassTag]( } } +object UnionRDD { + private[spark] lazy val partitionEvalTaskSupport = + new ForkJoinTaskSupport(new ForkJoinPool(8)) +} + @DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @@ -68,13 +73,10 @@ class UnionRDD[T: ClassTag]( private[spark] val isPartitionListingParallel: Boolean = rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) - @transient private lazy val partitionEvalTaskSupport = - new ForkJoinTaskSupport(new ForkJoinPool(8)) - override def getPartitions: Array[Partition] = { val parRDDs = if (isPartitionListingParallel) { val parArray = rdds.par - parArray.tasksupport = partitionEvalTaskSupport + parArray.tasksupport = UnionRDD.partitionEvalTaskSupport parArray } else { rdds From 9e6680eadae9d827e2cae8ed5dc291ddc1914be9 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Sat, 10 Sep 2016 09:52:53 -0700 Subject: [PATCH 07/83] [SPARK-16445][MLLIB][SPARKR] Fix @return description for sparkR mlp summary() method ## What changes were proposed in this pull request? Fix summary() method's `return` description for spark.mlp ## How was this patch tested? Ran tests locally on my laptop. Author: Xin Ren Closes #15015 from keypointt/SPARK-16445-2. --- R/pkg/R/mllib.R | 6 +++--- R/pkg/inst/tests/testthat/test_mllib.R | 2 ++ 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/R/pkg/R/mllib.R b/R/pkg/R/mllib.R index f8d1095a493dc..234b208166b54 100644 --- a/R/pkg/R/mllib.R +++ b/R/pkg/R/mllib.R @@ -720,8 +720,9 @@ setMethod("predict", signature(object = "MultilayerPerceptronClassificationModel # Returns the summary of a Multilayer Perceptron Classification Model produced by \code{spark.mlp} #' @param object a Multilayer Perceptron Classification Model fitted by \code{spark.mlp} -#' @return \code{summary} returns a list containing \code{layers}, the label distribution, and -#' \code{tables}, conditional probabilities given the target label. +#' @return \code{summary} returns a list containing \code{labelCount}, \code{layers}, and +#' \code{weights}. For \code{weights}, it is a numeric vector with length equal to +#' the expected given the architecture (i.e., for 8-10-2 network, 100 connection weights). #' @rdname spark.mlp #' @export #' @aliases summary,MultilayerPerceptronClassificationModel-method @@ -732,7 +733,6 @@ setMethod("summary", signature(object = "MultilayerPerceptronClassificationModel labelCount <- callJMethod(jobj, "labelCount") layers <- unlist(callJMethod(jobj, "layers")) weights <- callJMethod(jobj, "weights") - weights <- matrix(weights, nrow = length(weights)) list(labelCount = labelCount, layers = layers, weights = weights) }) diff --git a/R/pkg/inst/tests/testthat/test_mllib.R b/R/pkg/inst/tests/testthat/test_mllib.R index ac896cfbcfff7..5b1404c621bd1 100644 --- a/R/pkg/inst/tests/testthat/test_mllib.R +++ b/R/pkg/inst/tests/testthat/test_mllib.R @@ -369,6 +369,8 @@ test_that("spark.mlp", { expect_equal(summary$labelCount, 3) expect_equal(summary$layers, c(4, 5, 4, 3)) expect_equal(length(summary$weights), 64) + expect_equal(head(summary$weights, 5), list(-0.878743, 0.2154151, -1.16304, -0.6583214, 1.009825), + tolerance = 1e-6) # Test predict method mlpTestDF <- df From 71d62919b7e5ece3befa394a017c3847bbba4f54 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 11 Sep 2016 08:00:55 +0100 Subject: [PATCH 08/83] [SPARK-17389][ML][MLLIB] KMeans speedup with better choice of k-means|| init steps = 2 ## What changes were proposed in this pull request? Reduce default k-means|| init steps to 2 from 5. See JIRA for discussion. See also https://github.com/apache/spark/pull/14948 ## How was this patch tested? Existing tests. Author: Sean Owen Closes #14956 from srowen/SPARK-17389.2. --- .../org/apache/spark/mllib/clustering/KMeans.scala | 6 +++--- .../clustering/PowerIterationClusteringSuite.scala | 10 +++------- 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index de9fa4aebf484..23141aaf42b49 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -51,10 +51,10 @@ class KMeans private ( /** * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, - * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4, seed: random}. + * initializationMode: "k-means||", initializationSteps: 2, epsilon: 1e-4, seed: random}. */ @Since("0.8.0") - def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4, Utils.random.nextLong()) + def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 2, 1e-4, Utils.random.nextLong()) /** * Number of clusters to create (k). @@ -134,7 +134,7 @@ class KMeans private ( /** * Set the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 5 is almost always enough. Default: 5. + * setting -- the default of 2 is almost always enough. Default: 2. */ @Since("0.8.0") def setInitializationSteps(initializationSteps: Int): this.type = { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 3d81d375c716e..b33b86b39a42f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -49,7 +49,7 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon val r1 = 1.0 val n1 = 10 val r2 = 4.0 - val n2 = 40 + val n2 = 10 val n = n1 + n2 val points = genCircle(r1, n1) ++ genCircle(r2, n2) val similarities = for (i <- 1 until n; j <- 0 until i) yield { @@ -83,7 +83,7 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon val r1 = 1.0 val n1 = 10 val r2 = 4.0 - val n2 = 40 + val n2 = 10 val n = n1 + n2 val points = genCircle(r1, n1) ++ genCircle(r2, n2) val similarities = for (i <- 1 until n; j <- 0 until i) yield { @@ -91,11 +91,7 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon } val edges = similarities.flatMap { case (i, j, s) => - if (i != j) { - Seq(Edge(i, j, s), Edge(j, i, s)) - } else { - None - } + Seq(Edge(i, j, s), Edge(j, i, s)) } val graph = Graph.fromEdges(sc.parallelize(edges, 2), 0.0) From eb10a06fb604912bfdb1d96bdd7573362b5ee438 Mon Sep 17 00:00:00 2001 From: Timothy Hunter Date: Sun, 11 Sep 2016 08:03:45 +0100 Subject: [PATCH 09/83] [SPARK-17439][SQL] Fixing compression issues with approximate quantiles and adding more tests ## What changes were proposed in this pull request? This PR build on #14976 and fixes a correctness bug that would cause the wrong quantile to be returned for small target errors. ## How was this patch tested? This PR adds 8 unit tests that were failing without the fix. Author: Timothy Hunter Author: Sean Owen Closes #15002 from thunterdb/ml-1783. --- .../sql/catalyst/util/QuantileSummaries.scala | 16 +++++++--- .../util/QuantileSummariesSuite.scala | 29 +++++++++++++++++-- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index 7512ace188569..fd62bd511fac0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.util -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, ListBuffer} import org.apache.spark.sql.catalyst.util.QuantileSummaries.Stats @@ -61,7 +61,12 @@ class QuantileSummaries( def insert(x: Double): QuantileSummaries = { headSampled += x if (headSampled.size >= defaultHeadSize) { - this.withHeadBufferInserted + val result = this.withHeadBufferInserted + if (result.sampled.length >= compressThreshold) { + result.compress() + } else { + result + } } else { this } @@ -236,7 +241,7 @@ object QuantileSummaries { if (currentSamples.isEmpty) { return Array.empty[Stats] } - val res: ArrayBuffer[Stats] = ArrayBuffer.empty + val res = ListBuffer.empty[Stats] // Start for the last element, which is always part of the set. // The head contains the current new head, that may be merged with the current element. var head = currentSamples.last @@ -258,7 +263,10 @@ object QuantileSummaries { } res.prepend(head) // If necessary, add the minimum element: - res.prepend(currentSamples.head) + val currHead = currentSamples.head + if (currHead.value < head.value) { + res.prepend(currentSamples.head) + } res.toArray } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala index 89b2a22a3de45..5e90970b1bb2e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/QuantileSummariesSuite.scala @@ -40,6 +40,20 @@ class QuantileSummariesSuite extends SparkFunSuite { summary.compress() } + /** + * Interleaves compression and insertions. + */ + private def buildCompressSummary( + data: Seq[Double], + epsi: Double, + threshold: Int): QuantileSummaries = { + var summary = new QuantileSummaries(threshold, epsi) + data.foreach { x => + summary = summary.insert(x).compress() + } + summary + } + private def checkQuantile(quant: Double, data: Seq[Double], summary: QuantileSummaries): Unit = { val approx = summary.query(quant) // The rank of the approximation. @@ -54,8 +68,8 @@ class QuantileSummariesSuite extends SparkFunSuite { for { (seq_name, data) <- Seq(increasing, decreasing, random) - epsi <- Seq(0.1, 0.0001) - compression <- Seq(1000, 10) + epsi <- Seq(0.1, 0.0001) // With a significant value and with full precision + compression <- Seq(1000, 10) // This interleaves n so that we test without and with compression } { test(s"Extremas with epsi=$epsi and seq=$seq_name, compression=$compression") { @@ -75,6 +89,17 @@ class QuantileSummariesSuite extends SparkFunSuite { checkQuantile(0.1, data, s) checkQuantile(0.001, data, s) } + + test(s"Some quantile values with epsi=$epsi and seq=$seq_name, compression=$compression " + + s"(interleaved)") { + val s = buildCompressSummary(data, epsi, compression) + assert(s.count == data.size, s"Found count=${s.count} but data size=${data.size}") + checkQuantile(0.9999, data, s) + checkQuantile(0.9, data, s) + checkQuantile(0.5, data, s) + checkQuantile(0.1, data, s) + checkQuantile(0.001, data, s) + } } // Tests for merging procedure From 8ce0d5e2c756b29d6bc446fadffeacd2386d4513 Mon Sep 17 00:00:00 2001 From: tone-zhang Date: Sun, 11 Sep 2016 10:17:53 +0100 Subject: [PATCH 10/83] [SPARK-17330][SPARK UT] Clean up spark-warehouse in UT ## What changes were proposed in this pull request? Check the database warehouse used in Spark UT, and remove the existing database file before run the UT (SPARK-8368). ## How was this patch tested? Run Spark UT with the command for several times: ./build/sbt -Pyarn -Phadoop-2.6 -Phive -Phive-thriftserver "test-only *HiveSparkSubmitSuit*" Without the patch, the test case can be passed only at the first time, and always failed from the second time. With the patch the test case always can be passed correctly. Author: tone-zhang Closes #14894 from tone-zhang/issue1. --- .../org/apache/spark/sql/execution/command/DDLSuite.scala | 2 ++ .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 6 +++++- 2 files changed, 7 insertions(+), 1 deletion(-) 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 05f826a11b586..95672e01f5546 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 @@ -43,6 +43,8 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { // drop all databases, tables and functions after each test spark.sessionState.catalog.reset() } finally { + val path = System.getProperty("user.dir") + "/spark-warehouse" + Utils.deleteRecursively(new File(path)) super.afterEach() } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index af282866669b8..29317e2887861 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -590,7 +590,9 @@ object SparkSubmitClassLoaderTest extends Logging { def main(args: Array[String]) { Utils.configTestLog4j("INFO") val conf = new SparkConf() + val hiveWarehouseLocation = Utils.createTempDir() conf.set("spark.ui.enabled", "false") + conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j") @@ -699,11 +701,13 @@ object SPARK_9757 extends QueryTest { def main(args: Array[String]): Unit = { Utils.configTestLog4j("INFO") + val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() .set("spark.sql.hive.metastore.version", "0.13.1") .set("spark.sql.hive.metastore.jars", "maven") - .set("spark.ui.enabled", "false")) + .set("spark.ui.enabled", "false") + .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession From 1d4165f69bb2265edb72da6b9e2770b86e8fbe15 Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Sun, 11 Sep 2016 10:19:39 +0100 Subject: [PATCH 11/83] [SPARK-17336][PYSPARK] Fix appending multiple times to PYTHONPATH from spark-config.sh ## What changes were proposed in this pull request? During startup of Spark standalone, the script file spark-config.sh appends to the PYTHONPATH and can be sourced many times, causing duplicates in the path. This change adds a env flag that is set when the PYTHONPATH is appended so it will happen only one time. ## How was this patch tested? Manually started standalone master/worker and verified PYTHONPATH has no duplicate entries. Author: Bryan Cutler Closes #15028 from BryanCutler/fix-duplicate-pythonpath-SPARK-17336. --- sbin/spark-config.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index a7a44cdde6c77..b7284487c511d 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -26,5 +26,8 @@ fi export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: -export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:${PYTHONPATH}" +if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then + export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.3-src.zip:${PYTHONPATH}" + export PYSPARK_PYTHONPATH_SET=1 +fi From 7d309ff9860c1e04f1a6b480acc861f0e634e968 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Sun, 11 Sep 2016 13:47:13 +0100 Subject: [PATCH 12/83] [SPARK-17389][FOLLOW-UP][ML] Change KMeans k-means|| default init steps from 5 to 2. ## What changes were proposed in this pull request? #14956 reduced default k-means|| init steps to 2 from 5 only for spark.mllib package, we should also do same change for spark.ml and PySpark. ## How was this patch tested? Existing tests. Author: Yanbo Liang Closes #15050 from yanboliang/spark-17389. --- .../scala/org/apache/spark/ml/clustering/KMeans.scala | 4 ++-- .../org/apache/spark/ml/clustering/KMeansSuite.scala | 2 +- python/pyspark/ml/clustering.py | 10 +++++----- python/pyspark/mllib/clustering.py | 6 +++--- 4 files changed, 11 insertions(+), 11 deletions(-) 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 6c46be719674b..b04e82838e714 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 @@ -69,7 +69,7 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe /** * Param for the number of steps for the k-means|| initialization mode. This is an advanced - * setting -- the default of 5 is almost always enough. Must be > 0. Default: 5. + * setting -- the default of 2 is almost always enough. Must be > 0. Default: 2. * @group expertParam */ @Since("1.5.0") @@ -262,7 +262,7 @@ class KMeans @Since("1.5.0") ( k -> 2, maxIter -> 20, initMode -> MLlibKMeans.K_MEANS_PARALLEL, - initSteps -> 5, + initSteps -> 2, tol -> 1e-4) @Since("1.5.0") diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index 88f31a1cd26fb..c9ba5a288aadf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -45,7 +45,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultR assert(kmeans.getPredictionCol === "prediction") assert(kmeans.getMaxIter === 20) assert(kmeans.getInitMode === MLlibKMeans.K_MEANS_PARALLEL) - assert(kmeans.getInitSteps === 5) + assert(kmeans.getInitSteps === 2) assert(kmeans.getTol === 1e-4) } diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 4dab83362a0a4..7632f05c3b68c 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -254,14 +254,14 @@ class KMeans(JavaEstimator, HasFeaturesCol, HasPredictionCol, HasMaxIter, HasTol @keyword_only def __init__(self, featuresCol="features", predictionCol="prediction", k=2, - initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None): + initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None): """ __init__(self, featuresCol="features", predictionCol="prediction", k=2, \ - initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None) + initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None) """ super(KMeans, self).__init__() self._java_obj = self._new_java_obj("org.apache.spark.ml.clustering.KMeans", self.uid) - self._setDefault(k=2, initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20) + self._setDefault(k=2, initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -271,10 +271,10 @@ def _create_model(self, java_model): @keyword_only @since("1.5.0") def setParams(self, featuresCol="features", predictionCol="prediction", k=2, - initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None): + initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None): """ setParams(self, featuresCol="features", predictionCol="prediction", k=2, \ - initMode="k-means||", initSteps=5, tol=1e-4, maxIter=20, seed=None) + initMode="k-means||", initSteps=2, tol=1e-4, maxIter=20, seed=None) Sets params for KMeans. """ diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 29aa615125770..2036168e456fd 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -306,7 +306,7 @@ class KMeans(object): @classmethod @since('0.9.0') def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||", - seed=None, initializationSteps=5, epsilon=1e-4, initialModel=None): + seed=None, initializationSteps=2, epsilon=1e-4, initialModel=None): """ Train a k-means clustering model. @@ -330,9 +330,9 @@ def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||" (default: None) :param initializationSteps: Number of steps for the k-means|| initialization mode. - This is an advanced setting -- the default of 5 is almost + This is an advanced setting -- the default of 2 is almost always enough. - (default: 5) + (default: 2) :param epsilon: Distance threshold within which a center will be considered to have converged. If all centers move less than this Euclidean From ed112bdc336b23238f3b6e7d75d0cd6ea7d922f5 Mon Sep 17 00:00:00 2001 From: Sameer Agarwal Date: Sun, 11 Sep 2016 17:35:27 +0200 Subject: [PATCH 13/83] [SPARK-17415][SQL] Better error message for driver-side broadcast join OOMs ## What changes were proposed in this pull request? This is a trivial patch that catches all `OutOfMemoryError` while building the broadcast hash relation and rethrows it by wrapping it in a nice error message. ## How was this patch tested? Existing Tests Author: Sameer Agarwal Closes #14979 from sameeragarwal/broadcast-join-error. --- .../exchange/BroadcastExchangeExec.scala | 73 +++++++++++-------- 1 file changed, 42 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala index a809076de5419..7be5d31d4a765 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/BroadcastExchangeExec.scala @@ -21,6 +21,7 @@ import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ import org.apache.spark.{broadcast, SparkException} +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow @@ -28,6 +29,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPar import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.ThreadUtils /** @@ -70,38 +72,47 @@ case class BroadcastExchangeExec( // This will run in another thread. Set the execution id so that we can connect these jobs // with the correct execution. SQLExecution.withExecutionId(sparkContext, executionId) { - val beforeCollect = System.nanoTime() - // Note that we use .executeCollect() because we don't want to convert data to Scala types - val input: Array[InternalRow] = child.executeCollect() - if (input.length >= 512000000) { - throw new SparkException( - s"Cannot broadcast the table with more than 512 millions rows: ${input.length} rows") + try { + val beforeCollect = System.nanoTime() + // Note that we use .executeCollect() because we don't want to convert data to Scala types + val input: Array[InternalRow] = child.executeCollect() + if (input.length >= 512000000) { + throw new SparkException( + s"Cannot broadcast the table with more than 512 millions rows: ${input.length} rows") + } + val beforeBuild = System.nanoTime() + longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 + val dataSize = input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + longMetric("dataSize") += dataSize + if (dataSize >= (8L << 30)) { + throw new SparkException( + s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") + } + + // Construct and broadcast the relation. + val relation = mode.transform(input) + val beforeBroadcast = System.nanoTime() + longMetric("buildTime") += (beforeBroadcast - beforeBuild) / 1000000 + + val broadcasted = sparkContext.broadcast(relation) + longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000 + + // There are some cases we don't care about the metrics and call `SparkPlan.doExecute` + // directly without setting an execution id. We should be tolerant to it. + if (executionId != null) { + sparkContext.listenerBus.post(SparkListenerDriverAccumUpdates( + executionId.toLong, metrics.values.map(m => m.id -> m.value).toSeq)) + } + + broadcasted + } catch { + case oe: OutOfMemoryError => + throw new OutOfMemoryError(s"Not enough memory to build and broadcast the table to " + + s"all worker nodes. As a workaround, you can either disable broadcast by setting " + + s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark driver " + + s"memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value") + .initCause(oe.getCause) } - val beforeBuild = System.nanoTime() - longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 - val dataSize = input.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum - longMetric("dataSize") += dataSize - if (dataSize >= (8L << 30)) { - throw new SparkException( - s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") - } - - // Construct and broadcast the relation. - val relation = mode.transform(input) - val beforeBroadcast = System.nanoTime() - longMetric("buildTime") += (beforeBroadcast - beforeBuild) / 1000000 - - val broadcasted = sparkContext.broadcast(relation) - longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000 - - // There are some cases we don't care about the metrics and call `SparkPlan.doExecute` - // directly without setting an execution id. We should be tolerant to it. - if (executionId != null) { - sparkContext.listenerBus.post(SparkListenerDriverAccumUpdates( - executionId.toLong, metrics.values.map(m => m.id -> m.value).toSeq)) - } - - broadcasted } }(BroadcastExchangeExec.executionContext) } From 7afab774edba93bf84f4339d1561652555a3d939 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 11 Sep 2016 21:51:22 -0700 Subject: [PATCH 14/83] [SPARK-17486] Remove unused TaskMetricsUIData.updatedBlockStatuses field The `TaskMetricsUIData.updatedBlockStatuses` field is assigned to but never read, increasing the memory consumption of the web UI. We should remove this field. Author: Josh Rosen Closes #15038 from JoshRosen/remove-updated-block-statuses-from-TaskMetricsUIData. --- core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 66b88129ee414..74bca9931acf7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -23,7 +23,6 @@ import scala.collection.mutable.{HashMap, LinkedHashMap} import org.apache.spark.JobExecutionStatus import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} -import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet @@ -145,7 +144,6 @@ private[spark] object UIData { memoryBytesSpilled = m.memoryBytesSpilled, diskBytesSpilled = m.diskBytesSpilled, peakExecutionMemory = m.peakExecutionMemory, - updatedBlockStatuses = m.updatedBlockStatuses.toList, inputMetrics = InputMetricsUIData(m.inputMetrics.bytesRead, m.inputMetrics.recordsRead), outputMetrics = OutputMetricsUIData(m.outputMetrics.bytesWritten, m.outputMetrics.recordsWritten), @@ -193,7 +191,6 @@ private[spark] object UIData { memoryBytesSpilled: Long, diskBytesSpilled: Long, peakExecutionMemory: Long, - updatedBlockStatuses: Seq[(BlockId, BlockStatus)], inputMetrics: InputMetricsUIData, outputMetrics: OutputMetricsUIData, shuffleReadMetrics: ShuffleReadMetricsUIData, From d608f997d9f5c25e2947fd31a0c447488daa0c08 Mon Sep 17 00:00:00 2001 From: cenyuhai Date: Mon, 12 Sep 2016 11:52:56 +0100 Subject: [PATCH 15/83] [SPARK-17171][WEB UI] DAG will list all partitions in the graph MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? DAG will list all partitions in the graph, it is too slow and hard to see all graph. Always we don't want to see all partitions,we just want to see the relations of DAG graph. So I just show 2 root nodes for Rdds. Before this PR, the DAG graph looks like [dag1.png](https://issues.apache.org/jira/secure/attachment/12824702/dag1.png), [dag3.png](https://issues.apache.org/jira/secure/attachment/12825456/dag3.png), after this PR, the DAG graph looks like [dag2.png](https://issues.apache.org/jira/secure/attachment/12824703/dag2.png),[dag4.png](https://issues.apache.org/jira/secure/attachment/12825457/dag4.png) Author: cenyuhai Author: 岑玉海 <261810726@qq.com> Closes #14737 from cenyuhai/SPARK-17171. --- .../spark/ui/scope/RDDOperationGraph.scala | 35 +++++++++++++++---- .../ui/scope/RDDOperationGraphListener.scala | 6 +++- 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala index 84ca750e1a96a..0e330879d50f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraph.scala @@ -26,7 +26,7 @@ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.internal.Logging import org.apache.spark.scheduler.StageInfo -import org.apache.spark.storage.StorageLevel +import org.apache.spark.storage.{RDDInfo, StorageLevel} /** * A representation of a generic cluster graph used for storing information on RDD operations. @@ -107,7 +107,7 @@ private[ui] object RDDOperationGraph extends Logging { * supporting in the future if we decide to group certain stages within the same job under * a common scope (e.g. part of a SQL query). */ - def makeOperationGraph(stage: StageInfo): RDDOperationGraph = { + def makeOperationGraph(stage: StageInfo, retainedNodes: Int): RDDOperationGraph = { val edges = new ListBuffer[RDDOperationEdge] val nodes = new mutable.HashMap[Int, RDDOperationNode] val clusters = new mutable.HashMap[String, RDDOperationCluster] // indexed by cluster ID @@ -119,18 +119,37 @@ private[ui] object RDDOperationGraph extends Logging { { if (stage.attemptId == 0) "" else s" (attempt ${stage.attemptId})" } val rootCluster = new RDDOperationCluster(stageClusterId, stageClusterName) + var rootNodeCount = 0 + val addRDDIds = new mutable.HashSet[Int]() + val dropRDDIds = new mutable.HashSet[Int]() + // Find nodes, edges, and operation scopes that belong to this stage - stage.rddInfos.foreach { rdd => - edges ++= rdd.parentIds.map { parentId => RDDOperationEdge(parentId, rdd.id) } + stage.rddInfos.sortBy(_.id).foreach { rdd => + val parentIds = rdd.parentIds + val isAllowed = + if (parentIds.isEmpty) { + rootNodeCount += 1 + rootNodeCount <= retainedNodes + } else { + parentIds.exists(id => addRDDIds.contains(id) || !dropRDDIds.contains(id)) + } + + if (isAllowed) { + addRDDIds += rdd.id + edges ++= parentIds.filter(id => !dropRDDIds.contains(id)).map(RDDOperationEdge(_, rdd.id)) + } else { + dropRDDIds += rdd.id + } // TODO: differentiate between the intention to cache an RDD and whether it's actually cached val node = nodes.getOrElseUpdate(rdd.id, RDDOperationNode( rdd.id, rdd.name, rdd.storageLevel != StorageLevel.NONE, rdd.callSite)) - if (rdd.scope.isEmpty) { // This RDD has no encompassing scope, so we put it directly in the root cluster // This should happen only if an RDD is instantiated outside of a public RDD API - rootCluster.attachChildNode(node) + if (isAllowed) { + rootCluster.attachChildNode(node) + } } else { // Otherwise, this RDD belongs to an inner cluster, // which may be nested inside of other clusters @@ -154,7 +173,9 @@ private[ui] object RDDOperationGraph extends Logging { rootCluster.attachChildCluster(cluster) } } - rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) } + if (isAllowed) { + rddClusters.lastOption.foreach { cluster => cluster.attachChildNode(node) } + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala index bcae56e2f114c..37a12a8646938 100644 --- a/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/scope/RDDOperationGraphListener.scala @@ -41,6 +41,10 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen private[ui] val jobIds = new mutable.ArrayBuffer[Int] private[ui] val stageIds = new mutable.ArrayBuffer[Int] + // How many root nodes to retain in DAG Graph + private[ui] val retainedNodes = + conf.getInt("spark.ui.dagGraph.retainedRootRDDs", Int.MaxValue) + // How many jobs or stages to retain graph metadata for private val retainedJobs = conf.getInt("spark.ui.retainedJobs", SparkUI.DEFAULT_RETAINED_JOBS) @@ -82,7 +86,7 @@ private[ui] class RDDOperationGraphListener(conf: SparkConf) extends SparkListen val stageId = stageInfo.stageId stageIds += stageId stageIdToJobId(stageId) = jobId - stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo) + stageIdToGraph(stageId) = RDDOperationGraph.makeOperationGraph(stageInfo, retainedNodes) trimStagesIfNecessary() } From 10b1feb81466cfed9e7fe6a05a579ebeba3ac0d9 Mon Sep 17 00:00:00 2001 From: codlife <1004910847@qq.com> Date: Mon, 12 Sep 2016 12:10:46 +0100 Subject: [PATCH 16/83] [SPARK-17447] Performance improvement in Partitioner.defaultPartitioner without sortBy ## What changes were proposed in this pull request? if there are many rdds in some situations,the sort will loss he performance servely,actually we needn't sort the rdds , we can just scan the rdds one time to gain the same goal. ## How was this patch tested? manual tests Author: codlife <1004910847@qq.com> Closes #15039 from codlife/master. --- .../scala/org/apache/spark/Partitioner.scala | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 98c3abe93b553..93dfbc0e6ed65 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -55,14 +55,16 @@ object Partitioner { * We use two method parameters (rdd, others) to enforce callers passing at least 1 RDD. */ def defaultPartitioner(rdd: RDD[_], others: RDD[_]*): Partitioner = { - val bySize = (Seq(rdd) ++ others).sortBy(_.partitions.length).reverse - for (r <- bySize if r.partitioner.isDefined && r.partitioner.get.numPartitions > 0) { - return r.partitioner.get - } - if (rdd.context.conf.contains("spark.default.parallelism")) { - new HashPartitioner(rdd.context.defaultParallelism) + val rdds = (Seq(rdd) ++ others) + val hasPartitioner = rdds.filter(_.partitioner.exists(_.numPartitions > 0)) + if (hasPartitioner.nonEmpty) { + hasPartitioner.maxBy(_.partitions.length).partitioner.get } else { - new HashPartitioner(bySize.head.partitions.length) + if (rdd.context.conf.contains("spark.default.parallelism")) { + new HashPartitioner(rdd.context.defaultParallelism) + } else { + new HashPartitioner(rdds.map(_.partitions.length).max) + } } } } From 28f6379bf2dca6ad5975a8284672853aa68ba3b5 Mon Sep 17 00:00:00 2001 From: Gaetan Semet Date: Mon, 12 Sep 2016 12:21:33 +0100 Subject: [PATCH 17/83] [SPARK-16992][PYSPARK] use map comprehension in doc Code is equivalent, but map comprehency is most of the time faster than a map. Author: Gaetan Semet Closes #14863 from Stibbons/map_comprehension. --- examples/src/main/python/ml/quantile_discretizer_example.py | 2 +- examples/src/main/python/ml/vector_slicer_example.py | 4 ++-- examples/src/main/python/sql/hive.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/src/main/python/ml/quantile_discretizer_example.py b/examples/src/main/python/ml/quantile_discretizer_example.py index 788a0baffebb4..0fc1d1949a77d 100644 --- a/examples/src/main/python/ml/quantile_discretizer_example.py +++ b/examples/src/main/python/ml/quantile_discretizer_example.py @@ -29,7 +29,7 @@ .getOrCreate() # $example on$ - data = [(0, 18.0,), (1, 19.0,), (2, 8.0,), (3, 5.0,), (4, 2.2,)] + data = [(0, 18.0), (1, 19.0), (2, 8.0), (3, 5.0), (4, 2.2)] df = spark.createDataFrame(data, ["id", "hour"]) # $example off$ diff --git a/examples/src/main/python/ml/vector_slicer_example.py b/examples/src/main/python/ml/vector_slicer_example.py index d2f46b190f9a8..68c8cfe27e375 100644 --- a/examples/src/main/python/ml/vector_slicer_example.py +++ b/examples/src/main/python/ml/vector_slicer_example.py @@ -32,8 +32,8 @@ # $example on$ df = spark.createDataFrame([ - Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3}),), - Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]),)]) + Row(userFeatures=Vectors.sparse(3, {0: -2.0, 1: 2.3})), + Row(userFeatures=Vectors.dense([-2.0, 2.3, 0.0]))]) slicer = VectorSlicer(inputCol="userFeatures", outputCol="features", indices=[1]) diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 9b2a2c4e6a16b..98b48908b5a12 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -79,7 +79,7 @@ # You can also use DataFrames to create temporary views within a SparkSession. Record = Row("key", "value") - recordsDF = spark.createDataFrame(map(lambda i: Record(i, "val_" + str(i)), range(1, 101))) + recordsDF = spark.createDataFrame([Record(i, "val_" + str(i)) for i in range(1, 101)]) recordsDF.createOrReplaceTempView("records") # Queries can then join DataFrame data with data stored in Hive. From 7037d03a436ba4b299d6eab121ccd68a95023149 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Mon, 12 Sep 2016 12:23:16 +0100 Subject: [PATCH 18/83] [SPARK CORE][MINOR] fix "default partitioner cannot partition array keys" error message in PairRDDfunctions ## What changes were proposed in this pull request? In order to avoid confusing user, error message in `PairRDDfunctions` `Default partitioner cannot partition array keys.` is updated, the one in `partitionBy` is replaced with `Specified partitioner cannot partition array keys.` other is replaced with `Specified or default partitioner cannot partition array keys.` ## How was this patch tested? N/A Author: WeichenXu Closes #15045 from WeichenXu123/fix_partitionBy_error_message. --- .../scala/org/apache/spark/rdd/PairRDDFunctions.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 7d6a8805bc016..068f4ed8ad745 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -83,7 +83,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) throw new SparkException("Cannot use map-side combining with array keys.") } if (partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } } val aggregator = new Aggregator[K, V, C]( @@ -530,7 +530,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = self.withScope { if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } if (self.partitioner == Some(partitioner)) { self @@ -784,7 +784,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2], Iterable[W3]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2, other3), partitioner) cg.mapValues { case Array(vs, w1s, w2s, w3s) => @@ -802,7 +802,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) cg.mapValues { case Array(vs, w1s) => @@ -817,7 +817,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = self.withScope { if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { - throw new SparkException("Default partitioner cannot partition array keys.") + throw new SparkException("HashPartitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Array(vs, w1s, w2s) => From cb6e59c9bcd2aac96d7a3f9e62919150328a1595 Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Mon, 12 Sep 2016 11:30:06 -0700 Subject: [PATCH 19/83] [SPARK-17503][CORE] Fix memory leak in Memory store when unable to cache the whole RDD in memory ## What changes were proposed in this pull request? MemoryStore may throw OutOfMemoryError when trying to cache a super big RDD that cannot fit in memory. ``` scala> sc.parallelize(1 to 1000000000, 100).map(x => new Array[Long](1000)).cache().count() java.lang.OutOfMemoryError: Java heap space at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:24) at $line14.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$1.apply(:23) at scala.collection.Iterator$$anon$11.next(Iterator.scala:409) at scala.collection.Iterator$JoinIterator.next(Iterator.scala:232) at org.apache.spark.storage.memory.PartiallyUnrolledIterator.next(MemoryStore.scala:683) at org.apache.spark.InterruptibleIterator.next(InterruptibleIterator.scala:43) at org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1684) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134) at org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:1134) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915) at org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1915) at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:70) at org.apache.spark.scheduler.Task.run(Task.scala:86) at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:274) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` Spark MemoryStore uses SizeTrackingVector as a temporary unrolling buffer to store all input values that it has read so far before transferring the values to storage memory cache. The problem is that when the input RDD is too big for caching in memory, the temporary unrolling memory SizeTrackingVector is not garbage collected in time. As SizeTrackingVector can occupy all available storage memory, it may cause the executor JVM to run out of memory quickly. More info can be found at https://issues.apache.org/jira/browse/SPARK-17503 ## How was this patch tested? Unit test and manual test. ### Before change Heap memory consumption screen shot 2016-09-12 at 4 16 15 pm Heap dump screen shot 2016-09-12 at 4 34 19 pm ### After change Heap memory consumption screen shot 2016-09-12 at 4 29 10 pm Author: Sean Zhong Closes #15056 from clockfly/memory_store_leak. --- .../spark/storage/memory/MemoryStore.scala | 40 +++++++----- .../PartiallyUnrolledIteratorSuite.scala | 61 +++++++++++++++++++ 2 files changed, 87 insertions(+), 14 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index d220ab51d115b..1a3bf2bb672c6 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -663,31 +663,43 @@ private[spark] class MemoryStore( private[storage] class PartiallyUnrolledIterator[T]( memoryStore: MemoryStore, unrollMemory: Long, - unrolled: Iterator[T], + private[this] var unrolled: Iterator[T], rest: Iterator[T]) extends Iterator[T] { - private[this] var unrolledIteratorIsConsumed: Boolean = false - private[this] var iter: Iterator[T] = { - val completionIterator = CompletionIterator[T, Iterator[T]](unrolled, { - unrolledIteratorIsConsumed = true - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) - }) - completionIterator ++ rest + private def releaseUnrollMemory(): Unit = { + memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) + // SPARK-17503: Garbage collects the unrolling memory before the life end of + // PartiallyUnrolledIterator. + unrolled = null } - override def hasNext: Boolean = iter.hasNext - override def next(): T = iter.next() + override def hasNext: Boolean = { + if (unrolled == null) { + rest.hasNext + } else if (!unrolled.hasNext) { + releaseUnrollMemory() + rest.hasNext + } else { + true + } + } + + override def next(): T = { + if (unrolled == null) { + rest.next() + } else { + unrolled.next() + } + } /** * Called to dispose of this iterator and free its memory. */ def close(): Unit = { - if (!unrolledIteratorIsConsumed) { - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) - unrolledIteratorIsConsumed = true + if (unrolled != null) { + releaseUnrollMemory() } - iter = null } } diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala new file mode 100644 index 0000000000000..02c2331dc3946 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.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.storage + +import org.mockito.Matchers +import org.mockito.Mockito._ +import org.scalatest.mock.MockitoSugar + +import org.apache.spark.SparkFunSuite +import org.apache.spark.memory.MemoryMode.ON_HEAP +import org.apache.spark.storage.memory.{MemoryStore, PartiallyUnrolledIterator} + +class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { + test("join two iterators") { + val unrollSize = 1000 + val unroll = (0 until unrollSize).iterator + val restSize = 500 + val rest = (unrollSize until restSize + unrollSize).iterator + + val memoryStore = mock[MemoryStore] + val joinIterator = new PartiallyUnrolledIterator(memoryStore, unrollSize, unroll, rest) + + // Firstly iterate over unrolling memory iterator + (0 until unrollSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.hasNext + joinIterator.hasNext + verify(memoryStore, times(1)) + .releaseUnrollMemoryForThisTask(Matchers.eq(ON_HEAP), Matchers.eq(unrollSize.toLong)) + + // Secondly, iterate over rest iterator + (unrollSize until unrollSize + restSize).foreach { value => + assert(joinIterator.hasNext) + assert(joinIterator.hasNext) + assert(joinIterator.next() == value) + } + + joinIterator.close() + // MemoryMode.releaseUnrollMemoryForThisTask is called only once + verifyNoMoreInteractions(memoryStore) + } +} From 198428945c780092cc42534282e744b3fdb4e338 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 13:09:33 -0700 Subject: [PATCH 20/83] [SPARK-17483] Refactoring in BlockManager status reporting and block removal This patch makes three minor refactorings to the BlockManager: - Move the `if (info.tellMaster)` check out of `reportBlockStatus`; this fixes an issue where a debug logging message would incorrectly claim to have reported a block status to the master even though no message had been sent (in case `info.tellMaster == false`). This also makes it easier to write code which unconditionally sends block statuses to the master (which is necessary in another patch of mine). - Split `removeBlock()` into two methods, the existing method and an internal `removeBlockInternal()` method which is designed to be called by internal code that already holds a write lock on the block. This is also needed by a followup patch. - Instead of calling `getCurrentBlockStatus()` in `removeBlock()`, just pass `BlockStatus.empty`; the block status should always be empty following complete removal of a block. These changes were originally authored as part of a bug fix patch which is targeted at branch-2.0 and master; I've split them out here into their own separate PR in order to make them easier to review and so that the behavior-changing parts of my other patch can be isolated to their own PR. Author: Josh Rosen Closes #15036 from JoshRosen/cache-failure-race-conditions-refactorings-only. --- .../apache/spark/storage/BlockManager.scala | 87 +++++++++---------- 1 file changed, 42 insertions(+), 45 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 0614646771bd0..9e63777caf037 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -217,7 +217,7 @@ private[spark] class BlockManager( logInfo(s"Reporting ${blockInfoManager.size} blocks to the master.") for ((blockId, info) <- blockInfoManager.entries) { val status = getCurrentBlockStatus(blockId, info) - if (!tryToReportBlockStatus(blockId, info, status)) { + if (info.tellMaster && !tryToReportBlockStatus(blockId, status)) { logError(s"Failed to report $blockId to master; giving up.") return } @@ -298,7 +298,7 @@ private[spark] class BlockManager( /** * Get the BlockStatus for the block identified by the given ID, if it exists. - * NOTE: This is mainly for testing, and it doesn't fetch information from external block store. + * NOTE: This is mainly for testing. */ def getStatus(blockId: BlockId): Option[BlockStatus] = { blockInfoManager.get(blockId).map { info => @@ -333,10 +333,9 @@ private[spark] class BlockManager( */ private def reportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Unit = { - val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) + val needReregister = !tryToReportBlockStatus(blockId, status, droppedMemorySize) if (needReregister) { logInfo(s"Got told to re-register updating block $blockId") // Re-registering will report our new block for free. @@ -352,17 +351,12 @@ private[spark] class BlockManager( */ private def tryToReportBlockStatus( blockId: BlockId, - info: BlockInfo, status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - if (info.tellMaster) { - val storageLevel = status.storageLevel - val inMemSize = Math.max(status.memSize, droppedMemorySize) - val onDiskSize = status.diskSize - master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) - } else { - true - } + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) } /** @@ -374,7 +368,7 @@ private[spark] class BlockManager( info.synchronized { info.level match { case null => - BlockStatus(StorageLevel.NONE, memSize = 0L, diskSize = 0L) + BlockStatus.empty case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) @@ -807,12 +801,10 @@ private[spark] class BlockManager( // Now that the block is in either the memory or disk store, // tell the master about it. info.size = size - if (tellMaster) { - reportBlockStatus(blockId, info, putBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> putBlockStatus) + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) } logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { @@ -961,15 +953,12 @@ private[spark] class BlockManager( val putBlockStatus = getCurrentBlockStatus(blockId, info) val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid if (blockWasSuccessfullyStored) { - // Now that the block is in either the memory, externalBlockStore, or disk store, - // tell the master about it. + // Now that the block is in either the memory or disk store, tell the master about it. info.size = size - if (tellMaster) { - reportBlockStatus(blockId, info, putBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> putBlockStatus) + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) if (level.replication > 1) { val remoteStartTime = System.currentTimeMillis @@ -1271,12 +1260,10 @@ private[spark] class BlockManager( val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster) { - reportBlockStatus(blockId, info, status, droppedMemorySize) + reportBlockStatus(blockId, status, droppedMemorySize) } if (blockIsUpdated) { - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> status) - } + addUpdatedBlockStatusToTaskMetrics(blockId, status) } status.storageLevel } @@ -1316,21 +1303,31 @@ private[spark] class BlockManager( // The block has already been removed; do nothing. logWarning(s"Asked to remove block $blockId, which does not exist") case Some(info) => - // Removals are idempotent in disk store and memory store. At worst, we get a warning. - val removedFromMemory = memoryStore.remove(blockId) - val removedFromDisk = diskStore.remove(blockId) - if (!removedFromMemory && !removedFromDisk) { - logWarning(s"Block $blockId could not be removed as it was not found in either " + - "the disk, memory, or external block store") - } - blockInfoManager.removeBlock(blockId) - val removeBlockStatus = getCurrentBlockStatus(blockId, info) - if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, info, removeBlockStatus) - } - Option(TaskContext.get()).foreach { c => - c.taskMetrics().incUpdatedBlockStatuses(blockId -> removeBlockStatus) - } + removeBlockInternal(blockId, tellMaster = tellMaster && info.tellMaster) + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) + } + } + + /** + * Internal version of [[removeBlock()]] which assumes that the caller already holds a write + * lock on the block. + */ + private def removeBlockInternal(blockId: BlockId, tellMaster: Boolean): Unit = { + // Removals are idempotent in disk store and memory store. At worst, we get a warning. + val removedFromMemory = memoryStore.remove(blockId) + val removedFromDisk = diskStore.remove(blockId) + if (!removedFromMemory && !removedFromDisk) { + logWarning(s"Block $blockId could not be removed as it was not found on disk or in memory") + } + blockInfoManager.removeBlock(blockId) + if (tellMaster) { + reportBlockStatus(blockId, BlockStatus.empty) + } + } + + private def addUpdatedBlockStatusToTaskMetrics(blockId: BlockId, status: BlockStatus): Unit = { + Option(TaskContext.get()).foreach { c => + c.taskMetrics().incUpdatedBlockStatuses(blockId -> status) } } From 1c60199de0bd29830ac14822c4ad8d28828f2ffd Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 15:24:33 -0700 Subject: [PATCH 21/83] [SPARK-14818] Post-2.0 MiMa exclusion and build changes This patch makes a handful of post-Spark-2.0 MiMa exclusion and build updates. It should be merged to master and a subset of it should be picked into branch-2.0 in order to test Spark 2.0.1-SNAPSHOT. - Remove the ` sketch`, `mllibLocal`, and `streamingKafka010` from the list of excluded subprojects so that MiMa checks them. - Remove now-unnecessary special-case handling of the Kafka 0.8 artifact in `mimaSettings`. - Move the exclusion added in SPARK-14743 from `v20excludes` to `v21excludes`, since that patch was only merged into master and not branch-2.0. - Add exclusions for an API change introduced by SPARK-17096 / #14675. - Add missing exclusions for the `o.a.spark.internal` and `o.a.spark.sql.internal` packages. Author: Josh Rosen Closes #15061 from JoshRosen/post-2.0-mima-changes. --- project/MimaBuild.scala | 11 ++--------- project/MimaExcludes.scala | 12 +++++++++--- project/SparkBuild.scala | 2 +- 3 files changed, 12 insertions(+), 13 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 2a989dd4f7a1d..77397eab81ede 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -88,15 +88,8 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "1.6.0" - // This check can be removed post-2.0 - val project = if (previousSparkVersion == "1.6.0" && - projectRef.project == "streaming-kafka-0-8" - ) { - "streaming-kafka" - } else { - projectRef.project - } + val previousSparkVersion = "2.0.0" + val project = projectRef.project val fullId = "spark-" + project + "_2.11" mimaDefaultSettings ++ Seq(previousArtifact := Some(organization % fullId % previousSparkVersion), diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 16f26e7d283b4..fbd78aeb20dd6 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -37,6 +37,8 @@ object MimaExcludes { // Exclude rules for 2.1.x lazy val v21excludes = v20excludes ++ { Seq( + // [SPARK-14743] Improve delegation token handling in secure cluster + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal"), // [SPARK-16199][SQL] Add a method to list the referenced columns in data source Filter ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.Filter.references"), // [SPARK-16853][SQL] Fixes encoder error in DataSet typed select @@ -51,12 +53,15 @@ object MimaExcludes { Seq( excludePackage("org.apache.spark.rpc"), excludePackage("org.spark-project.jetty"), + excludePackage("org.spark_project.jetty"), + excludePackage("org.apache.spark.internal"), excludePackage("org.apache.spark.unused"), excludePackage("org.apache.spark.unsafe"), excludePackage("org.apache.spark.memory"), excludePackage("org.apache.spark.util.collection.unsafe"), excludePackage("org.apache.spark.sql.catalyst"), excludePackage("org.apache.spark.sql.execution"), + excludePackage("org.apache.spark.sql.internal"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.mllib.feature.PCAModel.this"), ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.status.api.v1.StageData.this"), ProblemFilters.exclude[MissingMethodProblem]( @@ -787,9 +792,10 @@ object MimaExcludes { ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.parquetFile"), ProblemFilters.exclude[IncompatibleResultTypeProblem]("org.apache.spark.sql.SQLContext.applySchema") ) ++ Seq( - // [SPARK-14743] Improve delegation token handling in secure cluster - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getTimeFromNowToRenewal") - ) + // SPARK-17096: Improve exception string reported through the StreamingQueryListener + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") + ) } def excludes(version: String) = version match { diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d164ead4ba73e..a39c93e9574fa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -353,7 +353,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sketch, mllibLocal, streamingKafka010 + unsafe, tags ).contains(x) } From 671d8ec8871d5283b11c295ff5b1762f337372f2 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 12 Sep 2016 15:43:57 -0700 Subject: [PATCH 22/83] [SPARK-17485] Prevent failed remote reads of cached blocks from failing entire job ## What changes were proposed in this pull request? In Spark's `RDD.getOrCompute` we first try to read a local copy of a cached RDD block, then a remote copy, and only fall back to recomputing the block if no cached copy (local or remote) can be read. This logic works correctly in the case where no remote copies of the block exist, but if there _are_ remote copies and reads of those copies fail (due to network issues or internal Spark bugs) then the BlockManager will throw a `BlockFetchException` that will fail the task (and which could possibly fail the whole job if the read failures keep occurring). In the cases of TorrentBroadcast and task result fetching we really do want to fail the entire job in case no remote blocks can be fetched, but this logic is inappropriate for reads of cached RDD blocks because those can/should be recomputed in case cached blocks are unavailable. Therefore, I think that the `BlockManager.getRemoteBytes()` method should never throw on remote fetch errors and, instead, should handle failures by returning `None`. ## How was this patch tested? Block manager changes should be covered by modified tests in `BlockManagerSuite`: the old tests expected exceptions to be thrown on failed remote reads, while the modified tests now expect `None` to be returned from the `getRemote*` method. I also manually inspected all usages of `BlockManager.getRemoteValues()`, `getRemoteBytes()`, and `get()` to verify that they correctly pattern-match on the result and handle `None`. Note that these `None` branches are already exercised because the old `getRemoteBytes` returned `None` when no remote locations for the block could be found (which could occur if an executor died and its block manager de-registered with the master). Author: Josh Rosen Closes #15037 from JoshRosen/SPARK-17485. --- .../spark/storage/BlockFetchException.scala | 24 ------------------- .../apache/spark/storage/BlockManager.scala | 5 ++-- .../spark/storage/BlockManagerSuite.scala | 10 +++----- 3 files changed, 6 insertions(+), 33 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala deleted file mode 100644 index f6e46ae9a481a..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetchException.scala +++ /dev/null @@ -1,24 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.storage - -import org.apache.spark.SparkException - -private[spark] -case class BlockFetchException(messages: String, throwable: Throwable) - extends SparkException(messages, throwable) 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 9e63777caf037..a724fdf009789 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -559,8 +559,9 @@ private[spark] class BlockManager( // Give up trying anymore locations. Either we've tried all of the original locations, // or we've refreshed the list of locations from the master, and have still // hit failures after trying locations from the refreshed list. - throw new BlockFetchException(s"Failed to fetch block after" + - s" ${totalFailureCount} fetch failures. Most recent failure cause:", e) + logWarning(s"Failed to fetch block after $totalFailureCount fetch failures. " + + s"Most recent failure cause:", e) + return None } logWarning(s"Failed to fetch remote block $blockId " + 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 87c8628ce97e9..fdf28b7dcbcf4 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -513,10 +513,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") store3.stop() store3 = null - // exception throw because there is no locations - intercept[BlockFetchException] { - store.getRemoteBytes("list1") - } + // Should return None instead of throwing an exception: + assert(store.getRemoteBytes("list1").isEmpty) } test("SPARK-14252: getOrElseUpdate should still read from remote storage") { @@ -1186,9 +1184,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) - intercept[BlockFetchException] { - store.getRemoteBytes("item") - } + assert(store.getRemoteBytes("item").isEmpty) } test("SPARK-13328: refresh block locations (fetch should succeed after location refresh)") { From 77808e00fe4780dca2ab0a5b612ed3b9f262225c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 12 Sep 2016 16:35:42 -0700 Subject: [PATCH 23/83] [SPARK-17474] [SQL] fix python udf in TakeOrderedAndProjectExec ## What changes were proposed in this pull request? When there is any Python UDF in the Project between Sort and Limit, it will be collected into TakeOrderedAndProjectExec, ExtractPythonUDFs failed to pull the Python UDFs out because QueryPlan.expressions does not include the expression inside Option[Seq[Expression]]. Ideally, we should fix the `QueryPlan.expressions`, but tried with no luck (it always run into infinite loop). In PR, I changed the TakeOrderedAndProjectExec to no use Option[Seq[Expression]] to workaround it. cc JoshRosen ## How was this patch tested? Added regression test. Author: Davies Liu Closes #15030 from davies/all_expr. --- python/pyspark/sql/tests.py | 8 ++++++++ .../apache/spark/sql/execution/SparkStrategies.scala | 8 ++++---- .../scala/org/apache/spark/sql/execution/limit.scala | 12 ++++++------ .../sql/execution/TakeOrderedAndProjectSuite.scala | 4 ++-- 4 files changed, 20 insertions(+), 12 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index fd8e9cec3e0bd..769e4540720e7 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -376,6 +376,14 @@ def test_udf_in_generate(self): row = df.select(explode(f(*df))).groupBy().sum().first() self.assertEqual(row[0], 10) + 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) + res.explain(True) + self.assertEqual(res.collect(), [Row(id=0, copy=0)]) + def test_basic_functions(self): rdd = self.sc.parallelize(['{"foo":"bar"}', '{"foo":"baz"}']) df = self.spark.read.json(rdd) 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 c389593b4f767..3441ccf53b45b 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 @@ -66,22 +66,22 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.ReturnAnswer(rootPlan) => rootPlan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => execution.TakeOrderedAndProjectExec( - limit, order, Some(projectList), planLater(child)) :: Nil + limit, order, projectList, planLater(child)) :: Nil case logical.Limit(IntegerLiteral(limit), child) => execution.CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) => - execution.TakeOrderedAndProjectExec(limit, order, None, planLater(child)) :: Nil + execution.TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil case logical.Limit( IntegerLiteral(limit), logical.Project(projectList, logical.Sort(order, true, child))) => execution.TakeOrderedAndProjectExec( - limit, order, Some(projectList), planLater(child)) :: Nil + limit, order, projectList, planLater(child)) :: Nil case _ => Nil } } 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 781c016095427..01fbe5b7c2c03 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 @@ -114,11 +114,11 @@ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { case class TakeOrderedAndProjectExec( limit: Int, sortOrder: Seq[SortOrder], - projectList: Option[Seq[NamedExpression]], + projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = { - projectList.map(_.map(_.toAttribute)).getOrElse(child.output) + projectList.map(_.toAttribute) } override def outputPartitioning: Partitioning = SinglePartition @@ -126,8 +126,8 @@ case class TakeOrderedAndProjectExec( override def executeCollect(): Array[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val data = child.execute().map(_.copy()).takeOrdered(limit)(ord) - if (projectList.isDefined) { - val proj = UnsafeProjection.create(projectList.get, child.output) + if (projectList != child.output) { + val proj = UnsafeProjection.create(projectList, child.output) data.map(r => proj(r).copy()) } else { data @@ -148,8 +148,8 @@ case class TakeOrderedAndProjectExec( localTopK, child.output, SinglePartition, serializer)) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) - if (projectList.isDefined) { - val proj = UnsafeProjection.create(projectList.get, child.output) + if (projectList != child.output) { + val proj = UnsafeProjection.create(projectList, child.output) topK.map(r => proj(r)) } else { topK diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 3217e34bd8ad3..7e317a4d80265 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -59,7 +59,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { checkThatPlansAgree( generateRandomInputData(), input => - noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, None, input)), + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, @@ -74,7 +74,7 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { generateRandomInputData(), input => noOpFilter( - TakeOrderedAndProjectExec(limit, sortOrder, Some(Seq(input.output.last)), input)), + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), input => GlobalLimitExec(limit, LocalLimitExec(limit, From 67b0a5eb09950665221b913680c4a0c81189a4e8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 13 Sep 2016 12:54:03 +0200 Subject: [PATCH 24/83] [SPARK-17515] CollectLimit.execute() should perform per-partition limits ## What changes were proposed in this pull request? CollectLimit.execute() incorrectly omits per-partition limits, leading to performance regressions in case this case is hit (which should not happen in normal operation, but can occur in some cases (see #15068 for one example). ## How was this patch tested? Regression test in SQLQuerySuite that asserts the number of records scanned from the input RDD. Author: Josh Rosen Closes #15070 from JoshRosen/SPARK-17515. --- .../scala/org/apache/spark/sql/execution/limit.scala | 3 ++- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) 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 01fbe5b7c2c03..86a8770715600 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 @@ -39,9 +39,10 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) protected override def doExecute(): RDD[InternalRow] = { + val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( ShuffleExchange.prepareShuffleDependency( - child.execute(), child.output, SinglePartition, serializer)) + locallyLimited, child.output, SinglePartition, serializer)) shuffled.mapPartitionsInternal(_.take(limit)) } } 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 eac266cba55b8..a2164f9ae3d3e 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 @@ -2661,4 +2661,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { data.selectExpr("`part.col1`", "`col.1`")) } } + + test("SPARK-17515: CollectLimit.execute() should perform per-partition limits") { + val numRecordsRead = spark.sparkContext.longAccumulator + spark.range(1, 100, 1, numPartitions = 10).map { x => + numRecordsRead.add(1) + x + }.limit(1).queryExecution.toRdd.count() + assert(numRecordsRead.value === 10) + } } From 2dda3bba9ce13dd1023422feb77953295e5fcf45 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Tue, 13 Sep 2016 17:04:51 +0200 Subject: [PATCH 25/83] [SPARK-17142][SQL] Complex query triggers binding error in HashAggregateExec ## What changes were proposed in this pull request? In `ReorderAssociativeOperator` rule, we extract foldable expressions with Add/Multiply arithmetics, and replace with eval literal. For example, `(a + 1) + (b + 2)` is optimized to `(a + b + 3)` by this rule. For aggregate operator, output expressions should be derived from groupingExpressions, current implemenation of `ReorderAssociativeOperator` rule may break this promise. A instance could be: ``` SELECT ((t1.a + 1) + (t2.a + 2)) AS out_col FROM testdata2 AS t1 INNER JOIN testdata2 AS t2 ON (t1.a = t2.a) GROUP BY (t1.a + 1), (t2.a + 2) ``` `((t1.a + 1) + (t2.a + 2))` is optimized to `(t1.a + t2.a + 3)`, which could not be derived from `ExpressionSet((t1.a +1), (t2.a + 2))`. Maybe we should improve the rule of `ReorderAssociativeOperator` by adding a GroupingExpressionSet to keep Aggregate.groupingExpressions, and respect these expressions during the optimize stage. ## How was this patch tested? Add new test case in `ReorderAssociativeOperatorSuite`. Author: jiangxingbo Closes #14917 from jiangxb1987/rao. --- .../sql/catalyst/optimizer/expressions.scala | 31 ++++++++++++++----- .../ReorderAssociativeOperatorSuite.scala | 16 +++++++++- 2 files changed, 39 insertions(+), 8 deletions(-) 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 82ab111aa2259..b7458910da13e 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 @@ -57,20 +57,37 @@ object ConstantFolding extends Rule[LogicalPlan] { * Reorder associative integral-type operators and fold all constants into one. */ object ReorderAssociativeOperator extends Rule[LogicalPlan] { - private def flattenAdd(e: Expression): Seq[Expression] = e match { - case Add(l, r) => flattenAdd(l) ++ flattenAdd(r) + private def flattenAdd( + expression: Expression, + groupSet: ExpressionSet): Seq[Expression] = expression match { + case expr @ Add(l, r) if !groupSet.contains(expr) => + flattenAdd(l, groupSet) ++ flattenAdd(r, groupSet) case other => other :: Nil } - private def flattenMultiply(e: Expression): Seq[Expression] = e match { - case Multiply(l, r) => flattenMultiply(l) ++ flattenMultiply(r) + private def flattenMultiply( + expression: Expression, + groupSet: ExpressionSet): Seq[Expression] = expression match { + case expr @ Multiply(l, r) if !groupSet.contains(expr) => + flattenMultiply(l, groupSet) ++ flattenMultiply(r, groupSet) case other => other :: Nil } + private def collectGroupingExpressions(plan: LogicalPlan): ExpressionSet = plan match { + case Aggregate(groupingExpressions, aggregateExpressions, child) => + ExpressionSet.apply(groupingExpressions) + case _ => ExpressionSet(Seq()) + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case q: LogicalPlan => q transformExpressionsDown { + case q: LogicalPlan => + // We have to respect aggregate expressions which exists in grouping expressions when plan + // is an Aggregate operator, otherwise the optimized expression could not be derived from + // grouping expressions. + val groupingExpressionSet = collectGroupingExpressions(q) + q transformExpressionsDown { case a: Add if a.deterministic && a.dataType.isInstanceOf[IntegralType] => - val (foldables, others) = flattenAdd(a).partition(_.foldable) + val (foldables, others) = flattenAdd(a, groupingExpressionSet).partition(_.foldable) if (foldables.size > 1) { val foldableExpr = foldables.reduce((x, y) => Add(x, y)) val c = Literal.create(foldableExpr.eval(EmptyRow), a.dataType) @@ -79,7 +96,7 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] { a } case m: Multiply if m.deterministic && m.dataType.isInstanceOf[IntegralType] => - val (foldables, others) = flattenMultiply(m).partition(_.foldable) + val (foldables, others) = flattenMultiply(m, groupingExpressionSet).partition(_.foldable) if (foldables.size > 1) { val foldableExpr = foldables.reduce((x, y) => Multiply(x, y)) val c = Literal.create(foldableExpr.eval(EmptyRow), m.dataType) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala index 05e15e9ec4728..a1ab0a834474f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor @@ -60,4 +60,18 @@ class ReorderAssociativeOperatorSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("nested expression with aggregate operator") { + val originalQuery = + testRelation.as("t1") + .join(testRelation.as("t2"), Inner, Some("t1.a".attr === "t2.a".attr)) + .groupBy("t1.a".attr + 1, "t2.a".attr + 1)( + (("t1.a".attr + 1) + ("t2.a".attr + 1)).as("col")) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = originalQuery.analyze + + comparePlans(optimized, correctAnswer) + } } From 33fd9a7c3b505b12eafb8ecf719f44aa356eecb6 Mon Sep 17 00:00:00 2001 From: Burak Yavuz Date: Tue, 13 Sep 2016 15:11:55 -0700 Subject: [PATCH 26/83] [SPARK-17531] Don't initialize Hive Listeners for the Execution Client ## What changes were proposed in this pull request? If a user provides listeners inside the Hive Conf, the configuration for these listeners are passed to the Hive Execution Client as well. This may cause issues for two reasons: 1. The Execution Client will actually generate garbage 2. The listener class needs to be both in the Spark Classpath and Hive Classpath This PR empties the listener configurations in `HiveUtils.newTemporaryConfiguration` so that the execution client will not contain the listener confs, but the metadata client will. ## How was this patch tested? Unit tests Author: Burak Yavuz Closes #15086 from brkyvz/null-listeners. --- .../org/apache/spark/sql/hive/HiveUtils.scala | 7 ++++ .../spark/sql/hive/HiveUtilsSuite.scala | 36 +++++++++++++++++++ 2 files changed, 43 insertions(+) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala 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 bdec611453b2d..39d71e164bf51 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 @@ -394,6 +394,13 @@ private[spark] object HiveUtils extends Logging { // hive.metastore.uris is not set. propMap.put(ConfVars.METASTOREURIS.varname, "") + // The execution client will generate garbage events, therefore the listeners that are generated + // for the execution clients are useless. In order to not output garbage, we don't generate + // these listeners. + propMap.put(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_EVENT_LISTENERS.varname, "") + propMap.put(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname, "") + propMap.toMap } 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 new file mode 100644 index 0000000000000..667a7ddd8bb61 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.QueryTest + +class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + test("newTemporaryConfiguration overwrites listener configurations") { + Seq(true, false).foreach { useInMemoryDerby => + val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) + assert(conf(ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_EVENT_LISTENERS.varname) === "") + assert(conf(ConfVars.METASTORE_END_FUNCTION_LISTENERS.varname) === "") + } + } +} From d020ea37b3f6af15d181ef28ed43129a90b427d4 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 14 Sep 2016 00:37:42 +0200 Subject: [PATCH 27/83] [SPARK-17530][SQL] Add Statistics into DESCRIBE FORMATTED ### What changes were proposed in this pull request? Statistics is missing in the output of `DESCRIBE FORMATTED`. This PR is to add it. After the PR, the output will be like: ``` +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ |key |string |null | |value |string |null | | | | | |# Detailed Table Information| | | |Database: |default | | |Owner: |xiaoli | | |Create Time: |Tue Sep 13 14:36:57 PDT 2016 | | |Last Access Time: |Wed Dec 31 16:00:00 PST 1969 | | |Location: |file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-9982e1db-df17-4376-a140-dbbee0203d83/texttable| | |Table Type: |MANAGED | | |Statistics: |sizeInBytes=5812, rowCount=500, isBroadcastable=false | | |Table Parameters: | | | | rawDataSize |-1 | | | numFiles |1 | | | transient_lastDdlTime |1473802620 | | | totalSize |5812 | | | COLUMN_STATS_ACCURATE |false | | | numRows |-1 | | | | | | |# Storage Information | | | |SerDe Library: |org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe | | |InputFormat: |org.apache.hadoop.mapred.TextInputFormat | | |OutputFormat: |org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat | | |Compressed: |No | | |Storage Desc Parameters: | | | | serialization.format |1 | | +----------------------------+----------------------------------------------------------------------------------------------------------------------+-------+ ``` Also improve the output of statistics in `DESCRIBE EXTENDED` by removing duplicate `Statistics`. Below is the example after the PR: ``` +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ |col_name |data_type |comment| +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ |key |string |null | |value |string |null | | | | | |# Detailed Table Information|CatalogTable( Table: `default`.`texttable` Owner: xiaoli Created: Tue Sep 13 14:38:43 PDT 2016 Last Access: Wed Dec 31 16:00:00 PST 1969 Type: MANAGED Schema: [StructField(key,StringType,true), StructField(value,StringType,true)] Provider: hive Properties: [rawDataSize=-1, numFiles=1, transient_lastDdlTime=1473802726, totalSize=5812, COLUMN_STATS_ACCURATE=false, numRows=-1] Statistics: sizeInBytes=5812, rowCount=500, isBroadcastable=false Storage(Location: file:/private/var/folders/4b/sgmfldk15js406vk7lw5llzw0000gn/T/warehouse-8ea5c5a0-5680-4778-91cb-c6334cf8a708/texttable, InputFormat: org.apache.hadoop.mapred.TextInputFormat, OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, Serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, Properties: [serialization.format=1]))| | +----------------------------+-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------+ ``` ### How was this patch tested? Manually tested. Author: gatorsmile Closes #15083 from gatorsmile/descFormattedStats. --- .../spark/sql/catalyst/catalog/interface.scala | 2 +- .../sql/catalyst/plans/logical/Statistics.scala | 15 ++++++++------- .../spark/sql/execution/command/tables.scala | 1 + 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index e74fa6e638a0b..e52251f960ff4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -191,7 +191,7 @@ case class CatalogTable( viewText.map("View: " + _).getOrElse(""), comment.map("Comment: " + _).getOrElse(""), if (properties.nonEmpty) s"Properties: $tableProperties" else "", - if (stats.isDefined) s"Statistics: ${stats.get}" else "", + if (stats.isDefined) s"Statistics: ${stats.get.simpleString}" else "", s"$storage") output.filter(_.nonEmpty).mkString("CatalogTable(\n\t", "\n\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 58fa537a18e3e..3cf20385dd712 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 @@ -38,12 +38,13 @@ case class Statistics( sizeInBytes: BigInt, rowCount: Option[BigInt] = None, isBroadcastable: Boolean = false) { - override def toString: String = { - val output = - Seq(s"sizeInBytes=$sizeInBytes", - if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "", - s"isBroadcastable=$isBroadcastable" - ) - output.filter(_.nonEmpty).mkString("Statistics(", ", ", ")") + override def toString: String = "Statistics(" + simpleString + ")" + + /** Readable string representation for the Statistics. */ + def simpleString: String = { + Seq(s"sizeInBytes=$sizeInBytes", + if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "", + s"isBroadcastable=$isBroadcastable" + ).filter(_.nonEmpty).mkString("", ", ", "") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 027f3588e2922..9fbcd48b4a911 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -468,6 +468,7 @@ case class DescribeTableCommand(table: TableIdentifier, isExtended: Boolean, isF append(buffer, "Last Access Time:", new Date(table.lastAccessTime).toString, "") append(buffer, "Location:", table.storage.locationUri.getOrElse(""), "") append(buffer, "Table Type:", table.tableType.name, "") + table.stats.foreach(s => append(buffer, "Statistics:", s.simpleString, "")) append(buffer, "Table Parameters:", "", "") table.properties.foreach { case (key, value) => From 9b4854614de6855eed45fb8d5260a21b17add8c4 Mon Sep 17 00:00:00 2001 From: junyangq Date: Tue, 13 Sep 2016 21:01:03 -0700 Subject: [PATCH 28/83] [SPARK-17317][SPARKR] Add SparkR vignette ## What changes were proposed in this pull request? This PR tries to add a SparkR vignette, which works as a friendly guidance going through the functionality provided by SparkR. ## How was this patch tested? Manual test. Author: junyangq Author: Shivaram Venkataraman Author: Junyang Qian Closes #14980 from junyangq/SPARKR-vignette. --- R/create-docs.sh | 11 +- R/pkg/vignettes/sparkr-vignettes.Rmd | 861 +++++++++++++++++++++++++++ 2 files changed, 870 insertions(+), 2 deletions(-) create mode 100644 R/pkg/vignettes/sparkr-vignettes.Rmd diff --git a/R/create-docs.sh b/R/create-docs.sh index d2ae160b50021..0dfba22463396 100755 --- a/R/create-docs.sh +++ b/R/create-docs.sh @@ -17,11 +17,13 @@ # limitations under the License. # -# Script to create API docs for SparkR -# This requires `devtools` and `knitr` to be installed on the machine. +# Script to create API docs and vignettes for SparkR +# This requires `devtools`, `knitr` and `rmarkdown` to be installed on the machine. # After running this script the html docs can be found in # $SPARK_HOME/R/pkg/html +# The vignettes can be found in +# $SPARK_HOME/R/pkg/vignettes/sparkr_vignettes.html set -o pipefail set -e @@ -43,4 +45,9 @@ Rscript -e 'libDir <- "../../lib"; library(SparkR, lib.loc=libDir); library(knit popd +# render creates SparkR vignettes +Rscript -e 'library(rmarkdown); paths <- .libPaths(); .libPaths(c("lib", paths)); Sys.setenv(SPARK_HOME=tools::file_path_as_absolute("..")); render("pkg/vignettes/sparkr-vignettes.Rmd"); .libPaths(paths)' + +find pkg/vignettes/. -not -name '.' -not -name '*.Rmd' -not -name '*.md' -not -name '*.pdf' -not -name '*.html' -delete + popd diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd new file mode 100644 index 0000000000000..aea52db8b8556 --- /dev/null +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -0,0 +1,861 @@ +--- +title: "SparkR - Practical Guide" +output: + html_document: + theme: united + toc: true + toc_depth: 4 + toc_float: true + highlight: textmate +--- + +## Overview + +SparkR is an R package that provides a light-weight frontend to use Apache Spark from R. With Spark `r packageVersion("SparkR")`, SparkR provides a distributed data frame implementation that supports data processing operations like selection, filtering, aggregation etc. and distributed machine learning using [MLlib](http://spark.apache.org/mllib/). + +## Getting Started + +We begin with an example running on the local machine and provide an overview of the use of SparkR: data ingestion, data processing and machine learning. + +First, let's load and attach the package. +```{r, message=FALSE} +library(SparkR) +``` + +`SparkSession` is the entry point into SparkR which connects your R program to a Spark cluster. You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any Spark packages depended on, etc. + +We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). + +```{r, message=FALSE} +sparkR.session() +``` + +The operations in SparkR are centered around an R class called `SparkDataFrame`. It is a distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R, but with richer optimizations under the hood. + +`SparkDataFrame` can be constructed from a wide array of sources such as: structured data files, tables in Hive, external databases, or existing local R data frames. For example, we create a `SparkDataFrame` from a local R data frame, + +```{r} +cars <- cbind(model = rownames(mtcars), mtcars) +carsDF <- createDataFrame(cars) +``` + +We can view the first few rows of the `SparkDataFrame` by `head` or `showDF` function. +```{r} +head(carsDF) +``` + +Common data processing operations such as `filter`, `select` are supported on the `SparkDataFrame`. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "hp") +carsSubDF <- filter(carsSubDF, carsSubDF$hp >= 200) +head(carsSubDF) +``` + +SparkR can use many common aggregation functions after grouping. + +```{r} +carsGPDF <- summarize(groupBy(carsDF, carsDF$gear), count = n(carsDF$gear)) +head(carsGPDF) +``` + +The results `carsDF` and `carsSubDF` are `SparkDataFrame` objects. To convert back to R `data.frame`, we can use `collect`. **Caution**: This can cause your interactive environment to run out of memory, though, because `collect()` fetches the entire distributed `DataFrame` to your client, which is acting as a Spark driver. +```{r} +carsGP <- collect(carsGPDF) +class(carsGP) +``` + +SparkR supports a number of commonly used machine learning algorithms. Under the hood, SparkR uses MLlib to train the model. Users can call `summary` to print a summary of the fitted model, `predict` to make predictions on new data, and `write.ml`/`read.ml` to save/load fitted models. + +SparkR supports a subset of R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘. We use linear regression as an example. +```{r} +model <- spark.glm(carsDF, mpg ~ wt + cyl) +``` + +The result matches that returned by R `glm` function applied to the corresponding `data.frame` `mtcars` of `carsDF`. In fact, for Generalized Linear Model, we specifically expose `glm` for `SparkDataFrame` as well so that the above is equivalent to `model <- glm(mpg ~ wt + cyl, data = carsDF)`. + +```{r} +summary(model) +``` + +The model can be saved by `write.ml` and loaded back using `read.ml`. +```{r, eval=FALSE} +write.ml(model, path = "/HOME/tmp/mlModel/glmModel") +``` + +In the end, we can stop Spark Session by running +```{r, eval=FALSE} +sparkR.session.stop() +``` + +## Setup + +### Installation + +Different from many other R packages, to use SparkR, you need an additional installation of Apache Spark. The Spark installation will be used to run a backend process that will compile and execute SparkR programs. + +If you don't have Spark installed on the computer, you may download it from [Apache Spark Website](http://spark.apache.org/downloads.html). Alternatively, we provide an easy-to-use function `install.spark` to complete this process. You don't have to call it explicitly. We will check the installation when `sparkR.session` is called and `install.spark` function will be triggered automatically if no installation is found. + +```{r, eval=FALSE} +install.spark() +``` + +If you already have Spark installed, you don't have to install again and can pass the `sparkHome` argument to `sparkR.session` to let SparkR know where the Spark installation is. + +```{r, eval=FALSE} +sparkR.session(sparkHome = "/HOME/spark") +``` + +### Spark Session {#SetupSparkSession} + + +In addition to `sparkHome`, many other options can be specified in `sparkR.session`. For a complete list, see [Starting up: SparkSession](http://spark.apache.org/docs/latest/sparkr.html#starting-up-sparksession) and [SparkR API doc](http://spark.apache.org/docs/latest/api/R/sparkR.session.html). + +In particular, the following Spark driver properties can be set in `sparkConfig`. + +Property Name | Property group | spark-submit equivalent +---------------- | ------------------ | ---------------------- +spark.driver.memory | Application Properties | --driver-memory +spark.driver.extraClassPath | Runtime Environment | --driver-class-path +spark.driver.extraJavaOptions | Runtime Environment | --driver-java-options +spark.driver.extraLibraryPath | Runtime Environment | --driver-library-path + +**For Windows users**: Due to different file prefixes across operating systems, to avoid the issue of potential wrong prefix, a current workaround is to specify `spark.sql.warehouse.dir` when starting the `SparkSession`. + +```{r, eval=FALSE} +spark_warehouse_path <- file.path(path.expand('~'), "spark-warehouse") +sparkR.session(spark.sql.warehouse.dir = spark_warehouse_path) +``` + + +#### Cluster Mode +SparkR can connect to remote Spark clusters. [Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) is a good introduction to different Spark cluster modes. + +When connecting SparkR to a remote Spark cluster, make sure that the Spark version and Hadoop version on the machine match the corresponding versions on the cluster. Current SparkR package is compatible with +```{r, echo=FALSE, tidy = TRUE} +paste("Spark", packageVersion("SparkR")) +``` +It should be used both on the local computer and on the remote cluster. + +To connect, pass the URL of the master node to `sparkR.session`. A complete list can be seen in [Spark Master URLs](http://spark.apache.org/docs/latest/submitting-applications.html#master-urls). +For example, to connect to a local standalone Spark master, we can call + +```{r, eval=FALSE} +sparkR.session(master = "spark://local:7077") +``` + +For YARN cluster, SparkR supports the client mode with the master set as "yarn". +```{r, eval=FALSE} +sparkR.session(master = "yarn") +``` +Yarn cluster mode is not supported in the current version. + +## Data Import + +### Local Data Frame +The simplest way is to convert a local R data frame into a `SparkDataFrame`. Specifically we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a `SparkDataFrame`. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. +```{r} +df <- as.DataFrame(faithful) +head(df) +``` + +### Data Sources +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. You can check the Spark SQL programming guide for more [specific options](https://spark.apache.org/docs/latest/sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. + +The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session'.` + +```{r, eval=FALSE} +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +``` + +We can see how to use data sources using an example CSV input file. For more information please refer to SparkR [read.df](https://spark.apache.org/docs/latest/api/R/read.df.html) API documentation. +```{r, eval=FALSE} +df <- read.df(csvPath, "csv", header = "true", inferSchema = "true", na.strings = "NA") +``` + +The data sources API natively supports JSON formatted input files. Note that the file that is used here is not a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. As a consequence, a regular multi-line JSON file will most often fail. + +Let's take a look at the first two lines of the raw JSON file used here. + +```{r} +filePath <- paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json") +readLines(filePath, n = 2L) +``` + +We use `read.df` to read that into a `SparkDataFrame`. + +```{r} +people <- read.df(filePath, "json") +count(people) +head(people) +``` + +SparkR automatically infers the schema from the JSON file. +```{r} +printSchema(people) +``` + +If we want to read multiple JSON files, `read.json` can be used. +```{r} +people <- read.json(paste0(Sys.getenv("SPARK_HOME"), + c("/examples/src/main/resources/people.json", + "/examples/src/main/resources/people.json"))) +count(people) +``` + +The data sources API can also be used to save out `SparkDataFrames` into multiple file formats. For example we can save the `SparkDataFrame` from the previous example to a Parquet file using `write.df`. +```{r, eval=FALSE} +write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite") +``` + +### Hive Tables +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with Hive support and more details can be found in the [SQL programming guide](https://spark.apache.org/docs/latest/sql-programming-guide.html). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). + +```{r, eval=FALSE} +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + +txtPath <- paste0(sparkR.conf("spark.home"), "/examples/src/main/resources/kv1.txt") +sqlCMD <- sprintf("LOAD DATA LOCAL INPATH '%s' INTO TABLE src", txtPath) +sql(sqlCMD) + +results <- sql("FROM src SELECT key, value") + +# results is now a SparkDataFrame +head(results) +``` + + +## Data Processing + +**To dplyr users**: SparkR has similar interface as dplyr in data processing. However, some noticeable differences are worth mentioning in the first place. We use `df` to represent a `SparkDataFrame` and `col` to represent the name of column here. + +1. indicate columns. SparkR uses either a character string of the column name or a Column object constructed with `$` to indicate a column. For example, to select `col` in `df`, we can write `select(df, "col")` or `select(df, df$col)`. + +2. describe conditions. In SparkR, the Column object representation can be inserted into the condition directly, or we can use a character string to describe the condition, without referring to the `SparkDataFrame` used. For example, to select rows with value > 1, we can write `filter(df, df$col > 1)` or `filter(df, "col > 1")`. + +Here are more concrete examples. + +dplyr | SparkR +-------- | --------- +`select(mtcars, mpg, hp)` | `select(carsDF, "mpg", "hp")` +`filter(mtcars, mpg > 20, hp > 100)` | `filter(carsDF, carsDF$mpg > 20, carsDF$hp > 100)` + +Other differences will be mentioned in the specific methods. + +We use the `SparkDataFrame` `carsDF` created above. We can get basic information about the `SparkDataFrame`. +```{r} +carsDF +``` + +Print out the schema in tree format. +```{r} +printSchema(carsDF) +``` + +### SparkDataFrame Operations + +#### Selecting rows, columns + +SparkDataFrames support a number of functions to do structured data processing. Here we include some basic examples and a complete list can be found in the [API](https://spark.apache.org/docs/latest/api/R/index.html) docs: + +You can also pass in column name as strings. +```{r} +head(select(carsDF, "mpg")) +``` + +Filter the SparkDataFrame to only retain rows with mpg less than 20 miles/gallon. +```{r} +head(filter(carsDF, carsDF$mpg < 20)) +``` + +#### Grouping, Aggregation + +A common flow of grouping and aggregation is + +1. Use `groupBy` or `group_by` with respect to some grouping variables to create a `GroupedData` object + +2. Feed the `GroupedData` object to `agg` or `summarize` functions, with some provided aggregation functions to compute a number within each group. + +A number of widely used functions are supported to aggregate data after grouping, including `avg`, `countDistinct`, `count`, `first`, `kurtosis`, `last`, `max`, `mean`, `min`, `sd`, `skewness`, `stddev_pop`, `stddev_samp`, `sumDistinct`, `sum`, `var_pop`, `var_samp`, `var`. See the [API doc for `mean`](http://spark.apache.org/docs/latest/api/R/mean.html) and other `agg_funcs` linked there. + +For example we can compute a histogram of the number of cylinders in the `mtcars` dataset as shown below. + +```{r} +numCyl <- summarize(groupBy(carsDF, carsDF$cyl), count = n(carsDF$cyl)) +head(numCyl) +``` + +#### 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. + +```{r} +carsDF_km <- carsDF +carsDF_km$kmpg <- carsDF_km$mpg * 1.61 +head(select(carsDF_km, "model", "mpg", "kmpg")) +``` + + +### Window Functions +A window function is a variation of aggregation function. In simple words, + +* aggregation function: `n` to `1` mapping - returns a single value for a group of entries. Examples include `sum`, `count`, `max`. + +* window function: `n` to `n` mapping - returns one value for each entry in the group, but the value may depend on all the entries of the *group*. Examples include `rank`, `lead`, `lag`. + +Formally, the *group* mentioned above is called the *frame*. Every input row can have a unique frame associated with it and the output of the window function on that row is based on the rows confined in that frame. + +Window functions are often used in conjunction with the following functions: `windowPartitionBy`, `windowOrderBy`, `partitionBy`, `orderBy`, `over`. To illustrate this we next look at an example. + +We still use the `mtcars` dataset. The corresponding `SparkDataFrame` is `carsDF`. Suppose for each number of cylinders, we want to calculate the rank of each car in `mpg` within the group. +```{r} +carsSubDF <- select(carsDF, "model", "mpg", "cyl") +ws <- orderBy(windowPartitionBy("cyl"), "mpg") +carsRank <- withColumn(carsSubDF, "rank", over(rank(), ws)) +head(carsRank, n = 20L) +``` + +We explain in detail the above steps. + +* `windowPartitionBy` creates a window specification object `WindowSpec` that defines the partition. It controls which rows will be in the same partition as the given row. In this case, rows with the same value in `cyl` will be put in the same partition. `orderBy` further defines the ordering - the position a given row is in the partition. The resulting `WindowSpec` is returned as `ws`. + +More window specification methods include `rangeBetween`, which can define boundaries of the frame by value, and `rowsBetween`, which can define the boundaries by row indices. + +* `withColumn` appends a Column called `rank` to the `SparkDataFrame`. `over` returns a windowing column. The first argument is usually a Column returned by window function(s) such as `rank()`, `lead(carsDF$wt)`. That calculates the corresponding values according to the partitioned-and-ordered table. + +### User-Defined Function + +In SparkR, we support several kinds of user-defined functions (UDFs). + +#### Apply by Partition + +`dapply` can apply a function to each partition of a `SparkDataFrame`. The function to be applied to each partition of the `SparkDataFrame` should have only one parameter, a `data.frame` corresponding to a partition, and the output should be a `data.frame` as well. Schema specifies the row format of the resulting a `SparkDataFrame`. It must match to data types of returned value. See [here](#DataTypes) for mapping between R and Spark. + +We convert `mpg` to `kmpg` (kilometers per gallon). `carsSubDF` is a `SparkDataFrame` with a subset of `carsDF` columns. + +```{r} +carsSubDF <- select(carsDF, "model", "mpg") +schema <- structType(structField("model", "string"), structField("mpg", "double"), + structField("kmpg", "double")) +out <- dapply(carsSubDF, function(x) { x <- cbind(x, x$mpg * 1.61) }, schema) +head(collect(out)) +``` + +Like `dapply`, apply a function to each partition of a `SparkDataFrame` and collect the result back. The output of function should be a `data.frame`, but no schema is required in this case. Note that `dapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +```{r} +out <- dapplyCollect( + carsSubDF, + function(x) { + x <- cbind(x, "kmpg" = x$mpg * 1.61) + }) +head(out, 3) +``` + +#### Apply by Group +`gapply` can apply a function to each group of a `SparkDataFrame`. The function is to be applied to each group of the `SparkDataFrame` and should have only two parameters: grouping key and R `data.frame` corresponding to that key. The groups are chosen from `SparkDataFrames` column(s). The output of function should be a `data.frame`. Schema specifies the row format of the resulting `SparkDataFrame`. It must represent R function’s output schema on the basis of Spark data types. The column names of the returned `data.frame` are set by user. See [here](#DataTypes) for mapping between R and Spark. + +```{r} +schema <- structType(structField("cyl", "double"), structField("max_mpg", "double")) +result <- gapply( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + }, + schema) +head(arrange(result, "max_mpg", decreasing = TRUE)) +``` + +Like gapply, `gapplyCollect` applies a function to each partition of a `SparkDataFrame` and collect the result back to R `data.frame`. The output of the function should be a `data.frame` but no schema is required in this case. Note that `gapplyCollect` can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory. + +```{r} +result <- gapplyCollect( + carsDF, + "cyl", + function(key, x) { + y <- data.frame(key, max(x$mpg)) + colnames(y) <- c("cyl", "max_mpg") + y + }) +head(result[order(result$max_mpg, decreasing = TRUE), ]) +``` + +#### Distribute Local Functions + +Similar to `lapply` in native R, `spark.lapply` runs a function over a list of elements and distributes the computations with Spark. `spark.lapply` works in a manner that is similar to `doParallel` or `lapply` to elements of a list. The results of all the computations should fit in a single machine. If that is not the case you can do something like `df <- createDataFrame(list)` and then use `dapply`. + +We use `svm` in package `e1071` as an example. We use all default settings except for varying costs of constraints violation. `spark.lapply` can train those different models in parallel. + +```{r} +costs <- exp(seq(from = log(1), to = log(1000), length.out = 5)) +train <- function(cost) { + stopifnot(requireNamespace("e1071", quietly = TRUE)) + model <- e1071::svm(Species ~ ., data = iris, cost = cost) + summary(model) +} +``` + +Return a list of model's summaries. +```{r} +model.summaries <- spark.lapply(costs, train) +``` + +```{r} +class(model.summaries) +``` + + +To avoid lengthy display, we only present the result of the second fitted model. You are free to inspect other models as well. +```{r} +print(model.summaries[[2]]) +``` + + +### SQL Queries +A `SparkDataFrame` can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The sql function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +```{r} +people <- read.df(paste0(sparkR.conf("spark.home"), + "/examples/src/main/resources/people.json"), "json") +``` + +Register this SparkDataFrame as a temporary view. + +```{r} +createOrReplaceTempView(people, "people") +``` + +SQL statements can be run by using the sql method. +```{r} +teenagers <- sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") +head(teenagers) +``` + + +## Machine Learning + +SparkR supports the following machine learning models and algorithms. + +* Generalized Linear Model (GLM) + +* Naive Bayes Model + +* $k$-means Clustering + +* Accelerated Failure Time (AFT) Survival Model + +* Gaussian Mixture Model (GMM) + +* Latent Dirichlet Allocation (LDA) + +* Multilayer Perceptron Model + +* Collaborative Filtering with Alternating Least Squares (ALS) + +* Isotonic Regression Model + +More will be added in the future. + +### R Formula + +For most above, SparkR supports **R formula operators**, including `~`, `.`, `:`, `+` and `-` for model fitting. This makes it a similar experience as using R functions. + +### Training and Test Sets + +We can easily split `SparkDataFrame` into random training and test sets by the `randomSplit` function. It returns a list of split `SparkDataFrames` with provided `weights`. We use `carsDF` as an example and want to have about $70%$ training data and $30%$ test data. +```{r} +splitDF_list <- randomSplit(carsDF, c(0.7, 0.3), seed = 0) +carsDF_train <- splitDF_list[[1]] +carsDF_test <- splitDF_list[[2]] +``` + +```{r} +count(carsDF_train) +head(carsDF_train) +``` + +```{r} +count(carsDF_test) +head(carsDF_test) +``` + + +### Models and Algorithms + +#### Generalized Linear Model + +The main function is `spark.glm`. The following families and link functions are supported. The default is gaussian. + +Family | Link Function +------ | --------- +gaussian | identity, log, inverse +binomial | logit, probit, cloglog (complementary log-log) +poisson | log, identity, sqrt +gamma | inverse, identity, log + +There are three ways to specify the `family` argument. + +* Family name as a character string, e.g. `family = "gaussian"`. + +* Family function, e.g. `family = binomial`. + +* Result returned by a family function, e.g. `family = poisson(link = log)` + +For more information regarding the families and their link functions, see the Wikipedia page [Generalized Linear Model](https://en.wikipedia.org/wiki/Generalized_linear_model). + +We use the `mtcars` dataset as an illustration. The corresponding `SparkDataFrame` is `carsDF`. After fitting the model, we print out a summary and see the fitted values by making predictions on the original dataset. We can also pass into a new `SparkDataFrame` of same schema to predict on new data. + +```{r} +gaussianGLM <- spark.glm(carsDF, mpg ~ wt + hp) +summary(gaussianGLM) +``` +When doing prediction, a new column called `prediction` will be appended. Let's look at only a subset of columns here. +```{r} +gaussianFitted <- predict(gaussianGLM, carsDF) +head(select(gaussianFitted, "model", "prediction", "mpg", "wt", "hp")) +``` + +#### Naive Bayes Model + +Naive Bayes model assumes independence among the features. `spark.naiveBayes` fits a [Bernoulli naive Bayes model](https://en.wikipedia.org/wiki/Naive_Bayes_classifier#Bernoulli_naive_Bayes) against a SparkDataFrame. The data should be all categorical. These models are often used for document classification. + +```{r} +titanic <- as.data.frame(Titanic) +titanicDF <- createDataFrame(titanic[titanic$Freq > 0, -5]) +naiveBayesModel <- spark.naiveBayes(titanicDF, Survived ~ Class + Sex + Age) +summary(naiveBayesModel) +naiveBayesPrediction <- predict(naiveBayesModel, titanicDF) +head(select(naiveBayesPrediction, "Class", "Sex", "Age", "Survived", "prediction")) +``` + +#### k-Means Clustering + +`spark.kmeans` fits a $k$-means clustering model against a `SparkDataFrame`. As an unsupervised learning method, we don't need a response variable. Hence, the left hand side of the R formula should be left blank. The clustering is based only on the variables on the right hand side. + +```{r} +kmeansModel <- spark.kmeans(carsDF, ~ mpg + hp + wt, k = 3) +summary(kmeansModel) +kmeansPredictions <- predict(kmeansModel, carsDF) +head(select(kmeansPredictions, "model", "mpg", "hp", "wt", "prediction"), n = 20L) +``` + +#### AFT Survival Model +Survival analysis studies the expected duration of time until an event happens, and often the relationship with risk factors or treatment taken on the subject. In contrast to standard regression analysis, survival modeling has to deal with special characteristics in the data including non-negative survival time and censoring. + +Accelerated Failure Time (AFT) model is a parametric survival model for censored data that assumes the effect of a covariate is to accelerate or decelerate the life course of an event by some constant. For more information, refer to the Wikipedia page [AFT Model](https://en.wikipedia.org/wiki/Accelerated_failure_time_model) and the references there. Different from a [Proportional Hazards Model](https://en.wikipedia.org/wiki/Proportional_hazards_model) designed for the same purpose, the AFT model is easier to parallelize because each instance contributes to the objective function independently. +```{r} +library(survival) +ovarianDF <- createDataFrame(ovarian) +aftModel <- spark.survreg(ovarianDF, Surv(futime, fustat) ~ ecog_ps + rx) +summary(aftModel) +aftPredictions <- predict(aftModel, ovarianDF) +head(aftPredictions) +``` + +#### Gaussian Mixture Model + +(Coming in 2.1.0) + +`spark.gaussianMixture` fits multivariate [Gaussian Mixture Model](https://en.wikipedia.org/wiki/Mixture_model#Multivariate_Gaussian_mixture_model) (GMM) against a `SparkDataFrame`. [Expectation-Maximization](https://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm) (EM) is used to approximate the maximum likelihood estimator (MLE) of the model. + +We use a simulated example to demostrate the usage. +```{r} +X1 <- data.frame(V1 = rnorm(4), V2 = rnorm(4)) +X2 <- data.frame(V1 = rnorm(6, 3), V2 = rnorm(6, 4)) +data <- rbind(X1, X2) +df <- createDataFrame(data) +gmmModel <- spark.gaussianMixture(df, ~ V1 + V2, k = 2) +summary(gmmModel) +gmmFitted <- predict(gmmModel, df) +head(select(gmmFitted, "V1", "V2", "prediction")) +``` + + +#### Latent Dirichlet Allocation + +(Coming in 2.1.0) + +`spark.lda` fits a [Latent Dirichlet Allocation](https://en.wikipedia.org/wiki/Latent_Dirichlet_allocation) model on a `SparkDataFrame`. It is often used in topic modeling in which topics are inferred from a collection of text documents. LDA can be thought of as a clustering algorithm as follows: + +* Topics correspond to cluster centers, and documents correspond to examples (rows) in a dataset. + +* Topics and documents both exist in a feature space, where feature vectors are vectors of word counts (bag of words). + +* Rather than estimating a clustering using a traditional distance, LDA uses a function based on a statistical model of how text documents are generated. + +To use LDA, we need to specify a `features` column in `data` where each entry represents a document. There are two type options for the column: + +* character string: This can be a string of the whole document. It will be parsed automatically. Additional stop words can be added in `customizedStopWords`. + +* libSVM: Each entry is a collection of words and will be processed directly. + +There are several parameters LDA takes for fitting the model. + +* `k`: number of topics (default 10). + +* `maxIter`: maximum iterations (default 20). + +* `optimizer`: optimizer to train an LDA model, "online" (default) uses [online variational inference](https://www.cs.princeton.edu/~blei/papers/HoffmanBleiBach2010b.pdf). "em" uses [expectation-maximization](https://en.wikipedia.org/wiki/Expectation%E2%80%93maximization_algorithm). + +* `subsamplingRate`: For `optimizer = "online"`. Fraction of the corpus to be sampled and used in each iteration of mini-batch gradient descent, in range (0, 1] (default 0.05). + +* `topicConcentration`: concentration parameter (commonly named beta or eta) for the prior placed on topic distributions over terms, default -1 to set automatically on the Spark side. Use `summary` to retrieve the effective topicConcentration. Only 1-size numeric is accepted. + +* `docConcentration`: concentration parameter (commonly named alpha) for the prior placed on documents distributions over topics (theta), default -1 to set automatically on the Spark side. Use `summary` to retrieve the effective docConcentration. Only 1-size or k-size numeric is accepted. + +* `maxVocabSize`: maximum vocabulary size, default 1 << 18. + +Two more functions are provided for the fitted model. + +* `spark.posterior` returns a `SparkDataFrame` containing a column of posterior probabilities vectors named "topicDistribution". + +* `spark.perplexity` returns the log perplexity of given `SparkDataFrame`, or the log perplexity of the training data if missing argument `data`. + +For more information, see the help document `?spark.lda`. + +Let's look an artificial example. +```{r} +corpus <- data.frame(features = c( + "1 2 6 0 2 3 1 1 0 0 3", + "1 3 0 1 3 0 0 2 0 0 1", + "1 4 1 0 0 4 9 0 1 2 0", + "2 1 0 3 0 0 5 0 2 3 9", + "3 1 1 9 3 0 2 0 0 1 3", + "4 2 0 3 4 5 1 1 1 4 0", + "2 1 0 3 0 0 5 0 2 2 9", + "1 1 1 9 2 1 2 0 0 1 3", + "4 4 0 3 4 2 1 3 0 0 0", + "2 8 2 0 3 0 2 0 2 7 2", + "1 1 1 9 0 2 2 0 0 3 3", + "4 1 0 0 4 5 1 3 0 1 0")) +corpusDF <- createDataFrame(corpus) +model <- spark.lda(data = corpusDF, k = 5, optimizer = "em") +summary(model) +``` + +```{r} +posterior <- spark.posterior(model, corpusDF) +head(posterior) +``` + +```{r} +perplexity <- spark.perplexity(model, corpusDF) +perplexity +``` + + +#### Multilayer Perceptron + +(Coming in 2.1.0) + +Multilayer perceptron classifier (MLPC) is a classifier based on the [feedforward artificial neural network](https://en.wikipedia.org/wiki/Feedforward_neural_network). MLPC consists of multiple layers of nodes. Each layer is fully connected to the next layer in the network. Nodes in the input layer represent the input data. All other nodes map inputs to outputs by a linear combination of the inputs with the node’s weights $w$ and bias $b$ and applying an activation function. This can be written in matrix form for MLPC with $K+1$ layers as follows: +$$ +y(x)=f_K(\ldots f_2(w_2^T f_1(w_1^T x + b_1) + b_2) \ldots + b_K). +$$ + +Nodes in intermediate layers use sigmoid (logistic) function: +$$ +f(z_i) = \frac{1}{1+e^{-z_i}}. +$$ + +Nodes in the output layer use softmax function: +$$ +f(z_i) = \frac{e^{z_i}}{\sum_{k=1}^N e^{z_k}}. +$$ + +The number of nodes $N$ in the output layer corresponds to the number of classes. + +MLPC employs backpropagation for learning the model. We use the logistic loss function for optimization and L-BFGS as an optimization routine. + +`spark.mlp` requires at least two columns in `data`: one named `"label"` and the other one `"features"`. The `"features"` column should be in libSVM-format. According to the description above, there are several additional parameters that can be set: + +* `layers`: integer vector containing the number of nodes for each layer. + +* `solver`: solver parameter, supported options: `"gd"` (minibatch gradient descent) or `"l-bfgs"`. + +* `maxIter`: maximum iteration number. + +* `tol`: convergence tolerance of iterations. + +* `stepSize`: step size for `"gd"`. + +* `seed`: seed parameter for weights initialization. + +#### Collaborative Filtering + +(Coming in 2.1.0) + +`spark.als` learns latent factors in [collaborative filtering](https://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) via [alternating least squares](http://dl.acm.org/citation.cfm?id=1608614). + +There are multiple options that can be configured in `spark.als`, including `rank`, `reg`, `nonnegative`. For a complete list, refer to the help file. + +```{r} +ratings <- list(list(0, 0, 4.0), list(0, 1, 2.0), list(1, 1, 3.0), list(1, 2, 4.0), + list(2, 1, 1.0), list(2, 2, 5.0)) +df <- createDataFrame(ratings, c("user", "item", "rating")) +model <- spark.als(df, "rating", "user", "item", rank = 10, reg = 0.1, nonnegative = TRUE) +``` + +Extract latent factors. +```{r} +stats <- summary(model) +userFactors <- stats$userFactors +itemFactors <- stats$itemFactors +head(userFactors) +head(itemFactors) +``` + +Make predictions. + +```{r} +predicted <- predict(model, df) +head(predicted) +``` + +#### Isotonic Regression Model + +(Coming in 2.1.0) + +`spark.isoreg` fits an [Isotonic Regression](https://en.wikipedia.org/wiki/Isotonic_regression) model against a `SparkDataFrame`. It solves a weighted univariate a regression problem under a complete order constraint. Specifically, given a set of real observed responses $y_1, \ldots, y_n$, corresponding real features $x_1, \ldots, x_n$, and optionally positive weights $w_1, \ldots, w_n$, we want to find a monotone (piecewise linear) function $f$ to minimize +$$ +\ell(f) = \sum_{i=1}^n w_i (y_i - f(x_i))^2. +$$ + +There are a few more arguments that may be useful. + +* `weightCol`: a character string specifying the weight column. + +* `isotonic`: logical value indicating whether the output sequence should be isotonic/increasing (`TRUE`) or antitonic/decreasing (`FALSE`). + +* `featureIndex`: the index of the feature on the right hand side of the formula if it is a vector column (default: 0), no effect otherwise. + +We use an artificial example to show the use. + +```{r} +y <- c(3.0, 6.0, 8.0, 5.0, 7.0) +x <- c(1.0, 2.0, 3.5, 3.0, 4.0) +w <- rep(1.0, 5) +data <- data.frame(y = y, x = x, w = w) +df <- createDataFrame(data) +isoregModel <- spark.isoreg(df, y ~ x, weightCol = "w") +isoregFitted <- predict(isoregModel, df) +head(select(isoregFitted, "x", "y", "prediction")) +``` + +In the prediction stage, based on the fitted monotone piecewise function, the rules are: + +* If the prediction input exactly matches a training feature then associated prediction is returned. In case there are multiple predictions with the same feature then one of them is returned. Which one is undefined. + +* If the prediction input is lower or higher than all training features then prediction with lowest or highest feature is returned respectively. In case there are multiple predictions with the same feature then the lowest or highest is returned respectively. + +* If the prediction input falls between two training features then prediction is treated as piecewise linear function and interpolated value is calculated from the predictions of the two closest features. In case there are multiple values with the same feature then the same rules as in previous point are used. + +For example, when the input is $3.2$, the two closest feature values are $3.0$ and $3.5$, then predicted value would be a linear interpolation between the predicted values at $3.0$ and $3.5$. + +```{r} +newDF <- createDataFrame(data.frame(x = c(1.5, 3.2))) +head(predict(isoregModel, newDF)) +``` + +#### What's More? +We also expect Decision Tree, Random Forest, Kolmogorov-Smirnov Test coming in the next version 2.1.0. + +### Model Persistence +The following example shows how to save/load an ML model by SparkR. +```{r} +irisDF <- suppressWarnings(createDataFrame(iris)) +gaussianGLM <- spark.glm(irisDF, Sepal_Length ~ Sepal_Width + Species, family = "gaussian") + +# Save and then load a fitted MLlib model +modelPath <- tempfile(pattern = "ml", fileext = ".tmp") +write.ml(gaussianGLM, modelPath) +gaussianGLM2 <- read.ml(modelPath) + +# Check model summary +summary(gaussianGLM2) + +# Check model prediction +gaussianPredictions <- predict(gaussianGLM2, irisDF) +head(gaussianPredictions) + +unlink(modelPath) +``` + + +## Advanced Topics + +### SparkR Object Classes + +There are three main object classes in SparkR you may be working with. + +* `SparkDataFrame`: the central component of SparkR. It is an S4 class representing distributed collection of data organized into named columns, which is conceptually equivalent to a table in a relational database or a data frame in R. It has two slots `sdf` and `env`. + + `sdf` stores a reference to the corresponding Spark Dataset in the Spark JVM backend. + + `env` saves the meta-information of the object such as `isCached`. + +It can be created by data import methods or by transforming an existing `SparkDataFrame`. We can manipulate `SparkDataFrame` by numerous data processing functions and feed that into machine learning algorithms. + +* `Column`: an S4 class representing column of `SparkDataFrame`. The slot `jc` saves a reference to the corresponding Column object in the Spark JVM backend. + +It can be obtained from a `SparkDataFrame` by `$` operator, `df$col`. More often, it is used together with other functions, for example, with `select` to select particular columns, with `filter` and constructed conditions to select rows, with aggregation functions to compute aggregate statistics for each group. + +* `GroupedData`: an S4 class representing grouped data created by `groupBy` or by transforming other `GroupedData`. Its `sgd` slot saves a reference to a RelationalGroupedDataset object in the backend. + +This is often an intermediate object with group information and followed up by aggregation operations. + +### Architecture + +A complete description of architecture can be seen in reference, in particular the paper *SparkR: Scaling R Programs with Spark*. + +Under the hood of SparkR is Spark SQL engine. This avoids the overheads of running interpreted R code, and the optimized SQL execution engine in Spark uses structural information about data and computation flow to perform a bunch of optimizations to speed up the computation. + +The main method calls of actual computation happen in the Spark JVM of the driver. We have a socket-based SparkR API that allows us to invoke functions on the JVM from R. We use a SparkR JVM backend that listens on a Netty-based socket server. + +Two kinds of RPCs are supported in the SparkR JVM backend: method invocation and creating new objects. Method invocation can be done in two ways. + +* `sparkR.invokeJMethod` takes a reference to an existing Java object and a list of arguments to be passed on to the method. + +* `sparkR.invokeJStatic` takes a class name for static method and a list of arguments to be passed on to the method. + +The arguments are serialized using our custom wire format which is then deserialized on the JVM side. We then use Java reflection to invoke the appropriate method. + +To create objects, `sparkR.newJObject` is used and then similarly the appropriate constructor is invoked with provided arguments. + +Finally, we use a new R class `jobj` that refers to a Java object existing in the backend. These references are tracked on the Java side and are automatically garbage collected when they go out of scope on the R side. + +## Appendix + +### R and Spark Data Types {#DataTypes} + +R | Spark +----------- | ------------- +byte | byte +integer | integer +float | float +double | double +numeric | double +character | string +string | string +binary | binary +raw | binary +logical | boolean +POSIXct | timestamp +POSIXlt | timestamp +Date | date +array | array +list | array +env | map + +## References + +* [Spark Cluster Mode Overview](http://spark.apache.org/docs/latest/cluster-overview.html) + +* [Submitting Spark Applications](http://spark.apache.org/docs/latest/submitting-applications.html) + +* [Machine Learning Library Guide (MLlib)](http://spark.apache.org/docs/latest/ml-guide.html) + +* [SparkR: Scaling R Programs with Spark](https://people.csail.mit.edu/matei/papers/2016/sigmod_sparkr.pdf), Shivaram Venkataraman, Zongheng Yang, Davies Liu, Eric Liang, Hossein Falaki, Xiangrui Meng, Reynold Xin, Ali Ghodsi, Michael Franklin, Ion Stoica, and Matei Zaharia. SIGMOD 2016. June 2016. + +```{r, echo=FALSE} +sparkR.session.stop() +``` From 8cbdf7983cee950c5adcb15ec868747e886e8b3e Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Wed, 14 Sep 2016 09:03:16 +0100 Subject: [PATCH 29/83] =?UTF-8?q?[SPARK-17449][DOCUMENTATION]=20Relation?= =?UTF-8?q?=20between=20heartbeatInterval=20and=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? The relation between spark.network.timeout and spark.executor.heartbeatInterval should be mentioned in the document. … network timeout] Author: Jagadeesan Closes #15042 from jagadeesanas2/SPARK-17449. --- core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala | 1 + docs/configuration.md | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index c3764ac671afb..5242ab6f55235 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -32,6 +32,7 @@ import org.apache.spark.util._ * A heartbeat from executors to the driver. This is a shared message used by several internal * components to convey liveness or execution information for in-progress tasks. It will also * expire the hosts that have not heartbeated for more than spark.network.timeout. + * spark.executor.heartbeatInterval should be significantly less than spark.network.timeout. */ private[spark] case class Heartbeat( executorId: String, diff --git a/docs/configuration.md b/docs/configuration.md index ebd0aa796db08..8aea74505e28b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -987,7 +987,8 @@ Apart from these, the following properties are also available, and may be useful 10s Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress - tasks. + tasks. spark.executor.heartbeatInterval should be significantly less than + spark.network.timeout spark.files.fetchTimeout From 7cd75c06022ae17ba14dd9551681bc9d474b6a34 Mon Sep 17 00:00:00 2001 From: Sami Jaktholm Date: Wed, 14 Sep 2016 09:38:30 +0100 Subject: [PATCH 30/83] [SPARK-17525][PYTHON] Remove SparkContext.clearFiles() from the PySpark API as it was removed from the Scala API prior to Spark 2.0.0 ## What changes were proposed in this pull request? This pull request removes the SparkContext.clearFiles() method from the PySpark API as the method was removed from the Scala API in 8ce645d4eeda203cf5e100c4bdba2d71edd44e6a. Using that method in PySpark leads to an exception as PySpark tries to call the non-existent method on the JVM side. ## How was this patch tested? Existing tests (though none of them tested this particular method). Author: Sami Jaktholm Closes #15081 from sjakthol/pyspark-sc-clearfiles. --- python/pyspark/context.py | 8 -------- 1 file changed, 8 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6e9f24ef1026b..2744bb9ec04e5 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -787,14 +787,6 @@ def addFile(self, path): """ self._jsc.sc().addFile(path) - def clearFiles(self): - """ - Clear the job's list of files added by L{addFile} or L{addPyFile} so - that they do not get downloaded to any new nodes. - """ - # TODO: remove added .py or .zip files from the PYTHONPATH? - self._jsc.sc().clearFiles() - def addPyFile(self, path): """ Add a .py or .zip dependency for all tasks to be executed on this From c37fda66a8c41440fdf7b9914baa2a4c624eb31d Mon Sep 17 00:00:00 2001 From: "wm624@hotmail.com" Date: Wed, 14 Sep 2016 09:49:15 +0100 Subject: [PATCH 31/83] [CORE][DOC] remove redundant comment ## What changes were proposed in this pull request? In the comment, there is redundant `the estimated`. This PR simply remove the redundant comment and adjusts format. Author: wm624@hotmail.com Closes #15091 from wangmiao1981/comment. --- .../spark/storage/memory/MemoryStore.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 1a3bf2bb672c6..baa3fde2d05f1 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -169,12 +169,12 @@ private[spark] class MemoryStore( * temporary unroll memory used during the materialization is "transferred" to storage memory, * so we won't acquire more memory than is actually needed to store the block. * - * @return in case of success, the estimated the estimated size of the stored data. In case of - * failure, return an iterator containing the values of the block. The returned iterator - * will be backed by the combination of the partially-unrolled block and the remaining - * elements of the original input iterator. The caller must either fully consume this - * iterator or call `close()` on it in order to free the storage memory consumed by the - * partially-unrolled block. + * @return in case of success, the estimated size of the stored data. In case of failure, return + * an iterator containing the values of the block. The returned iterator will be backed + * by the combination of the partially-unrolled block and the remaining elements of the + * original input iterator. The caller must either fully consume this iterator or call + * `close()` on it in order to free the storage memory consumed by the partially-unrolled + * block. */ private[storage] def putIteratorAsValues[T]( blockId: BlockId, @@ -298,9 +298,9 @@ private[spark] class MemoryStore( * temporary unroll memory used during the materialization is "transferred" to storage memory, * so we won't acquire more memory than is actually needed to store the block. * - * @return in case of success, the estimated the estimated size of the stored data. In case of - * failure, return a handle which allows the caller to either finish the serialization - * by spilling to disk or to deserialize the partially-serialized block and reconstruct + * @return in case of success, the estimated size of the stored data. In case of failure, + * return a handle which allows the caller to either finish the serialization by + * spilling to disk or to deserialize the partially-serialized block and reconstruct * the original input iterator. The caller must either fully consume this result * iterator or call `discard()` on it in order to free the storage memory consumed by the * partially-unrolled block. From 84dced4a5770fe20efb3e571e3a5b2f4ee635df7 Mon Sep 17 00:00:00 2001 From: Ergin Seyfe Date: Wed, 14 Sep 2016 09:51:14 +0100 Subject: [PATCH 32/83] [SPARK-17480][SQL] Improve performance by removing or caching List.length which is O(n) ## What changes were proposed in this pull request? Scala's List.length method is O(N) and it makes the gatherCompressibilityStats function O(N^2). Eliminate the List.length calls by writing it in Scala way. https://github.com/scala/scala/blob/2.10.x/src/library/scala/collection/LinearSeqOptimized.scala#L36 As suggested. Extended the fix to HiveInspectors and AggregationIterator classes as well. ## How was this patch tested? Profiled a Spark job and found that CompressibleColumnBuilder is using 39% of the CPU. Out of this 39% CompressibleColumnBuilder->gatherCompressibilityStats is using 23% of it. 6.24% of the CPU is spend on List.length which is called inside gatherCompressibilityStats. After this change we started to save 6.24% of the CPU. Author: Ergin Seyfe Closes #15032 from seyfe/gatherCompressibilityStats. --- .../sql/execution/aggregate/AggregationIterator.scala | 7 ++++--- .../columnar/compression/CompressibleColumnBuilder.scala | 6 +----- .../scala/org/apache/spark/sql/hive/HiveInspectors.scala | 6 ++++-- 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index dfed084fe64a2..f335912ba2c32 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -73,9 +73,10 @@ abstract class AggregationIterator( startingInputBufferOffset: Int): Array[AggregateFunction] = { var mutableBufferOffset = 0 var inputBufferOffset: Int = startingInputBufferOffset - val functions = new Array[AggregateFunction](expressions.length) + val expressionsLength = expressions.length + val functions = new Array[AggregateFunction](expressionsLength) var i = 0 - while (i < expressions.length) { + while (i < expressionsLength) { val func = expressions(i).aggregateFunction val funcWithBoundReferences: AggregateFunction = expressions(i).mode match { case Partial | Complete if func.isInstanceOf[ImperativeAggregate] => @@ -171,7 +172,7 @@ abstract class AggregationIterator( case PartialMerge | Final => (buffer: MutableRow, row: InternalRow) => ae.merge(buffer, row) } - } + }.toArray // This projection is used to merge buffer values for all expression-based aggregates. val aggregationBufferSchema = functions.flatMap(_.aggBufferAttributes) val updateProjection = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala index 63eae1b8685ac..0f4680e502781 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/compression/CompressibleColumnBuilder.scala @@ -66,11 +66,7 @@ private[columnar] trait CompressibleColumnBuilder[T <: AtomicType] } private def gatherCompressibilityStats(row: InternalRow, ordinal: Int): Unit = { - var i = 0 - while (i < compressionEncoders.length) { - compressionEncoders(i).gatherCompressibilityStats(row, ordinal) - i += 1 - } + compressionEncoders.foreach(_.gatherCompressibilityStats(row, ordinal)) } abstract override def appendFrom(row: InternalRow, ordinal: Int): Unit = { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index bf5cc17a68f57..4e74452f6cd12 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -756,7 +756,8 @@ private[hive] trait HiveInspectors { cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { + val length = inspectors.length + while (i < length) { cache(i) = wrap(row.get(i, dataTypes(i)), inspectors(i), dataTypes(i)) i += 1 } @@ -769,7 +770,8 @@ private[hive] trait HiveInspectors { cache: Array[AnyRef], dataTypes: Array[DataType]): Array[AnyRef] = { var i = 0 - while (i < inspectors.length) { + val length = inspectors.length + while (i < length) { cache(i) = wrap(row(i), inspectors(i), dataTypes(i)) i += 1 } From 25ca2dea555211c710915a3c8be297283a449d25 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 14 Sep 2016 10:10:16 +0100 Subject: [PATCH 33/83] [SPARK-17445][DOCS] Reference an ASF page as the main place to find third-party packages ## What changes were proposed in this pull request? Point references to spark-packages.org to https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects This will be accompanied by a parallel change to the spark-website repo, and additional changes to this wiki. ## How was this patch tested? Jenkins tests. Author: Sean Owen Closes #15075 from srowen/SPARK-17445. --- CONTRIBUTING.md | 2 +- R/pkg/R/sparkR.R | 4 ++-- docs/_layouts/global.html | 2 +- docs/index.md | 2 +- docs/sparkr.md | 3 ++- docs/streaming-programming-guide.md | 2 +- .../spark/sql/execution/datasources/DataSource.scala | 7 ++++--- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 +++------ .../spark/sql/sources/ResolvedDataSourceSuite.scala | 6 +++--- 9 files changed, 18 insertions(+), 19 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index f10d7e277eea3..1a8206abe3838 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -6,7 +6,7 @@ It lists steps that are required before creating a PR. In particular, consider: - Is the change important and ready enough to ask the community to spend time reviewing? - Have you searched for existing, related JIRAs and pull requests? -- Is this a new feature that can stand alone as a package on http://spark-packages.org ? +- Is this a new feature that can stand alone as a [third party project](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects) ? - Is the change being proposed clearly explained and motivated? When you contribute code, you affirm that the contribution is your original work and that you diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 15afe01c24ed2..06015362e6bc1 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -100,7 +100,7 @@ sparkR.stop <- function() { #' @param sparkEnvir Named list of environment variables to set on worker nodes #' @param sparkExecutorEnv Named list of environment variables to be used when launching executors #' @param sparkJars Character vector of jar files to pass to the worker nodes -#' @param sparkPackages Character vector of packages from spark-packages.org +#' @param sparkPackages Character vector of package coordinates #' @seealso \link{sparkR.session} #' @rdname sparkR.init-deprecated #' @export @@ -327,7 +327,7 @@ sparkRHive.init <- function(jsc = NULL) { #' @param sparkHome Spark Home directory. #' @param sparkConfig named list of Spark configuration to set on worker nodes. #' @param sparkJars character vector of jar files to pass to the worker nodes. -#' @param sparkPackages character vector of packages from spark-packages.org +#' @param sparkPackages character vector of package coordinates #' @param enableHiveSupport enable support for Hive, fallback if not built with Hive support; once #' set, this cannot be turned off on an existing session #' @param ... named Spark properties passed to the method. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index d3bf082aa751a..ad5b5c9adfac8 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -114,7 +114,7 @@
  • Building Spark
  • Contributing to Spark
  • -
  • Supplemental Projects
  • +
  • Third Party Projects
  • diff --git a/docs/index.md b/docs/index.md index 0cb8803783a0f..a7a92f6c4f6d7 100644 --- a/docs/index.md +++ b/docs/index.md @@ -120,7 +120,7 @@ options for deployment: * [OpenStack Swift](storage-openstack-swift.html) * [Building Spark](building-spark.html): build Spark using the Maven system * [Contributing to Spark](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) -* [Supplemental Projects](https://cwiki.apache.org/confluence/display/SPARK/Supplemental+Spark+Projects): related third party Spark projects +* [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects): related third party Spark projects **External Resources:** diff --git a/docs/sparkr.md b/docs/sparkr.md index 4bbc362c52086..b881119731045 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -110,7 +110,8 @@ head(df) SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. -The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively and through [Spark Packages](http://spark-packages.org/) you can find data source connectors for popular file formats like [Avro](http://spark-packages.org/package/databricks/spark-avro). These packages can either be added by +The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. +SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects), you can find data source connectors for popular file formats like Avro. These packages can either be added by specifying `--packages` with `spark-submit` or `sparkR` commands, or if initializing SparkSession with `sparkPackages` parameter when in an interactive R shell or from RStudio.
    diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 5392b4a9bcf4b..43f1cf3e31871 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -2382,7 +2382,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - [Kafka Integration Guide](streaming-kafka-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) -* Third-party DStream data sources can be found in [Spark Packages](https://spark-packages.org/) +* Third-party DStream data sources can be found in [Third Party Projects](https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects) * API documentation - Scala docs * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and 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 71807b771a95f..825c01365dd1e 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 @@ -142,12 +142,13 @@ case class DataSource( } else if (provider.toLowerCase == "avro" || provider == "com.databricks.spark.avro") { throw new AnalysisException( - s"Failed to find data source: ${provider.toLowerCase}. Please use Spark " + - "package http://spark-packages.org/package/databricks/spark-avro") + s"Failed to find data source: ${provider.toLowerCase}. Please find an Avro " + + "package at " + + "https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects") } else { throw new ClassNotFoundException( s"Failed to find data source: $provider. Please find packages at " + - "http://spark-packages.org", + "https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects", error) } } 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 a2164f9ae3d3e..3cc3b319f5a57 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 @@ -1645,21 +1645,18 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { e = intercept[AnalysisException] { sql(s"select id from `com.databricks.spark.avro`.`file_path`") } - assert(e.message.contains("Failed to find data source: com.databricks.spark.avro. " + - "Please use Spark package http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: com.databricks.spark.avro.")) // data source type is case insensitive e = intercept[AnalysisException] { sql(s"select id from Avro.`file_path`") } - assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: avro.")) e = intercept[AnalysisException] { sql(s"select id from avro.`file_path`") } - assert(e.message.contains("Failed to find data source: avro. Please use Spark package " + - "http://spark-packages.org/package/databricks/spark-avro")) + assert(e.message.contains("Failed to find data source: avro.")) e = intercept[AnalysisException] { sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala index 5ea1f32433699..76ffb949f1293 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/ResolvedDataSourceSuite.scala @@ -74,16 +74,16 @@ class ResolvedDataSourceSuite extends SparkFunSuite { val error1 = intercept[AnalysisException] { getProvidingClass("avro") } - assert(error1.getMessage.contains("spark-packages")) + assert(error1.getMessage.contains("Failed to find data source: avro.")) val error2 = intercept[AnalysisException] { getProvidingClass("com.databricks.spark.avro") } - assert(error2.getMessage.contains("spark-packages")) + assert(error2.getMessage.contains("Failed to find data source: com.databricks.spark.avro.")) val error3 = intercept[ClassNotFoundException] { getProvidingClass("asfdwefasdfasdf") } - assert(error3.getMessage.contains("spark-packages")) + assert(error3.getMessage.contains("Failed to find data source: asfdwefasdfasdf.")) } } From 21259574100955fa646891af8f38763cd58321c8 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 14 Sep 2016 23:10:20 +0800 Subject: [PATCH 34/83] [SPARK-17409][SQL] Do Not Optimize Query in CTAS More Than Once ### What changes were proposed in this pull request? As explained in https://github.com/apache/spark/pull/14797: >Some analyzer rules have assumptions on logical plans, optimizer may break these assumption, we should not pass an optimized query plan into QueryExecution (will be analyzed again), otherwise we may some weird bugs. For example, we have a rule for decimal calculation to promote the precision before binary operations, use PromotePrecision as placeholder to indicate that this rule should not apply twice. But a Optimizer rule will remove this placeholder, that break the assumption, then the rule applied twice, cause wrong result. We should not optimize the query in CTAS more than once. For example, ```Scala spark.range(99, 101).createOrReplaceTempView("tab1") val sqlStmt = "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") checkAnswer(spark.table("tab2"), sql(sqlStmt)) ``` Before this PR, the results do not match ``` == Results == !== Correct Answer - 2 == == Spark Answer - 2 == ![100,100.000000000000000000] [100,null] [99,99.000000000000000000] [99,99.000000000000000000] ``` After this PR, the results match. ``` +---+----------------------+ |id |num | +---+----------------------+ |99 |99.000000000000000000 | |100|100.000000000000000000| +---+----------------------+ ``` In this PR, we do not treat the `query` in CTAS as a child. Thus, the `query` will not be optimized when optimizing CTAS statement. However, we still need to analyze it for normalizing and verifying the CTAS in the Analyzer. Thus, we do it in the analyzer rule `PreprocessDDL`, because so far only this rule needs the analyzed plan of the `query`. ### How was this patch tested? Added a test Author: gatorsmile Closes #15048 from gatorsmile/ctasOptimized. --- .../sql/catalyst/plans/logical/Command.scala | 7 +++++- .../analysis/UnsupportedOperationsSuite.scala | 5 +--- .../sql/execution/command/SetCommand.scala | 2 -- .../spark/sql/execution/command/cache.scala | 7 ------ .../sql/execution/command/commands.scala | 4 +--- .../sql/execution/command/databases.scala | 2 -- .../spark/sql/execution/command/ddl.scala | 6 ----- .../spark/sql/execution/datasources/ddl.scala | 12 +++++----- .../sql/execution/datasources/rules.scala | 24 ++++++++++++++----- .../spark/sql/internal/SessionState.scala | 2 +- .../sources/CreateTableAsSelectSuite.scala | 12 ++++++++++ .../spark/sql/hive/HiveSessionState.scala | 2 +- .../sql/hive/execution/HiveExplainSuite.scala | 6 ++--- 13 files changed, 49 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala index 75a5b10d9ed04..64f57835c8898 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Command.scala @@ -17,9 +17,14 @@ package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.expressions.Attribute + /** * A logical node that represents a non-query command to be executed by the system. For example, * commands can be used by parsers to represent DDL operations. Commands, unlike queries, are * eagerly executed. */ -trait Command +trait Command extends LeafNode { + final override def children: Seq[LogicalPlan] = Seq.empty + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 6df47acaba85b..ff1bb126f463d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -31,10 +31,7 @@ import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.IntegerType /** A dummy command for testing unsupported operations. */ -case class DummyCommand() extends LogicalPlan with Command { - override def output: Seq[Attribute] = Nil - override def children: Seq[LogicalPlan] = Nil -} +case class DummyCommand() extends Command class UnsupportedOperationsSuite extends SparkFunSuite { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala index b0e2d03af070d..af6def52d07d1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/SetCommand.scala @@ -129,6 +129,4 @@ case object ResetCommand extends RunnableCommand with Logging { sparkSession.sessionState.conf.clear() Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 697e2ff21159b..c31f4dc9aba4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -47,8 +46,6 @@ case class CacheTableCommand( Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } @@ -58,8 +55,6 @@ case class UncacheTableCommand(tableIdent: TableIdentifier) extends RunnableComm sparkSession.catalog.uncacheTable(tableIdent.quotedString) Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } /** @@ -71,6 +66,4 @@ case object ClearCacheCommand extends RunnableCommand { sparkSession.catalog.clearCache() Seq.empty[Row] } - - override def output: Seq[Attribute] = Seq.empty } 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 424a962b5eb1c..698c625d617fc 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 @@ -35,9 +35,7 @@ import org.apache.spark.sql.types._ * A logical command that is executed for its side-effects. `RunnableCommand`s are * wrapped in `ExecutedCommand` during execution. */ -trait RunnableCommand extends LogicalPlan with logical.Command { - override def output: Seq[Attribute] = Seq.empty - final override def children: Seq[LogicalPlan] = Seq.empty +trait RunnableCommand extends logical.Command { def run(sparkSession: SparkSession): Seq[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala index 597ec27ce6698..e5a6a5f60b8a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/databases.scala @@ -59,6 +59,4 @@ case class SetDatabaseCommand(databaseName: String) extends RunnableCommand { sparkSession.sessionState.catalog.setCurrentDatabase(databaseName) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } 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 bc1c4f85e3315..dcda2f8d1c52a 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 @@ -70,8 +70,6 @@ case class CreateDatabaseCommand( ifNotExists) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } @@ -101,8 +99,6 @@ case class DropDatabaseCommand( sparkSession.sessionState.catalog.dropDatabase(databaseName, ifExists, cascade) Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } /** @@ -126,8 +122,6 @@ case class AlterDatabasePropertiesCommand( Seq.empty[Row] } - - override val output: Seq[Attribute] = Seq.empty } /** 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 1b1e2123b7c47..fa95af2648cf9 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 @@ -20,13 +20,15 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.types._ -case class CreateTable(tableDesc: CatalogTable, mode: SaveMode, query: Option[LogicalPlan]) - extends LogicalPlan with Command { +case class CreateTable( + tableDesc: CatalogTable, + mode: SaveMode, + query: Option[LogicalPlan]) extends Command { assert(tableDesc.provider.isDefined, "The table to be created must have a provider.") if (query.isEmpty) { @@ -35,9 +37,7 @@ case class CreateTable(tableDesc: CatalogTable, mode: SaveMode, query: Option[Lo "create table without data insertion can only use ErrorIfExists or Ignore as SaveMode.") } - override def output: Seq[Attribute] = Seq.empty[Attribute] - - override def children: Seq[LogicalPlan] = query.toSeq + override def innerChildren: Seq[QueryPlan[_]] = query.toSeq } case class CreateTempViewUsing( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index fbf4063ff63b8..bd6eb6e0535ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -66,9 +66,10 @@ class ResolveDataSource(sparkSession: SparkSession) extends Rule[LogicalPlan] { } /** - * Preprocess some DDL plans, e.g. [[CreateTable]], to do some normalization and checking. + * Analyze [[CreateTable]] and do some normalization and checking. + * For CREATE TABLE AS SELECT, the SELECT query is also analyzed. */ -case class PreprocessDDL(conf: SQLConf) extends Rule[LogicalPlan] { +case class AnalyzeCreateTable(sparkSession: SparkSession) extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { // When we CREATE TABLE without specifying the table schema, we should fail the query if @@ -95,9 +96,19 @@ case class PreprocessDDL(conf: SQLConf) extends Rule[LogicalPlan] { // * can't use all table columns as partition columns. // * partition columns' type must be AtomicType. // * sort columns' type must be orderable. - case c @ CreateTable(tableDesc, mode, query) if c.childrenResolved => - val schema = if (query.isDefined) query.get.schema else tableDesc.schema - val columnNames = if (conf.caseSensitiveAnalysis) { + case c @ CreateTable(tableDesc, mode, query) => + val analyzedQuery = query.map { q => + // Analyze the query in CTAS and then we can do the normalization and checking. + val qe = sparkSession.sessionState.executePlan(q) + qe.assertAnalyzed() + qe.analyzed + } + val schema = if (analyzedQuery.isDefined) { + analyzedQuery.get.schema + } else { + tableDesc.schema + } + val columnNames = if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { schema.map(_.name) } else { schema.map(_.name.toLowerCase) @@ -106,7 +117,7 @@ case class PreprocessDDL(conf: SQLConf) extends Rule[LogicalPlan] { val partitionColsChecked = checkPartitionColumns(schema, tableDesc) val bucketColsChecked = checkBucketColumns(schema, partitionColsChecked) - c.copy(tableDesc = bucketColsChecked) + c.copy(tableDesc = bucketColsChecked, query = analyzedQuery) } private def checkPartitionColumns(schema: StructType, tableDesc: CatalogTable): CatalogTable = { @@ -176,6 +187,7 @@ case class PreprocessDDL(conf: SQLConf) extends Rule[LogicalPlan] { colName: String, colType: String): String = { val tableCols = schema.map(_.name) + val conf = sparkSession.sessionState.conf tableCols.find(conf.resolver(_, colName)).getOrElse { failAnalysis(s"$colType column $colName is not defined in table $tableIdent, " + s"defined table columns are: ${tableCols.mkString(", ")}") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index 8fdbd0f2c6dab..c899773b6b36f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -111,7 +111,7 @@ private[sql] class SessionState(sparkSession: SparkSession) { lazy val analyzer: Analyzer = { new Analyzer(catalog, conf) { override val extendedResolutionRules = - PreprocessDDL(conf) :: + AnalyzeCreateTable(sparkSession) :: PreprocessTableInsertion(conf) :: new FindDataSourceTable(sparkSession) :: DataSourceAnalysis(conf) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index 729c9fdda543e..344d4aa6cfea4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -236,4 +236,16 @@ class CreateTableAsSelectSuite assert(e.contains("Expected positive number of buckets, but got `0`")) } } + + test("CTAS of decimal calculation") { + withTable("tab2") { + withTempView("tab1") { + spark.range(99, 101).createOrReplaceTempView("tab1") + val sqlStmt = + "SELECT id, cast(id as long) * cast('1.0' as decimal(38, 18)) as num FROM tab1" + sql(s"CREATE TABLE tab2 USING PARQUET AS $sqlStmt") + checkAnswer(spark.table("tab2"), sql(sqlStmt)) + } + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala index 15e1255653f88..eb10c11382e83 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionState.scala @@ -60,7 +60,7 @@ private[hive] class HiveSessionState(sparkSession: SparkSession) override val extendedResolutionRules = catalog.ParquetConversions :: catalog.OrcConversions :: - PreprocessDDL(conf) :: + AnalyzeCreateTable(sparkSession) :: PreprocessTableInsertion(conf) :: DataSourceAnalysis(conf) :: (if (conf.runSQLonFile) new ResolveDataSource(sparkSession) :: Nil else Nil) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala index 98afd99a203ac..f9751e3d5f2eb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveExplainSuite.scala @@ -77,7 +77,7 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto "src") } - test("SPARK-6212: The EXPLAIN output of CTAS only shows the analyzed plan") { + test("SPARK-17409: The EXPLAIN output of CTAS only shows the analyzed plan") { withTempView("jt") { val rdd = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}""")) spark.read.json(rdd).createOrReplaceTempView("jt") @@ -98,8 +98,8 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } val physicalIndex = outputs.indexOf("== Physical Plan ==") - assert(!outputs.substring(physicalIndex).contains("Subquery"), - "Physical Plan should not contain Subquery since it's eliminated by optimizer") + assert(outputs.substring(physicalIndex).contains("Subquery"), + "Physical Plan should contain SubqueryAlias since the query should not be optimized") } } From f9f2c6b9f8342048329bd919cc4abc0e9a6125d3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 14 Sep 2016 10:10:01 -0700 Subject: [PATCH 35/83] [SPARK-17514] df.take(1) and df.limit(1).collect() should perform the same in Python ## What changes were proposed in this pull request? In PySpark, `df.take(1)` runs a single-stage job which computes only one partition of the DataFrame, while `df.limit(1).collect()` computes all partitions and runs a two-stage job. This difference in performance is confusing. The reason why `limit(1).collect()` is so much slower is that `collect()` internally maps to `df.rdd..toLocalIterator`, which causes Spark SQL to build a query where a global limit appears in the middle of the plan; this, in turn, ends up being executed inefficiently because limits in the middle of plans are now implemented by repartitioning to a single task rather than by running a `take()` job on the driver (this was done in #7334, a patch which was a prerequisite to allowing partition-local limits to be pushed beneath unions, etc.). In order to fix this performance problem I think that we should generalize the fix from SPARK-10731 / #8876 so that `DataFrame.collect()` also delegates to the Scala implementation and shares the same performance properties. This patch modifies `DataFrame.collect()` to first collect all results to the driver and then pass them to Python, allowing this query to be planned using Spark's `CollectLimit` optimizations. ## How was this patch tested? Added a regression test in `sql/tests.py` which asserts that the expected number of jobs, stages, and tasks are run for both queries. Author: Josh Rosen Closes #15068 from JoshRosen/pyspark-collect-limit. --- python/pyspark/sql/dataframe.py | 5 +---- python/pyspark/sql/tests.py | 18 ++++++++++++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 8 ++++++-- .../sql/execution/python/EvaluatePython.scala | 13 +------------ 4 files changed, 26 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e5eac918a93a0..0f7d8fba3bd54 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -357,10 +357,7 @@ def take(self, num): >>> df.take(2) [Row(age=2, name=u'Alice'), Row(age=5, name=u'Bob')] """ - with SCCallSiteSync(self._sc) as css: - port = self._sc._jvm.org.apache.spark.sql.execution.python.EvaluatePython.takeAndServe( - self._jdf, num) - return list(_load_from_socket(port, BatchedSerializer(PickleSerializer()))) + return self.limit(num).collect() @since(1.3) def foreach(self, f): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 769e4540720e7..1be0b72304ae8 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1862,6 +1862,24 @@ def test_collect_functions(self): sorted(df.select(functions.collect_list(df.value).alias('r')).collect()[0].r), ["1", "2", "2", "2"]) + def test_limit_and_take(self): + df = self.spark.range(1, 1000, numPartitions=10) + + def assert_runs_only_one_job_stage_and_task(job_group_name, f): + tracker = self.sc.statusTracker() + self.sc.setJobGroup(job_group_name, description="") + f() + jobs = tracker.getJobIdsForGroup(job_group_name) + self.assertEqual(1, len(jobs)) + stages = tracker.getJobInfo(jobs[0]).stageIds + self.assertEqual(1, len(stages)) + self.assertEqual(1, tracker.getStageInfo(stages[0]).numTasks) + + # Regression test for SPARK-10731: take should delegate to Scala implementation + assert_runs_only_one_job_stage_and_task("take", lambda: df.take(1)) + # Regression test for SPARK-17514: limit(n).collect() should the perform same as take(n) + assert_runs_only_one_job_stage_and_task("collect_limit", lambda: df.limit(1).collect()) + if __name__ == "__main__": from pyspark.sql.tests import * 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 3b3cb820788a2..9cfbdffd02582 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 @@ -29,7 +29,7 @@ import org.apache.commons.lang3.StringUtils import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ -import org.apache.spark.api.python.PythonRDD +import org.apache.spark.api.python.{PythonRDD, SerDeUtil} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst._ @@ -2567,8 +2567,12 @@ class Dataset[T] private[sql]( } private[sql] def collectToPython(): Int = { + EvaluatePython.registerPicklers() withNewExecutionId { - PythonRDD.collectAndServe(javaToPython.rdd) + val toJava: (Any) => Any = EvaluatePython.toJava(_, schema) + val iter = new SerDeUtil.AutoBatchedPickler( + queryExecution.executedPlan.executeCollect().iterator.map(toJava)) + PythonRDD.serveIterator(iter, "serve-DataFrame") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala index cf68ed4ec36a8..724025b4647f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvaluatePython.scala @@ -24,9 +24,8 @@ import scala.collection.JavaConverters._ import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} -import org.apache.spark.api.python.{PythonRDD, SerDeUtil} +import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} @@ -34,16 +33,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String object EvaluatePython { - def takeAndServe(df: DataFrame, n: Int): Int = { - registerPicklers() - df.withNewExecutionId { - val iter = new SerDeUtil.AutoBatchedPickler( - df.queryExecution.executedPlan.executeTake(n).iterator.map { row => - EvaluatePython.toJava(row, df.schema) - }) - PythonRDD.serveIterator(iter, s"serve-DataFrame") - } - } def needConversionInPython(dt: DataType): Boolean = dt match { case DateType | TimestampType => true From 08b9c75baf7ab5a1a2a19e0bd78b753d68acdd7c Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 15 Sep 2016 01:33:56 +0800 Subject: [PATCH 36/83] [MINOR][SQL] Add missing functions for some options in SQLConf and use them where applicable ## What changes were proposed in this pull request? I first thought they are missing because they are kind of hidden options but it seems they are just missing. For example, `spark.sql.parquet.mergeSchema` is documented in [sql-programming-guide.md](https://github.com/apache/spark/blob/master/docs/sql-programming-guide.md) but this function is missing whereas many options such as `spark.sql.join.preferSortMergeJoin` are not documented but have its own function individually. So, this PR suggests making them consistent by adding the missing functions for some options in `SQLConf` and use them where applicable, in order to make them more readable. ## How was this patch tested? Existing tests should cover this. Author: hyukjinkwon Closes #14678 from HyukjinKwon/sqlconf-cleanup. --- .../spark/sql/RelationalGroupedDataset.scala | 2 +- .../spark/sql/execution/QueryExecution.scala | 2 +- .../execution/datasources/DataSource.scala | 2 +- .../InsertIntoHadoopFsRelationCommand.scala | 2 +- .../PartitioningAwareFileCatalog.scala | 2 +- .../parquet/ParquetFileFormat.scala | 8 ++-- .../datasources/parquet/ParquetOptions.scala | 2 +- .../streaming/FileStreamSinkLog.scala | 6 +-- .../execution/streaming/StreamExecution.scala | 2 +- .../streaming/state/StateStoreConf.scala | 6 +-- .../apache/spark/sql/internal/SQLConf.scala | 42 ++++++++++++++----- .../sql/streaming/StreamingQueryManager.scala | 4 +- 12 files changed, 49 insertions(+), 31 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index 53d732403f979..6c3fe07709fa3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -313,7 +313,7 @@ class RelationalGroupedDataset protected[sql]( */ def pivot(pivotColumn: String): RelationalGroupedDataset = { // This is to prevent unintended OOM errors when the number of distinct values is large - val maxValues = df.sparkSession.conf.get(SQLConf.DATAFRAME_PIVOT_MAX_VALUES) + val maxValues = df.sparkSession.sessionState.conf.dataFramePivotMaxValues // Get the distinct values of the column and sort them so its consistent val values = df.select(pivotColumn) .distinct() 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 d4845637be049..383b3a233fc27 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 @@ -55,7 +55,7 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } def assertSupported(): Unit = { - if (sparkSession.sessionState.conf.getConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) { + if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { UnsupportedOperationChecker.checkForBatch(analyzed) } } 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 825c01365dd1e..93154bd2ca69c 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 @@ -231,7 +231,7 @@ case class DataSource( } } - val isSchemaInferenceEnabled = sparkSession.conf.get(SQLConf.STREAMING_SCHEMA_INFERENCE) + val isSchemaInferenceEnabled = sparkSession.sessionState.conf.streamingSchemaInference val isTextSource = providingClass == classOf[text.TextFileFormat] // If the schema inference is disabled, only text sources require schema to be specified if (!isSchemaInferenceEnabled && !isTextSource && userSpecifiedSchema.isEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala index 02ce7fab64729..99ca3df673568 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoHadoopFsRelationCommand.scala @@ -131,7 +131,7 @@ case class InsertIntoHadoopFsRelationCommand( dataColumns = dataColumns, inputSchema = query.output, PartitioningUtils.DEFAULT_PARTITION_NAME, - sparkSession.conf.get(SQLConf.PARTITION_MAX_FILES), + sparkSession.sessionState.conf.partitionMaxFiles, isAppend) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala index cef9d4d9c7f1b..d2d5b56c82946 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileCatalog.scala @@ -126,7 +126,7 @@ abstract class PartitioningAwareFileCatalog( PartitioningUtils.parsePartitions( leafDirs, PartitioningUtils.DEFAULT_PARTITION_NAME, - typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled(), + typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index 9208c82179d8d..e7c3545630fea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -151,7 +151,7 @@ class ParquetFileFormat // Should we merge schemas from all Parquet part-files? val shouldMergeSchemas = parquetOptions.mergeSchema - val mergeRespectSummaries = sparkSession.conf.get(SQLConf.PARQUET_SCHEMA_RESPECT_SUMMARIES) + val mergeRespectSummaries = sparkSession.sessionState.conf.isParquetSchemaRespectSummaries val filesByType = splitFiles(files) @@ -308,14 +308,14 @@ class ParquetFileFormat // Sets flags for `CatalystSchemaConverter` hadoopConf.setBoolean( SQLConf.PARQUET_BINARY_AS_STRING.key, - sparkSession.conf.get(SQLConf.PARQUET_BINARY_AS_STRING)) + sparkSession.sessionState.conf.isParquetBinaryAsString) hadoopConf.setBoolean( SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, - sparkSession.conf.get(SQLConf.PARQUET_INT96_AS_TIMESTAMP)) + sparkSession.sessionState.conf.isParquetINT96AsTimestamp) // Try to push down filters when filter push-down is enabled. val pushed = - if (sparkSession.conf.get(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key).toBoolean) { + if (sparkSession.sessionState.conf.parquetFilterPushDown) { filters // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala index 3eec582714e15..615731889dfad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOptions.scala @@ -52,7 +52,7 @@ private[parquet] class ParquetOptions( val mergeSchema: Boolean = parameters .get(MERGE_SCHEMA) .map(_.toBoolean) - .getOrElse(sqlConf.getConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED)) + .getOrElse(sqlConf.isParquetSchemaMergingEnabled) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala index 7520163522027..6f9f7c18c4dc7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSinkLog.scala @@ -93,11 +93,11 @@ class FileStreamSinkLog(sparkSession: SparkSession, path: String) * a live lock may happen if the compaction happens too frequently: one processing keeps deleting * old files while another one keeps retrying. Setting a reasonable cleanup delay could avoid it. */ - private val fileCleanupDelayMs = sparkSession.conf.get(SQLConf.FILE_SINK_LOG_CLEANUP_DELAY) + private val fileCleanupDelayMs = sparkSession.sessionState.conf.fileSinkLogCleanupDelay - private val isDeletingExpiredLog = sparkSession.conf.get(SQLConf.FILE_SINK_LOG_DELETION) + private val isDeletingExpiredLog = sparkSession.sessionState.conf.fileSinkLogDeletion - private val compactInterval = sparkSession.conf.get(SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL) + private val compactInterval = sparkSession.sessionState.conf.fileSinkLogCompatInterval require(compactInterval > 0, s"Please set ${SQLConf.FILE_SINK_LOG_COMPACT_INTERVAL.key} (was $compactInterval) " + "to a positive value.") 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 5e1e5eeb50936..a1aae61107baf 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 @@ -58,7 +58,7 @@ class StreamExecution( import org.apache.spark.sql.streaming.StreamingQueryListener._ - private val pollingDelayMs = sparkSession.conf.get(SQLConf.STREAMING_POLLING_DELAY) + private val pollingDelayMs = sparkSession.sessionState.conf.streamingPollingDelay /** * A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala index e55f63a6c8db8..de72f1cf2723d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreConf.scala @@ -24,11 +24,9 @@ private[streaming] class StateStoreConf(@transient private val conf: SQLConf) ex def this() = this(new SQLConf) - import SQLConf._ + val minDeltasForSnapshot = conf.stateStoreMinDeltasForSnapshot - val minDeltasForSnapshot = conf.getConf(STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT) - - val minVersionsToRetain = conf.getConf(STATE_STORE_MIN_VERSIONS_TO_RETAIN) + val minVersionsToRetain = conf.stateStoreMinVersionsToRetain } private[streaming] object StateStoreConf { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 1d6ca5a965cbf..428032b1fba83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -338,11 +338,6 @@ object SQLConf { .intConf .createWithDefault(4000) - val PARTITION_DISCOVERY_ENABLED = SQLConfigBuilder("spark.sql.sources.partitionDiscovery.enabled") - .doc("When true, automatically discover data partitions.") - .booleanConf - .createWithDefault(true) - val PARTITION_COLUMN_TYPE_INFERENCE = SQLConfigBuilder("spark.sql.sources.partitionColumnTypeInference.enabled") .doc("When true, automatically infer the data types for partitioned columns.") @@ -391,8 +386,10 @@ object SQLConf { val PARALLEL_PARTITION_DISCOVERY_THRESHOLD = SQLConfigBuilder("spark.sql.sources.parallelPartitionDiscovery.threshold") - .doc("The degree of parallelism for schema merging and partition discovery of " + - "Parquet data sources.") + .doc("The maximum number of files allowed for listing files at driver side. If the number " + + "of detected files exceeds this value during partition discovery, it tries to list the " + + "files with another Spark distributed job. This applies to Parquet, ORC, CSV, JSON and " + + "LibSVM data sources.") .intConf .createWithDefault(32) @@ -592,8 +589,24 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD) + def stateStoreMinDeltasForSnapshot: Int = getConf(STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT) + + def stateStoreMinVersionsToRetain: Int = getConf(STATE_STORE_MIN_VERSIONS_TO_RETAIN) + def checkpointLocation: Option[String] = getConf(CHECKPOINT_LOCATION) + def isUnsupportedOperationCheckEnabled: Boolean = getConf(UNSUPPORTED_OPERATION_CHECK_ENABLED) + + def fileSinkLogDeletion: Boolean = getConf(FILE_SINK_LOG_DELETION) + + def fileSinkLogCompatInterval: Int = getConf(FILE_SINK_LOG_COMPACT_INTERVAL) + + def fileSinkLogCleanupDelay: Long = getConf(FILE_SINK_LOG_CLEANUP_DELAY) + + def streamingSchemaInference: Boolean = getConf(STREAMING_SCHEMA_INFERENCE) + + def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY) + def filesMaxPartitionBytes: Long = getConf(FILES_MAX_PARTITION_BYTES) def filesOpenCostInBytes: Long = getConf(FILES_OPEN_COST_IN_BYTES) @@ -657,6 +670,12 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES, Long.MaxValue) + def isParquetSchemaMergingEnabled: Boolean = getConf(PARQUET_SCHEMA_MERGING_ENABLED) + + def isParquetSchemaRespectSummaries: Boolean = getConf(PARQUET_SCHEMA_RESPECT_SUMMARIES) + + def parquetOutputCommitterClass: String = getConf(PARQUET_OUTPUT_COMMITTER_CLASS) + def isParquetBinaryAsString: Boolean = getConf(PARQUET_BINARY_AS_STRING) def isParquetINT96AsTimestamp: Boolean = getConf(PARQUET_INT96_AS_TIMESTAMP) @@ -673,12 +692,11 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def convertCTAS: Boolean = getConf(CONVERT_CTAS) - def partitionDiscoveryEnabled(): Boolean = - getConf(SQLConf.PARTITION_DISCOVERY_ENABLED) - - def partitionColumnTypeInferenceEnabled(): Boolean = + def partitionColumnTypeInferenceEnabled: Boolean = getConf(SQLConf.PARTITION_COLUMN_TYPE_INFERENCE) + def partitionMaxFiles: Int = getConf(PARTITION_MAX_FILES) + def parallelPartitionDiscoveryThreshold: Int = getConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD) @@ -695,6 +713,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { def dataFrameRetainGroupColumns: Boolean = getConf(DATAFRAME_RETAIN_GROUP_COLUMNS) + def dataFramePivotMaxValues: Int = getConf(DATAFRAME_PIVOT_MAX_VALUES) + override def runSQLonFile: Boolean = getConf(RUN_SQL_ON_FILES) def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP) 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 bae7f56a23f81..bba7bc753eea9 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 @@ -204,7 +204,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) { val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified => new Path(userSpecified).toUri.toString }.orElse { - df.sparkSession.conf.get(SQLConf.CHECKPOINT_LOCATION).map { location => + df.sparkSession.sessionState.conf.checkpointLocation.map { location => new Path(location, name).toUri.toString } }.getOrElse { @@ -232,7 +232,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) { val analyzedPlan = df.queryExecution.analyzed df.queryExecution.assertAnalyzed() - if (sparkSession.conf.get(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED)) { + if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) } From a1d246eddd66a7087096418f94ee740aff37e6ec Mon Sep 17 00:00:00 2001 From: Xin Wu Date: Wed, 14 Sep 2016 21:14:29 +0200 Subject: [PATCH 37/83] [SPARK-10747][SQL] Support NULLS FIRST|LAST clause in ORDER BY ## What changes were proposed in this pull request? Currently, ORDER BY clause returns nulls value according to sorting order (ASC|DESC), considering null value is always smaller than non-null values. However, SQL2003 standard support NULLS FIRST or NULLS LAST to allow users to specify whether null values should be returned first or last, regardless of sorting order (ASC|DESC). This PR is to support this new feature. ## How was this patch tested? New test cases are added to test NULLS FIRST|LAST for regular select queries and windowing queries. (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: Xin Wu Closes #14842 from xwu0226/SPARK-10747. --- .../unsafe/sort/PrefixComparators.java | 58 +++- .../unsafe/sort/UnsafeInMemorySorter.java | 11 +- .../unsafe/sort/RadixSortSuite.scala | 27 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 7 +- .../sql/catalyst/analysis/Analyzer.scala | 4 +- .../SubstituteUnresolvedOrdinals.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../sql/catalyst/expressions/SortOrder.scala | 65 ++++- .../codegen/GenerateOrdering.scala | 16 +- .../sql/catalyst/expressions/ordering.scala | 6 +- .../sql/catalyst/parser/AstBuilder.scala | 14 +- .../spark/sql/execution/SortPrefixUtils.scala | 68 ++++- .../spark/sql/execution/SparkPlan.scala | 2 +- .../inputs/orderby-nulls-ordering.sql | 83 ++++++ .../results/orderby-nulls-ordering.sql.out | 254 ++++++++++++++++++ .../spark/sql/execution/SortSuite.scala | 3 +- sql/hive/src/test/resources/sqlgen/agg2.sql | 2 +- sql/hive/src/test/resources/sqlgen/agg3.sql | 2 +- .../sqlgen/broadcast_join_subquery.sql | 2 +- .../sqlgen/generate_with_other_1.sql | 2 +- .../sqlgen/generate_with_other_2.sql | 2 +- .../resources/sqlgen/grouping_sets_2_1.sql | 2 +- .../resources/sqlgen/grouping_sets_2_2.sql | 2 +- .../resources/sqlgen/grouping_sets_2_3.sql | 2 +- .../resources/sqlgen/grouping_sets_2_4.sql | 2 +- .../resources/sqlgen/grouping_sets_2_5.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_1.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_2.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_3.sql | 2 +- .../test/resources/sqlgen/rollup_cube_6_4.sql | 2 +- .../resources/sqlgen/sort_asc_nulls_last.sql | 4 + .../resources/sqlgen/sort_by_after_having.sql | 2 +- .../sqlgen/sort_desc_nulls_first.sql | 4 + .../resources/sqlgen/subquery_in_having_1.sql | 2 +- .../resources/sqlgen/subquery_in_having_2.sql | 2 +- .../test/resources/sqlgen/window_basic_2.sql | 2 +- .../test/resources/sqlgen/window_basic_3.sql | 2 +- .../sqlgen/window_basic_asc_nulls_last.sql | 5 + .../sqlgen/window_basic_desc_nulls_first.sql | 5 + .../resources/sqlgen/window_with_join.sql | 2 +- .../window_with_the_same_window_with_agg.sql | 2 +- ...w_with_the_same_window_with_agg_filter.sql | 2 +- ...ith_the_same_window_with_agg_functions.sql | 2 +- ...w_with_the_same_window_with_agg_having.sql | 2 +- .../catalyst/ExpressionSQLBuilderSuite.scala | 6 +- .../sql/catalyst/LogicalPlanToSQLSuite.scala | 24 ++ 46 files changed, 639 insertions(+), 80 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out create mode 100644 sql/hive/src/test/resources/sqlgen/sort_asc_nulls_last.sql create mode 100644 sql/hive/src/test/resources/sqlgen/sort_desc_nulls_first.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_basic_asc_nulls_last.sql create mode 100644 sql/hive/src/test/resources/sqlgen/window_basic_desc_nulls_first.sql 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 c44630fbbc2f0..116c84943e855 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 @@ -29,12 +29,23 @@ private PrefixComparators() {} public static final PrefixComparator STRING = new UnsignedPrefixComparator(); public static final PrefixComparator STRING_DESC = new UnsignedPrefixComparatorDesc(); + public static final PrefixComparator STRING_NULLS_LAST = new UnsignedPrefixComparatorNullsLast(); + public static final PrefixComparator STRING_DESC_NULLS_FIRST = new UnsignedPrefixComparatorDescNullsFirst(); + public static final PrefixComparator BINARY = new UnsignedPrefixComparator(); public static final PrefixComparator BINARY_DESC = new UnsignedPrefixComparatorDesc(); + public static final PrefixComparator BINARY_NULLS_LAST = new UnsignedPrefixComparatorNullsLast(); + public static final PrefixComparator BINARY_DESC_NULLS_FIRST = new UnsignedPrefixComparatorDescNullsFirst(); + public static final PrefixComparator LONG = new SignedPrefixComparator(); public static final PrefixComparator LONG_DESC = new SignedPrefixComparatorDesc(); + public static final PrefixComparator LONG_NULLS_LAST = new SignedPrefixComparatorNullsLast(); + public static final PrefixComparator LONG_DESC_NULLS_FIRST = new SignedPrefixComparatorDescNullsFirst(); + public static final PrefixComparator DOUBLE = new UnsignedPrefixComparator(); public static final PrefixComparator DOUBLE_DESC = new UnsignedPrefixComparatorDesc(); + public static final PrefixComparator DOUBLE_NULLS_LAST = new UnsignedPrefixComparatorNullsLast(); + public static final PrefixComparator DOUBLE_DESC_NULLS_FIRST = new UnsignedPrefixComparatorDescNullsFirst(); public static final class StringPrefixComparator { public static long computePrefix(UTF8String value) { @@ -74,6 +85,9 @@ public abstract static class RadixSortSupport extends PrefixComparator { /** @return Whether the sort should take into account the sign bit. */ public abstract boolean sortSigned(); + + /** @return Whether the sort should put nulls first or last. */ + public abstract boolean nullsFirst(); } // @@ -83,16 +97,34 @@ public abstract static class RadixSortSupport extends PrefixComparator { public static final class UnsignedPrefixComparator extends RadixSortSupport { @Override public boolean sortDescending() { return false; } @Override public boolean sortSigned() { return false; } - @Override + @Override public boolean nullsFirst() { return true; } + public int compare(long aPrefix, long bPrefix) { + return UnsignedLongs.compare(aPrefix, bPrefix); + } + } + + public static final class UnsignedPrefixComparatorNullsLast extends RadixSortSupport { + @Override public boolean sortDescending() { return false; } + @Override public boolean sortSigned() { return false; } + @Override public boolean nullsFirst() { return false; } public int compare(long aPrefix, long bPrefix) { return UnsignedLongs.compare(aPrefix, bPrefix); } } + public static final class UnsignedPrefixComparatorDescNullsFirst extends RadixSortSupport { + @Override public boolean sortDescending() { return true; } + @Override public boolean sortSigned() { return false; } + @Override public boolean nullsFirst() { return true; } + public int compare(long bPrefix, long aPrefix) { + return UnsignedLongs.compare(aPrefix, bPrefix); + } + } + public static final class UnsignedPrefixComparatorDesc extends RadixSortSupport { @Override public boolean sortDescending() { return true; } @Override public boolean sortSigned() { return false; } - @Override + @Override public boolean nullsFirst() { return false; } public int compare(long bPrefix, long aPrefix) { return UnsignedLongs.compare(aPrefix, bPrefix); } @@ -101,16 +133,34 @@ public int compare(long bPrefix, long aPrefix) { public static final class SignedPrefixComparator extends RadixSortSupport { @Override public boolean sortDescending() { return false; } @Override public boolean sortSigned() { return true; } - @Override + @Override public boolean nullsFirst() { return true; } + public int compare(long a, long b) { + return (a < b) ? -1 : (a > b) ? 1 : 0; + } + } + + public static final class SignedPrefixComparatorNullsLast extends RadixSortSupport { + @Override public boolean sortDescending() { return false; } + @Override public boolean sortSigned() { return true; } + @Override public boolean nullsFirst() { return false; } public int compare(long a, long b) { return (a < b) ? -1 : (a > b) ? 1 : 0; } } + public static final class SignedPrefixComparatorDescNullsFirst extends RadixSortSupport { + @Override public boolean sortDescending() { return true; } + @Override public boolean sortSigned() { return true; } + @Override public boolean nullsFirst() { return true; } + public int compare(long b, long a) { + return (a < b) ? -1 : (a > b) ? 1 : 0; + } + } + public static final class SignedPrefixComparatorDesc extends RadixSortSupport { @Override public boolean sortDescending() { return true; } @Override public boolean sortSigned() { return true; } - @Override + @Override public boolean nullsFirst() { return false; } public int compare(long b, long a) { return (a < b) ? -1 : (a > b) ? 1 : 0; } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 30d0f3006a04e..be382955c0d42 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -333,17 +333,18 @@ public UnsafeSorterIterator getSortedIterator() { if (nullBoundaryPos > 0) { assert radixSortSupport != null : "Nulls are only stored separately with radix sort"; LinkedList queue = new LinkedList<>(); - if (radixSortSupport.sortDescending()) { - // Nulls are smaller than non-nulls - queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); + + // The null order is either LAST or FIRST, regardless of sorting direction (ASC|DESC) + if (radixSortSupport.nullsFirst()) { queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); + queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); } else { - queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); queue.add(new SortedIterator((pos - nullBoundaryPos) / 2, offset)); + queue.add(new SortedIterator(nullBoundaryPos / 2, 0)); } return new UnsafeExternalSorter.ChainedIterator(queue); } else { return new SortedIterator(pos / 2, offset); } } -} +} \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index 2c13806410192..366ffda7788d3 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -40,23 +40,38 @@ class RadixSortSuite extends SparkFunSuite with Logging { case class RadixSortType( name: String, referenceComparator: PrefixComparator, - startByteIdx: Int, endByteIdx: Int, descending: Boolean, signed: Boolean) + startByteIdx: Int, endByteIdx: Int, descending: Boolean, signed: Boolean, nullsFirst: Boolean) val SORT_TYPES_TO_TEST = Seq( - RadixSortType("unsigned binary data asc", PrefixComparators.BINARY, 0, 7, false, false), - RadixSortType("unsigned binary data desc", PrefixComparators.BINARY_DESC, 0, 7, true, false), - RadixSortType("twos complement asc", PrefixComparators.LONG, 0, 7, false, true), - RadixSortType("twos complement desc", PrefixComparators.LONG_DESC, 0, 7, true, true), + RadixSortType("unsigned binary data asc nulls first", + PrefixComparators.BINARY, 0, 7, false, false, true), + RadixSortType("unsigned binary data asc nulls last", + PrefixComparators.BINARY_NULLS_LAST, 0, 7, false, false, false), + RadixSortType("unsigned binary data desc nulls last", + PrefixComparators.BINARY_DESC_NULLS_FIRST, 0, 7, true, false, false), + RadixSortType("unsigned binary data desc nulls first", + PrefixComparators.BINARY_DESC, 0, 7, true, false, true), + + RadixSortType("twos complement asc nulls first", + PrefixComparators.LONG, 0, 7, false, true, true), + RadixSortType("twos complement asc nulls last", + PrefixComparators.LONG_NULLS_LAST, 0, 7, false, true, false), + RadixSortType("twos complement desc nulls last", + PrefixComparators.LONG_DESC, 0, 7, true, true, false), + RadixSortType("twos complement desc nulls first", + PrefixComparators.LONG_DESC_NULLS_FIRST, 0, 7, true, true, true), + RadixSortType( "binary data partial", new PrefixComparators.RadixSortSupport { override def sortDescending = false override def sortSigned = false + override def nullsFirst = true override def compare(a: Long, b: Long): Int = { return PrefixComparators.BINARY.compare(a & 0xffffff0000L, b & 0xffffff0000L) } }, - 2, 4, false, false)) + 2, 4, false, false, true)) private def generateTestData(size: Int, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](size) { i => rand } 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 9a643465a9994..b475abdce2da9 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 @@ -324,7 +324,7 @@ queryPrimary ; sortItem - : expression ordering=(ASC | DESC)? + : expression ordering=(ASC | DESC)? (NULLS nullOrder=(LAST | FIRST))? ; querySpecification @@ -641,7 +641,8 @@ number nonReserved : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS | DATABASES | ADD - | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | MAP | ARRAY | STRUCT + | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST + | MAP | ARRAY | STRUCT | LATERAL | WINDOW | REDUCE | TRANSFORM | USING | SERDE | SERDEPROPERTIES | RECORDREADER | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | TEMPORARY | OPTIONS @@ -729,6 +730,8 @@ UNBOUNDED: 'UNBOUNDED'; PRECEDING: 'PRECEDING'; FOLLOWING: 'FOLLOWING'; CURRENT: 'CURRENT'; +FIRST: 'FIRST'; +LAST: 'LAST'; ROW: 'ROW'; WITH: 'WITH'; VALUES: 'VALUES'; 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 18f814d6cdfd4..92bf8e0536fc4 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 @@ -714,9 +714,9 @@ class Analyzer( case s @ Sort(orders, global, child) if orders.exists(_.child.isInstanceOf[UnresolvedOrdinal]) => val newOrders = orders map { - case s @ SortOrder(UnresolvedOrdinal(index), direction) => + case s @ SortOrder(UnresolvedOrdinal(index), direction, nullOrdering) => if (index > 0 && index <= child.output.size) { - SortOrder(child.output(index - 1), direction) + SortOrder(child.output(index - 1), direction, nullOrdering) } else { s.failAnalysis( s"ORDER BY position $index is not in select list " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala index 6d8dc8628229a..af0a565f73ae9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala @@ -36,7 +36,7 @@ class SubstituteUnresolvedOrdinals(conf: CatalystConf) extends Rule[LogicalPlan] def apply(plan: LogicalPlan): LogicalPlan = plan transform { case s: Sort if conf.orderByOrdinal && s.order.exists(o => isIntLiteral(o.child)) => val newOrders = s.order.map { - case order @ SortOrder(ordinal @ Literal(index: Int, IntegerType), _) => + case order @ SortOrder(ordinal @ Literal(index: Int, IntegerType), _, _) => val newOrdinal = withOrigin(ordinal.origin)(UnresolvedOrdinal(index)) withOrigin(order.origin)(order.copy(child = newOrdinal)) case other => other 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 8549187a66369..66e52ca68af19 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 @@ -109,8 +109,9 @@ package object dsl { def cast(to: DataType): Expression = Cast(expr, to) def asc: SortOrder = SortOrder(expr, Ascending) + def asc_nullsLast: SortOrder = SortOrder(expr, Ascending, NullsLast) def desc: SortOrder = SortOrder(expr, Descending) - + def desc_nullsFirst: SortOrder = SortOrder(expr, Descending, NullsFirst) def as(alias: String): NamedExpression = Alias(expr, alias)() def as(alias: Symbol): NamedExpression = Alias(expr, alias.name)() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index de779ed3702d3..d015125baccaf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -21,26 +21,43 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.types._ -import org.apache.spark.util.collection.unsafe.sort.PrefixComparators.BinaryPrefixComparator -import org.apache.spark.util.collection.unsafe.sort.PrefixComparators.DoublePrefixComparator +import org.apache.spark.util.collection.unsafe.sort.PrefixComparators._ abstract sealed class SortDirection { def sql: String + def defaultNullOrdering: NullOrdering +} + +abstract sealed class NullOrdering { + def sql: String } case object Ascending extends SortDirection { override def sql: String = "ASC" + override def defaultNullOrdering: NullOrdering = NullsFirst } case object Descending extends SortDirection { override def sql: String = "DESC" + override def defaultNullOrdering: NullOrdering = NullsLast +} + +case object NullsFirst extends NullOrdering{ + override def sql: String = "NULLS FIRST" +} + +case object NullsLast extends NullOrdering{ + override def sql: String = "NULLS LAST" } /** * An expression that can be used to sort a tuple. This class extends expression primarily so that * transformations over expression will descend into its child. */ -case class SortOrder(child: Expression, direction: SortDirection) +case class SortOrder( + child: Expression, + direction: SortDirection, + nullOrdering: NullOrdering) extends UnaryExpression with Unevaluable { /** Sort order is not foldable because we don't have an eval for it. */ @@ -57,12 +74,18 @@ case class SortOrder(child: Expression, direction: SortDirection) override def dataType: DataType = child.dataType override def nullable: Boolean = child.nullable - override def toString: String = s"$child ${direction.sql}" - override def sql: String = child.sql + " " + direction.sql + override def toString: String = s"$child ${direction.sql} ${nullOrdering.sql}" + override def sql: String = child.sql + " " + direction.sql + " " + nullOrdering.sql def isAscending: Boolean = direction == Ascending } +object SortOrder { + def apply(child: Expression, direction: SortDirection): SortOrder = { + new SortOrder(child, direction, direction.defaultNullOrdering) + } +} + /** * An expression to generate a 64-bit long prefix used in sorting. If the sort must operate over * null keys as well, this.nullValue can be used in place of emitted null prefixes in the sort. @@ -71,14 +94,35 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val nullValue = child.child.dataType match { case BooleanType | DateType | TimestampType | _: IntegralType => - Long.MinValue + if (nullAsSmallest) { + Long.MinValue + } else { + Long.MaxValue + } case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => - Long.MinValue + if (nullAsSmallest) { + Long.MinValue + } else { + Long.MaxValue + } case _: DecimalType => - DoublePrefixComparator.computePrefix(Double.NegativeInfinity) - case _ => 0L + if (nullAsSmallest) { + DoublePrefixComparator.computePrefix(Double.NegativeInfinity) + } else { + DoublePrefixComparator.computePrefix(Double.NaN) + } + case _ => + if (nullAsSmallest) { + 0L + } else { + -1L + } } + private def nullAsSmallest: Boolean = (child.isAscending && child.nullOrdering == NullsFirst) || + (!child.isAscending && child.nullOrdering == NullsLast) + + override def eval(input: InternalRow): Any = throw new UnsupportedOperationException override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -86,6 +130,7 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { val input = childCode.value val BinaryPrefixCmp = classOf[BinaryPrefixComparator].getName val DoublePrefixCmp = classOf[DoublePrefixComparator].getName + val StringPrefixCmp = classOf[StringPrefixComparator].getName val prefixCode = child.child.dataType match { case BooleanType => s"$input ? 1L : 0L" @@ -95,7 +140,7 @@ case class SortPrefix(child: SortOrder) extends UnaryExpression { s"(long) $input" case FloatType | DoubleType => s"$DoublePrefixCmp.computePrefix((double)$input)" - case StringType => s"$input.getPrefix()" + case StringType => s"$StringPrefixCmp.computePrefix($input)" case BinaryType => s"$BinaryPrefixCmp.computePrefix($input)" case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => if (dt.precision <= Decimal.MAX_LONG_DIGITS) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index f4d35d232e691..e7df95e1142ca 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -63,7 +63,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR */ def genComparisons(ctx: CodegenContext, schema: StructType): String = { val ordering = schema.fields.map(_.dataType).zipWithIndex.map { - case(dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + case(dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) } genComparisons(ctx, ordering) } @@ -74,7 +74,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR def genComparisons(ctx: CodegenContext, ordering: Seq[SortOrder]): String = { val comparisons = ordering.map { order => val eval = order.child.genCode(ctx) - val asc = order.direction == Ascending + val asc = order.isAscending val isNullA = ctx.freshName("isNullA") val primitiveA = ctx.freshName("primitiveA") val isNullB = ctx.freshName("isNullB") @@ -99,9 +99,17 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR if ($isNullA && $isNullB) { // Nothing } else if ($isNullA) { - return ${if (order.direction == Ascending) "-1" else "1"}; + return ${ + order.nullOrdering match { + case NullsFirst => "-1" + case NullsLast => "1" + }}; } else if ($isNullB) { - return ${if (order.direction == Ascending) "1" else "-1"}; + return ${ + order.nullOrdering match { + case NullsFirst => "1" + case NullsLast => "-1" + }}; } else { int comp = ${ctx.genComp(order.child.dataType, primitiveA, primitiveB)}; if (comp != 0) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index 6112259fed619..79d2052c38a27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -39,9 +39,9 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow if (left == null && right == null) { // Both null, continue looking. } else if (left == null) { - return if (order.direction == Ascending) -1 else 1 + return if (order.nullOrdering == NullsFirst) -1 else 1 } else if (right == null) { - return if (order.direction == Ascending) 1 else -1 + return if (order.nullOrdering == NullsFirst) 1 else -1 } else { val comparison = order.dataType match { case dt: AtomicType if order.direction == Ascending => @@ -76,7 +76,7 @@ object InterpretedOrdering { */ def forSchema(dataTypes: Seq[DataType]): InterpretedOrdering = { new InterpretedOrdering(dataTypes.zipWithIndex.map { - case (dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) }) } } 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 bbbb14df88f8c..69d68fa6f92ef 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 @@ -1206,11 +1206,19 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging { * Create a [[SortOrder]] expression. */ override def visitSortItem(ctx: SortItemContext): SortOrder = withOrigin(ctx) { - if (ctx.DESC != null) { - SortOrder(expression(ctx.expression), Descending) + val direction = if (ctx.DESC != null) { + Descending } else { - SortOrder(expression(ctx.expression), Ascending) + Ascending } + val nullOrdering = if (ctx.FIRST != null) { + NullsFirst + } else if (ctx.LAST != null) { + NullsLast + } else { + direction.defaultNullOrdering + } + SortOrder(expression(ctx.expression), direction, nullOrdering) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala index 940467e74d597..c6665d273fd27 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala @@ -40,22 +40,70 @@ object SortPrefixUtils { def getPrefixComparator(sortOrder: SortOrder): PrefixComparator = { sortOrder.dataType match { - case StringType => - if (sortOrder.isAscending) PrefixComparators.STRING else PrefixComparators.STRING_DESC - case BinaryType => - if (sortOrder.isAscending) PrefixComparators.BINARY else PrefixComparators.BINARY_DESC + case StringType => stringPrefixComparator(sortOrder) + case BinaryType => binaryPrefixComparator(sortOrder) case BooleanType | ByteType | ShortType | IntegerType | LongType | DateType | TimestampType => - if (sortOrder.isAscending) PrefixComparators.LONG else PrefixComparators.LONG_DESC + longPrefixComparator(sortOrder) case dt: DecimalType if dt.precision - dt.scale <= Decimal.MAX_LONG_DIGITS => - if (sortOrder.isAscending) PrefixComparators.LONG else PrefixComparators.LONG_DESC - case FloatType | DoubleType => - if (sortOrder.isAscending) PrefixComparators.DOUBLE else PrefixComparators.DOUBLE_DESC - case dt: DecimalType => - if (sortOrder.isAscending) PrefixComparators.DOUBLE else PrefixComparators.DOUBLE_DESC + longPrefixComparator(sortOrder) + case FloatType | DoubleType => doublePrefixComparator(sortOrder) + case dt: DecimalType => doublePrefixComparator(sortOrder) case _ => NoOpPrefixComparator } } + private def stringPrefixComparator(sortOrder: SortOrder): PrefixComparator = { + sortOrder.direction match { + case Ascending if (sortOrder.nullOrdering == NullsLast) => + PrefixComparators.STRING_NULLS_LAST + case Ascending => + PrefixComparators.STRING + case Descending if (sortOrder.nullOrdering == NullsFirst) => + PrefixComparators.STRING_DESC_NULLS_FIRST + case Descending => + PrefixComparators.STRING_DESC + } + } + + private def binaryPrefixComparator(sortOrder: SortOrder): PrefixComparator = { + sortOrder.direction match { + case Ascending if (sortOrder.nullOrdering == NullsLast) => + PrefixComparators.BINARY_NULLS_LAST + case Ascending => + PrefixComparators.BINARY + case Descending if (sortOrder.nullOrdering == NullsFirst) => + PrefixComparators.BINARY_DESC_NULLS_FIRST + case Descending => + PrefixComparators.BINARY_DESC + } + } + + private def longPrefixComparator(sortOrder: SortOrder): PrefixComparator = { + sortOrder.direction match { + case Ascending if (sortOrder.nullOrdering == NullsLast) => + PrefixComparators.LONG_NULLS_LAST + case Ascending => + PrefixComparators.LONG + case Descending if (sortOrder.nullOrdering == NullsFirst) => + PrefixComparators.LONG_DESC_NULLS_FIRST + case Descending => + PrefixComparators.LONG_DESC + } + } + + private def doublePrefixComparator(sortOrder: SortOrder): PrefixComparator = { + sortOrder.direction match { + case Ascending if (sortOrder.nullOrdering == NullsLast) => + PrefixComparators.DOUBLE_NULLS_LAST + case Ascending => + PrefixComparators.DOUBLE + case Descending if (sortOrder.nullOrdering == NullsFirst) => + PrefixComparators.DOUBLE_DESC_NULLS_FIRST + case Descending => + PrefixComparators.DOUBLE_DESC + } + } + /** * Creates the prefix comparator for the first field in the given schema, in ascending order. */ 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 6a2d97c9b1797..6aeefa6eddafe 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 @@ -368,7 +368,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ */ protected def newNaturalAscendingOrdering(dataTypes: Seq[DataType]): Ordering[InternalRow] = { val order: Seq[SortOrder] = dataTypes.zipWithIndex.map { - case (dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + case (dt, index) => SortOrder(BoundReference(index, dt, nullable = true), Ascending) } newOrdering(order, Seq.empty) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql b/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql new file mode 100644 index 0000000000000..f7637b444b9fe --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/orderby-nulls-ordering.sql @@ -0,0 +1,83 @@ +-- Q1. testing window functions with order by +create table spark_10747(col1 int, col2 int, col3 int) using parquet; + +-- Q2. insert to tables +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null); + +-- Q3. windowing with order by DESC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q4. windowing with order by DESC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q5. windowing with order by ASC NULLS LAST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q6. windowing with order by ASC NULLS FIRST +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2; + +-- Q7. Regular query with ORDER BY ASC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2; + +-- Q8. Regular query with ORDER BY ASC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2; + +-- Q9. Regular query with ORDER BY DESC NULLS FIRST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2; + +-- Q10. Regular query with ORDER BY DESC NULLS LAST +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2; + +-- drop the test table +drop table spark_10747; + +-- Q11. mix datatype for ORDER BY NULLS FIRST|LAST +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet; + +-- Q12. Insert to the table +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null); + +-- Q13. Regular query with 2 NULLS LAST columns +select * from spark_10747_mix order by col1 nulls last, col5 nulls last; + +-- Q14. Regular query with 2 NULLS FIRST columns +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first; + +-- Q15. Regular query with mixed NULLS FIRST|LAST +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last; + +-- drop the test table +drop table spark_10747_mix; + + diff --git a/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out b/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out new file mode 100644 index 0000000000000..c1b63dfb8caef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/orderby-nulls-ordering.sql.out @@ -0,0 +1,254 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create table spark_10747(col1 int, col2 int, col3 int) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO spark_10747 VALUES (6, 12, 10), (6, 11, 4), (6, 9, 10), (6, 15, 8), +(6, 15, 8), (6, 7, 4), (6, 7, 8), (6, 13, null), (6, 10, null) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 2 schema +struct +-- !query 2 output +6 9 10 28 +6 13 NULL 34 +6 10 NULL 41 +6 12 10 43 +6 15 8 55 +6 15 8 56 +6 11 4 56 +6 7 8 58 +6 7 4 58 + + +-- !query 3 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 desc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 3 schema +struct +-- !query 3 output +6 10 NULL 32 +6 11 4 33 +6 13 NULL 44 +6 7 4 48 +6 9 10 51 +6 15 8 55 +6 12 10 56 +6 15 8 56 +6 7 8 58 + + +-- !query 4 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls last, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 4 schema +struct +-- !query 4 output +6 7 4 25 +6 13 NULL 35 +6 11 4 40 +6 10 NULL 44 +6 7 8 55 +6 15 8 57 +6 15 8 58 +6 12 10 59 +6 9 10 61 + + +-- !query 5 +select col1, col2, col3, sum(col2) + over (partition by col1 + order by col3 asc nulls first, col2 + rows between 2 preceding and 2 following ) as sum_col2 +from spark_10747 where col1 = 6 order by sum_col2 +-- !query 5 schema +struct +-- !query 5 output +6 10 NULL 30 +6 12 10 36 +6 13 NULL 41 +6 7 4 48 +6 9 10 51 +6 11 4 53 +6 7 8 55 +6 15 8 57 +6 15 8 58 + + +-- !query 6 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 ASC NULLS FIRST, COL2 +-- !query 6 schema +struct +-- !query 6 output +6 10 NULL +6 13 NULL +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 + + +-- !query 7 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 NULLS LAST, COL2 +-- !query 7 schema +struct +-- !query 7 output +6 7 4 +6 11 4 +6 7 8 +6 15 8 +6 15 8 +6 9 10 +6 12 10 +6 10 NULL +6 13 NULL + + +-- !query 8 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS FIRST, COL2 +-- !query 8 schema +struct +-- !query 8 output +6 10 NULL +6 13 NULL +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 + + +-- !query 9 +SELECT COL1, COL2, COL3 FROM spark_10747 ORDER BY COL3 DESC NULLS LAST, COL2 +-- !query 9 schema +struct +-- !query 9 output +6 9 10 +6 12 10 +6 7 8 +6 15 8 +6 15 8 +6 7 4 +6 11 4 +6 10 NULL +6 13 NULL + + +-- !query 10 +drop table spark_10747 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +create table spark_10747_mix( +col1 string, +col2 int, +col3 double, +col4 decimal(10,2), +col5 decimal(20,1)) +using parquet +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO spark_10747_mix VALUES +('b', 2, 1.0, 1.00, 10.0), +('d', 3, 2.0, 3.00, 0.0), +('c', 3, 2.0, 2.00, 15.1), +('d', 3, 0.0, 3.00, 1.0), +(null, 3, 0.0, 3.00, 1.0), +('d', 3, null, 4.00, 1.0), +('a', 1, 1.0, 1.00, null), +('c', 3, 2.0, 2.00, null) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select * from spark_10747_mix order by col1 nulls last, col5 nulls last +-- !query 13 schema +struct +-- !query 13 output +a 1 1.0 1 NULL +b 2 1.0 1 10 +c 3 2.0 2 15.1 +c 3 2.0 2 NULL +d 3 2.0 3 0 +d 3 0.0 3 1 +d 3 NULL 4 1 +NULL 3 0.0 3 1 + + +-- !query 14 +select * from spark_10747_mix order by col1 desc nulls first, col5 desc nulls first +-- !query 14 schema +struct +-- !query 14 output +NULL 3 0.0 3 1 +d 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 +c 3 2.0 2 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +a 1 1.0 1 NULL + + +-- !query 15 +select * from spark_10747_mix order by col5 desc nulls first, col3 desc nulls last +-- !query 15 schema +struct +-- !query 15 output +c 3 2.0 2 NULL +a 1 1.0 1 NULL +c 3 2.0 2 15.1 +b 2 1.0 1 10 +d 3 0.0 3 1 +NULL 3 0.0 3 1 +d 3 NULL 4 1 +d 3 2.0 3 0 + + +-- !query 16 +drop table spark_10747_mix +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala index ba3fa3732d0df..a7bbe34f4eedb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SortSuite.scala @@ -101,7 +101,8 @@ class SortSuite extends SparkPlanTest with SharedSQLContext { for ( dataType <- DataTypeTestUtils.atomicTypes ++ Set(NullType); nullable <- Seq(true, false); - sortOrder <- Seq('a.asc :: Nil, 'a.desc :: Nil); + sortOrder <- + Seq('a.asc :: Nil, 'a.asc_nullsLast :: Nil, 'a.desc :: Nil, 'a.desc_nullsFirst :: Nil); randomDataGenerator <- RandomDataGenerator.forType(dataType, nullable) ) { test(s"sorting on $dataType with nullable=$nullable, sortOrder=$sortOrder") { diff --git a/sql/hive/src/test/resources/sqlgen/agg2.sql b/sql/hive/src/test/resources/sqlgen/agg2.sql index 65d71714fe850..adbfdb7e79d64 100644 --- a/sql/hive/src/test/resources/sqlgen/agg2.sql +++ b/sql/hive/src/test/resources/sqlgen/agg2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` ORDER BY `gen_attr_1` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_2`) AS `gen_attr_1` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_2` ORDER BY `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/agg3.sql b/sql/hive/src/test/resources/sqlgen/agg3.sql index 14b19392cdce3..207542d226e23 100644 --- a/sql/hive/src/test/resources/sqlgen/agg3.sql +++ b/sql/hive/src/test/resources/sqlgen/agg3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key) -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` ASC, `gen_attr_2` ASC) AS gen_subquery_1) AS gen_subquery_2 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` ASC NULLS FIRST, `gen_attr_2` ASC NULLS FIRST) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql index ec881a216e0b0..3de4f8a059965 100644 --- a/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql +++ b/sql/hive/src/test/resources/sqlgen/broadcast_join_subquery.sql @@ -5,4 +5,4 @@ FROM (SELECT x.key as key1, x.value as value1, y.key as key2, y.value as value2 JOIN srcpart z ON (subq.key1 = z.key and z.ds='2008-04-08' and z.hr=11) ORDER BY subq.key1, z.value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_3 +SELECT `gen_attr_0` AS `key1`, `gen_attr_1` AS `value` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_7` AS `gen_attr_6`, `gen_attr_9` AS `gen_attr_8`, `gen_attr_11` AS `gen_attr_10` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_7` FROM `default`.`src1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_9`, `value` AS `gen_attr_11` FROM `default`.`src`) AS gen_subquery_1 ON (`gen_attr_5` = `gen_attr_9`)) AS subq INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_1`, `ds` AS `gen_attr_3`, `hr` AS `gen_attr_4` FROM `default`.`srcpart`) AS gen_subquery_2 ON (((`gen_attr_0` = `gen_attr_2`) AND (`gen_attr_3` = '2008-04-08')) AND (CAST(`gen_attr_4` AS DOUBLE) = CAST(11 AS DOUBLE))) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql index 805197a4ea11b..ab444d0c70936 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_1.sql @@ -5,4 +5,4 @@ WHERE id > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_0.`gen_attr_2`, gen_subquery_0.`gen_attr_3`, gen_subquery_0.`gen_attr_4`, gen_subquery_0.`gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr_1` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS parquet_t3 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_0.`gen_attr_2`, gen_subquery_0.`gen_attr_3`, gen_subquery_0.`gen_attr_4`, gen_subquery_0.`gen_attr_1` FROM (SELECT `arr` AS `gen_attr_2`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_4`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 WHERE (`gen_attr_1` > CAST(2 AS BIGINT))) AS gen_subquery_1 LATERAL VIEW explode(`gen_attr_2`) gen_subquery_2 AS `gen_attr_0` ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST LIMIT 5) AS parquet_t3 diff --git a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql index ef9a596197b8b..42a2369f34d1c 100644 --- a/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql +++ b/sql/hive/src/test/resources/sqlgen/generate_with_other_2.sql @@ -7,4 +7,4 @@ WHERE val > 2 ORDER BY val, id LIMIT 5 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0` WHERE (`gen_attr_0` > CAST(2 AS BIGINT)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC LIMIT 5) AS gen_subquery_1 +SELECT `gen_attr_0` AS `val`, `gen_attr_1` AS `id` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `arr` AS `gen_attr_4`, `arr2` AS `gen_attr_3`, `json` AS `gen_attr_5`, `id` AS `gen_attr_1` FROM `default`.`parquet_t3`) AS gen_subquery_0 LATERAL VIEW explode(`gen_attr_3`) gen_subquery_2 AS `gen_attr_2` LATERAL VIEW explode(`gen_attr_2`) gen_subquery_3 AS `gen_attr_0` WHERE (`gen_attr_0` > CAST(2 AS BIGINT)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST LIMIT 5) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql index b2c426c660d80..245b52341658f 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`), (`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`), (`gen_attr_6`)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql index 96ee8e85951e8..1505dea11ec68 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (a) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql index 9b8b230c879c2..281add6aabb64 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_6`)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql index c35db74a5c5b5..f8d64742b11e3 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS (()) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS(()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS(()) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql index e47f6d5dcf465..09e6ec2a5f8c9 100644 --- a/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql +++ b/sql/hive/src/test/resources/sqlgen/grouping_sets_2_5.sql @@ -2,4 +2,4 @@ SELECT a, b, sum(c) FROM parquet_t2 GROUP BY a, b GROUPING SETS ((), (a), (a, b)) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((), (`gen_attr_5`), (`gen_attr_5`, `gen_attr_6`)) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((), (`gen_attr_5`), (`gen_attr_5`, `gen_attr_6`)) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql index 22df578518ef3..c364c32dd5c55 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_1.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql index f44b652343acb..36c0223fceced 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_2.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(c) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), (`gen_attr_6`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(c)` FROM (SELECT `gen_attr_5` AS `gen_attr_0`, `gen_attr_6` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_5`, `b` AS `gen_attr_6`, `c` AS `gen_attr_4`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_5`, `gen_attr_6` GROUPING SETS((`gen_attr_5`, `gen_attr_6`), (`gen_attr_5`), (`gen_attr_6`), ()) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql index 40f6924913765..ed33f2a1de3cf 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_3.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY ROLLUP(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), ()) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql index 608e644dee6d0..e0e40241480da 100644 --- a/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql +++ b/sql/hive/src/test/resources/sqlgen/rollup_cube_6_4.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT a, b, sum(a) FROM parquet_t2 GROUP BY CUBE(a, b) ORDER BY a, b -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC) AS gen_subquery_1 +SELECT `gen_attr_0` AS `a`, `gen_attr_1` AS `b`, `gen_attr_3` AS `sum(a)` FROM (SELECT `gen_attr_4` AS `gen_attr_0`, `gen_attr_5` AS `gen_attr_1`, sum(`gen_attr_4`) AS `gen_attr_3` FROM (SELECT `a` AS `gen_attr_4`, `b` AS `gen_attr_5`, `c` AS `gen_attr_6`, `d` AS `gen_attr_7` FROM `default`.`parquet_t2`) AS gen_subquery_0 GROUP BY `gen_attr_4`, `gen_attr_5` GROUPING SETS((`gen_attr_4`, `gen_attr_5`), (`gen_attr_4`), (`gen_attr_5`), ()) ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_1 diff --git a/sql/hive/src/test/resources/sqlgen/sort_asc_nulls_last.sql b/sql/hive/src/test/resources/sqlgen/sort_asc_nulls_last.sql new file mode 100644 index 0000000000000..da4e3678a33b9 --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/sort_asc_nulls_last.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key nulls last, MAX(key) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` ASC NULLS LAST, `gen_attr_2` ASC NULLS FIRST) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql index da60204297a21..a4f3ddc761f30 100644 --- a/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql +++ b/sql/hive/src/test/resources/sqlgen/sort_by_after_having.sql @@ -1,4 +1,4 @@ -- This file is automatically generated by LogicalPlanToSQLSuite. SELECT COUNT(value) FROM parquet_t1 GROUP BY key HAVING MAX(key) > 0 SORT BY key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_1`) AS `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING (`gen_attr_2` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr_1` ASC) AS gen_subquery_2) AS gen_subquery_3 +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT count(`gen_attr_3`) AS `gen_attr_0`, max(`gen_attr_1`) AS `gen_attr_2`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_1`, `value` AS `gen_attr_3` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_1` HAVING (`gen_attr_2` > CAST(0 AS BIGINT))) AS gen_subquery_1 SORT BY `gen_attr_1` ASC NULLS FIRST) AS gen_subquery_2) AS gen_subquery_3 diff --git a/sql/hive/src/test/resources/sqlgen/sort_desc_nulls_first.sql b/sql/hive/src/test/resources/sqlgen/sort_desc_nulls_first.sql new file mode 100644 index 0000000000000..d995e3bdfad5c --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/sort_desc_nulls_first.sql @@ -0,0 +1,4 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key desc nulls first,MAX(key) +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `count(value)` FROM (SELECT `gen_attr_0` FROM (SELECT count(`gen_attr_4`) AS `gen_attr_0`, `gen_attr_3` AS `gen_attr_1`, max(`gen_attr_3`) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_3`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_3` ORDER BY `gen_attr_1` DESC NULLS FIRST, `gen_attr_2` ASC NULLS FIRST) AS gen_subquery_1) AS gen_subquery_2 diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql index 9894f5ab39c76..25882147463b9 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_1.sql @@ -5,4 +5,4 @@ group by key having count(*) in (select count(*) from src s1 where s1.key = '90' group by s1.key) order by key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS src +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `count(1)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, count(1) AS `gen_attr_1`, count(1) AS `gen_attr_2` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (`gen_attr_2` IN (SELECT `gen_attr_5` AS `_c0` FROM (SELECT `gen_attr_3` AS `gen_attr_5` FROM (SELECT count(1) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_6`, `value` AS `gen_attr_7` FROM `default`.`src`) AS gen_subquery_3 WHERE (CAST(`gen_attr_6` AS DOUBLE) = CAST('90' AS DOUBLE)) GROUP BY `gen_attr_6`) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS src diff --git a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql index c3a122aa889b9..de0116a4dcbaf 100644 --- a/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql +++ b/sql/hive/src/test/resources/sqlgen/subquery_in_having_2.sql @@ -7,4 +7,4 @@ having b.key in (select a.key where a.value > 'val_9' and a.value = min(b.value)) order by b.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC) AS b +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `min(value)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `gen_attr_0`, min(`gen_attr_5`) AS `gen_attr_1`, min(`gen_attr_5`) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_5` FROM `default`.`src`) AS gen_subquery_0 GROUP BY `gen_attr_0` HAVING (struct(`gen_attr_0`, `gen_attr_4`) IN (SELECT `gen_attr_6` AS `_c0`, `gen_attr_7` AS `_c1` FROM (SELECT `gen_attr_2` AS `gen_attr_6`, `gen_attr_3` AS `gen_attr_7` FROM (SELECT `gen_attr_2`, `gen_attr_3` FROM (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_3` FROM `default`.`src`) AS gen_subquery_3 WHERE (`gen_attr_3` > 'val_9')) AS gen_subquery_2) AS gen_subquery_4))) AS gen_subquery_1 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS b diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql index ec55d4b7146f2..0e2a9a54731fc 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_2.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_2.sql @@ -2,4 +2,4 @@ SELECT key, value, ROUND(AVG(key) OVER (), 2) FROM parquet_t1 ORDER BY key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` ASC) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` ASC NULLS FIRST) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql index c0ac9541e67ee..d727caa583e61 100644 --- a/sql/hive/src/test/resources/sqlgen/window_basic_3.sql +++ b/sql/hive/src/test/resources/sqlgen/window_basic_3.sql @@ -2,4 +2,4 @@ SELECT value, MAX(key + 1) OVER (PARTITION BY key % 5 ORDER BY key % 7) AS max FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `value`, `gen_attr_1` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, gen_subquery_1.`gen_attr_4`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_4` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, (`gen_attr_5` + CAST(1 AS BIGINT)) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, (`gen_attr_5` % CAST(7 AS BIGINT)) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_0` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `value`, `gen_attr_1` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_2`, gen_subquery_1.`gen_attr_3`, gen_subquery_1.`gen_attr_4`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_4` ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, (`gen_attr_5` + CAST(1 AS BIGINT)) AS `gen_attr_2`, (`gen_attr_5` % CAST(5 AS BIGINT)) AS `gen_attr_3`, (`gen_attr_5` % CAST(7 AS BIGINT)) AS `gen_attr_4` FROM (SELECT `key` AS `gen_attr_5`, `value` AS `gen_attr_0` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_asc_nulls_last.sql b/sql/hive/src/test/resources/sqlgen/window_basic_asc_nulls_last.sql new file mode 100644 index 0000000000000..4739f05808daf --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_basic_asc_nulls_last.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, ROUND(AVG(key) OVER (), 2) +FROM parquet_t1 ORDER BY key nulls last +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` ASC NULLS LAST) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_basic_desc_nulls_first.sql b/sql/hive/src/test/resources/sqlgen/window_basic_desc_nulls_first.sql new file mode 100644 index 0000000000000..1b9db2993b09d --- /dev/null +++ b/sql/hive/src/test/resources/sqlgen/window_basic_desc_nulls_first.sql @@ -0,0 +1,5 @@ +-- This file is automatically generated by LogicalPlanToSQLSuite. +SELECT key, value, ROUND(AVG(key) OVER (), 2) +FROM parquet_t1 ORDER BY key desc nulls first +-------------------------------------------------------------------------------- +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `round(avg(key) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING), 2)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, round(`gen_attr_3`, 2) AS `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, avg(`gen_attr_0`) OVER (ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2 ORDER BY `gen_attr_0` DESC NULLS FIRST) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_join.sql b/sql/hive/src/test/resources/sqlgen/window_with_join.sql index 030a4c0907a1c..43d5b47be8fba 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_join.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_join.sql @@ -2,4 +2,4 @@ SELECT x.key, MAX(y.key) OVER (PARTITION BY x.key % 5 ORDER BY x.key) FROM parquet_t1 x JOIN parquet_t1 y ON x.key = y.key -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_2.`gen_attr_0`, gen_subquery_2.`gen_attr_2`, gen_subquery_2.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_2`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`)) AS gen_subquery_2) AS gen_subquery_3) AS x +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `max(key) OVER (PARTITION BY (key % CAST(5 AS BIGINT)) ORDER BY key ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT gen_subquery_2.`gen_attr_0`, gen_subquery_2.`gen_attr_2`, gen_subquery_2.`gen_attr_3`, max(`gen_attr_2`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_1` FROM (SELECT `gen_attr_0`, `gen_attr_2`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_4` FROM `default`.`parquet_t1`) AS gen_subquery_0 INNER JOIN (SELECT `key` AS `gen_attr_2`, `value` AS `gen_attr_5` FROM `default`.`parquet_t1`) AS gen_subquery_1 ON (`gen_attr_0` = `gen_attr_2`)) AS gen_subquery_2) AS gen_subquery_3) AS x diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql index 7b99539a05480..33a8e83750be0 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `count(key)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, count(`gen_attr_0`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `count(key)` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, count(`gen_attr_0`) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql index 591a654a3888e..e01bc034d3d12 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_filter.sql @@ -4,4 +4,4 @@ DENSE_RANK() OVER (DISTRIBUTE BY key SORT BY key, value) AS dr, COUNT(key) OVER(DISTRIBUTE BY key SORT BY key, value) AS ca FROM parquet_t1 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `ca` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2`, count(`gen_attr_0`) OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC, `gen_attr_1` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `dr`, `gen_attr_3` AS `ca` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2`, `gen_attr_3` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, DENSE_RANK() OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2`, count(`gen_attr_0`) OVER (PARTITION BY `gen_attr_0` ORDER BY `gen_attr_0` ASC NULLS FIRST, `gen_attr_1` ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_3` FROM (SELECT `gen_attr_0`, `gen_attr_1` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql index d9169eab6e46a..dbfa408fa517e 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_functions.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key) AS max FROM parquet_t1 GROUP BY key, value -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` ASC NULLS FIRST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1`) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql index f0a820811ee0a..6f5741b946262 100644 --- a/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql +++ b/sql/hive/src/test/resources/sqlgen/window_with_the_same_window_with_agg_having.sql @@ -3,4 +3,4 @@ SELECT key, value, MAX(value) OVER (PARTITION BY key % 5 ORDER BY key DESC) AS max FROM parquet_t1 GROUP BY key, value HAVING key > 5 -------------------------------------------------------------------------------- -SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` DESC RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1` HAVING (`gen_attr_0` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 +SELECT `gen_attr_0` AS `key`, `gen_attr_1` AS `value`, `gen_attr_2` AS `max` FROM (SELECT `gen_attr_0`, `gen_attr_1`, `gen_attr_2` FROM (SELECT gen_subquery_1.`gen_attr_0`, gen_subquery_1.`gen_attr_1`, gen_subquery_1.`gen_attr_3`, max(`gen_attr_1`) OVER (PARTITION BY `gen_attr_3` ORDER BY `gen_attr_0` DESC NULLS LAST RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS `gen_attr_2` FROM (SELECT `gen_attr_0`, `gen_attr_1`, (`gen_attr_0` % CAST(5 AS BIGINT)) AS `gen_attr_3` FROM (SELECT `key` AS `gen_attr_0`, `value` AS `gen_attr_1` FROM `default`.`parquet_t1`) AS gen_subquery_0 GROUP BY `gen_attr_0`, `gen_attr_1` HAVING (`gen_attr_0` > CAST(5 AS BIGINT))) AS gen_subquery_1) AS gen_subquery_2) AS parquet_t1 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index d2b2f38fa1f71..ce5efe853ca4f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -106,17 +106,17 @@ class ExpressionSQLBuilderSuite extends SQLBuilderTest { checkSQL( WindowSpecDefinition(Nil, 'a.int.asc :: Nil, frame), - s"(ORDER BY `a` ASC $frame)" + s"(ORDER BY `a` ASC NULLS FIRST $frame)" ) checkSQL( WindowSpecDefinition(Nil, 'a.int.asc :: 'b.string.desc :: Nil, frame), - s"(ORDER BY `a` ASC, `b` DESC $frame)" + s"(ORDER BY `a` ASC NULLS FIRST, `b` DESC NULLS LAST $frame)" ) checkSQL( WindowSpecDefinition('a.int :: 'b.string :: Nil, 'c.int.asc :: 'd.string.desc :: Nil, frame), - s"(PARTITION BY `a`, `b` ORDER BY `c` ASC, `d` DESC $frame)" + s"(PARTITION BY `a`, `b` ORDER BY `c` ASC NULLS FIRST, `d` DESC NULLS LAST $frame)" ) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala index d80f894c22dd8..7fa5c29dc5b8f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/LogicalPlanToSQLSuite.scala @@ -235,6 +235,16 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key, MAX(key)", "agg3") } + test("order by asc nulls last") { + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key nulls last, MAX(key)", + "sort_asc_nulls_last") + } + + test("order by desc nulls first") { + checkSQL("SELECT COUNT(value) FROM parquet_t1 GROUP BY key ORDER BY key desc nulls first," + + "MAX(key)", "sort_desc_nulls_first") + } + test("type widening in union") { checkSQL("SELECT id FROM parquet_t0 UNION ALL SELECT CAST(id AS INT) AS id FROM parquet_t0", "type_widening") @@ -697,6 +707,20 @@ class LogicalPlanToSQLSuite extends SQLBuilderTest with SQLTestUtils { |FROM parquet_t1 """.stripMargin, "window_basic_3") + + checkSQL( + """ + |SELECT key, value, ROUND(AVG(key) OVER (), 2) + |FROM parquet_t1 ORDER BY key nulls last + """.stripMargin, + "window_basic_asc_nulls_last") + + checkSQL( + """ + |SELECT key, value, ROUND(AVG(key) OVER (), 2) + |FROM parquet_t1 ORDER BY key desc nulls first + """.stripMargin, + "window_basic_desc_nulls_first") } test("multiple window functions in one expression") { From a63556fdddd135b10428fcc23c10b2f7de00cc5d Mon Sep 17 00:00:00 2001 From: Kishor Patil Date: Wed, 14 Sep 2016 14:19:35 -0500 Subject: [PATCH 38/83] [SPARK-17511] Yarn Dynamic Allocation: Avoid marking released container as Failed ## What changes were proposed in this pull request? Due to race conditions, the ` assert(numExecutorsRunning <= targetNumExecutors)` can fail causing `AssertionError`. So removed the assertion, instead moved the conditional check before launching new container: ``` java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.deploy.yarn.YarnAllocator$$anonfun$runAllocatedContainers$1.org$apache$spark$deploy$yarn$YarnAllocator$$anonfun$$updateInternalState$1(YarnAllocator.scala:489) at org.apache.spark.deploy.yarn.YarnAllocator$$anonfun$runAllocatedContainers$1$$anon$1.run(YarnAllocator.scala:519) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) at java.lang.Thread.run(Thread.java:745) ``` ## How was this patch tested? This was manually tested using a large ForkAndJoin job with Dynamic Allocation enabled to validate the failing job succeeds, without any such exception. Author: Kishor Patil Closes #15069 from kishorvpatil/SPARK-17511. --- .../spark/deploy/yarn/YarnAllocator.scala | 62 ++++++++++--------- .../deploy/yarn/YarnAllocatorSuite.scala | 19 ++++++ 2 files changed, 52 insertions(+), 29 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 2f4b498b3ca74..0b66d1cf08eac 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -496,7 +496,6 @@ private[yarn] class YarnAllocator( def updateInternalState(): Unit = synchronized { numExecutorsRunning += 1 - assert(numExecutorsRunning <= targetNumExecutors) executorIdToContainer(executorId) = container containerIdToExecutorId(container.getId) = executorId @@ -506,36 +505,41 @@ private[yarn] class YarnAllocator( allocatedContainerToHostMap.put(containerId, executorHostname) } - if (launchContainers) { - launcherPool.execute(new Runnable { - override def run(): Unit = { - try { - new ExecutorRunnable( - Some(container), - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores, - appAttemptId.getApplicationId.toString, - securityMgr, - localResources - ).run() - updateInternalState() - } catch { - case NonFatal(e) => - logError(s"Failed to launch executor $executorId on container $containerId", e) - // Assigned container should be released immediately to avoid unnecessary resource - // occupation. - amClient.releaseAssignedContainer(containerId) + if (numExecutorsRunning < targetNumExecutors) { + if (launchContainers) { + launcherPool.execute(new Runnable { + override def run(): Unit = { + try { + new ExecutorRunnable( + Some(container), + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores, + appAttemptId.getApplicationId.toString, + securityMgr, + localResources + ).run() + updateInternalState() + } catch { + case NonFatal(e) => + logError(s"Failed to launch executor $executorId on container $containerId", e) + // Assigned container should be released immediately to avoid unnecessary resource + // occupation. + amClient.releaseAssignedContainer(containerId) + } } - } - }) + }) + } else { + // For test only + updateInternalState() + } } else { - // For test only - updateInternalState() + logInfo(("Skip launching executorRunnable as runnning Excecutors count: %d " + + "reached target Executors count: %d.").format(numExecutorsRunning, targetNumExecutors)) } } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 696e552c35d12..994dc75d34c30 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -136,6 +136,25 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter size should be (0) } + test("container should not be created if requested number if met") { + // request a single container and receive it + val handler = createAllocator(1) + handler.updateResourceRequests() + handler.getNumExecutorsRunning should be (0) + handler.getPendingAllocate.size should be (1) + + val container = createContainer("host1") + handler.handleAllocatedContainers(Array(container)) + + handler.getNumExecutorsRunning should be (1) + handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") + handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) + + val container2 = createContainer("host2") + handler.handleAllocatedContainers(Array(container2)) + handler.getNumExecutorsRunning should be (1) + } + test("some containers allocated") { // request a few containers and receive some of them val handler = createAllocator(4) From e5814ccbe794eb589233fa73885c2a03e1de0f96 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 14 Sep 2016 13:33:51 -0700 Subject: [PATCH 39/83] [SPARK-17463][CORE] Make CollectionAccumulator and SetAccumulator's value can be read thread-safely ## What changes were proposed in this pull request? Make CollectionAccumulator and SetAccumulator's value can be read thread-safely to fix the ConcurrentModificationException reported in [JIRA](https://issues.apache.org/jira/browse/SPARK-17463). ## How was this patch tested? Existing tests. Author: Shixiong Zhu Closes #15063 from zsxwing/SPARK-17463. --- .../apache/spark/executor/TaskMetrics.scala | 41 ++++++++++++------- .../org/apache/spark/util/AccumulatorV2.scala | 7 +++- .../org/apache/spark/util/JsonProtocol.scala | 11 ++--- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- .../spark/sql/execution/debug/package.scala | 24 +++++++---- 5 files changed, 54 insertions(+), 32 deletions(-) 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 dd149a919fe55..52a349919e336 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,6 +17,9 @@ package org.apache.spark.executor +import java.util.{ArrayList, Collections} + +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import org.apache.spark._ @@ -99,7 +102,11 @@ class TaskMetrics private[spark] () extends Serializable { /** * Storage statuses of any blocks that have been updated as a result of this task. */ - def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = _updatedBlockStatuses.value + def updatedBlockStatuses: Seq[(BlockId, BlockStatus)] = { + // This is called on driver. All accumulator updates have a fixed value. So it's safe to use + // `asScala` which accesses the internal values using `java.util.Iterator`. + _updatedBlockStatuses.value.asScala + } // Setters and increment-ers private[spark] def setExecutorDeserializeTime(v: Long): Unit = @@ -114,8 +121,10 @@ class TaskMetrics private[spark] () extends Serializable { private[spark] def incPeakExecutionMemory(v: Long): Unit = _peakExecutionMemory.add(v) private[spark] def incUpdatedBlockStatuses(v: (BlockId, BlockStatus)): Unit = _updatedBlockStatuses.add(v) - private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + private[spark] def setUpdatedBlockStatuses(v: java.util.List[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v) + private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = + _updatedBlockStatuses.setValue(v.asJava) /** * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted @@ -268,7 +277,7 @@ private[spark] object TaskMetrics extends Logging { val name = info.name.get val value = info.update.get if (name == UPDATED_BLOCK_STATUSES) { - tm.setUpdatedBlockStatuses(value.asInstanceOf[Seq[(BlockId, BlockStatus)]]) + tm.setUpdatedBlockStatuses(value.asInstanceOf[java.util.List[(BlockId, BlockStatus)]]) } else { tm.nameToAccums.get(name).foreach( _.asInstanceOf[LongAccumulator].setValue(value.asInstanceOf[Long]) @@ -299,8 +308,8 @@ private[spark] object TaskMetrics extends Logging { private[spark] class BlockStatusesAccumulator - extends AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]] { - private var _seq = ArrayBuffer.empty[(BlockId, BlockStatus)] + extends AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]] { + private val _seq = Collections.synchronizedList(new ArrayList[(BlockId, BlockStatus)]()) override def isZero(): Boolean = _seq.isEmpty @@ -308,25 +317,27 @@ private[spark] class BlockStatusesAccumulator override def copy(): BlockStatusesAccumulator = { val newAcc = new BlockStatusesAccumulator - newAcc._seq = _seq.clone() + newAcc._seq.addAll(_seq) newAcc } override def reset(): Unit = _seq.clear() - override def add(v: (BlockId, BlockStatus)): Unit = _seq += v + override def add(v: (BlockId, BlockStatus)): Unit = _seq.add(v) - override def merge(other: AccumulatorV2[(BlockId, BlockStatus), Seq[(BlockId, BlockStatus)]]) - : Unit = other match { - case o: BlockStatusesAccumulator => _seq ++= o.value - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + override def merge( + other: AccumulatorV2[(BlockId, BlockStatus), java.util.List[(BlockId, BlockStatus)]]): Unit = { + other match { + case o: BlockStatusesAccumulator => _seq.addAll(o.value) + case _ => throw new UnsupportedOperationException( + s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") + } } - override def value: Seq[(BlockId, BlockStatus)] = _seq + override def value: java.util.List[(BlockId, BlockStatus)] = _seq - def setValue(newValue: Seq[(BlockId, BlockStatus)]): Unit = { + def setValue(newValue: java.util.List[(BlockId, BlockStatus)]): Unit = { _seq.clear() - _seq ++= newValue + _seq.addAll(newValue) } } diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index d130a37db5b5d..470d912ecff13 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import java.{lang => jl} import java.io.ObjectInputStream -import java.util.ArrayList +import java.util.{ArrayList, Collections} import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong @@ -38,6 +38,9 @@ private[spark] case class AccumulatorMetadata( /** * The base class for accumulators, that can accumulate inputs of type `IN`, and produce output of * type `OUT`. + * + * `OUT` should be a type that can be read atomically (e.g., Int, Long), or thread-safely + * (e.g., synchronized collections) because it will be read from other threads. */ abstract class AccumulatorV2[IN, OUT] extends Serializable { private[spark] var metadata: AccumulatorMetadata = _ @@ -433,7 +436,7 @@ class DoubleAccumulator extends AccumulatorV2[jl.Double, jl.Double] { * @since 2.0.0 */ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { - private val _list: java.util.List[T] = new ArrayList[T] + private val _list: java.util.List[T] = Collections.synchronizedList(new ArrayList[T]()) override def isZero: Boolean = _list.isEmpty 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 022b226894105..41d947c4428ad 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -310,11 +310,12 @@ private[spark] object JsonProtocol { case v: Int => JInt(v) case v: Long => JInt(v) // We only have 3 kind of internal accumulator types, so if it's not int or long, it must be - // the blocks accumulator, whose type is `Seq[(BlockId, BlockStatus)]` + // the blocks accumulator, whose type is `java.util.List[(BlockId, BlockStatus)]` case v => - JArray(v.asInstanceOf[Seq[(BlockId, BlockStatus)]].toList.map { case (id, status) => - ("Block ID" -> id.toString) ~ - ("Status" -> blockStatusToJson(status)) + JArray(v.asInstanceOf[java.util.List[(BlockId, BlockStatus)]].asScala.toList.map { + case (id, status) => + ("Block ID" -> id.toString) ~ + ("Status" -> blockStatusToJson(status)) }) } } else { @@ -743,7 +744,7 @@ private[spark] object JsonProtocol { val id = BlockId((blockJson \ "Block ID").extract[String]) val status = blockStatusFromJson(blockJson \ "Status") (id, status) - } + }.asJava case _ => throw new IllegalArgumentException(s"unexpected json value $value for " + "accumulator " + name.get) } 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 85ca9d39d4a3f..c89be22a34c9d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.util import java.util.Properties +import scala.collection.JavaConverters._ import scala.collection.Map import org.json4s.jackson.JsonMethods._ @@ -415,7 +416,7 @@ class JsonProtocolSuite extends SparkFunSuite { }) testAccumValue(Some(RESULT_SIZE), 3L, JInt(3)) testAccumValue(Some(shuffleRead.REMOTE_BLOCKS_FETCHED), 2, JInt(2)) - testAccumValue(Some(UPDATED_BLOCK_STATUSES), blocks, blocksJson) + testAccumValue(Some(UPDATED_BLOCK_STATUSES), blocks.asJava, blocksJson) // For anything else, we just cast the value to a string testAccumValue(Some("anything"), blocks, JString(blocks.toString)) testAccumValue(Some("anything"), 123, JString("123")) 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 082f97a8808fa..d321f4cd76877 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,7 +17,9 @@ package org.apache.spark.sql.execution -import scala.collection.mutable.HashSet +import java.util.Collections + +import scala.collection.JavaConverters._ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -107,18 +109,20 @@ package object debug { case class DebugExec(child: SparkPlan) extends UnaryExecNode with CodegenSupport { def output: Seq[Attribute] = child.output - class SetAccumulator[T] extends AccumulatorV2[T, HashSet[T]] { - private val _set = new HashSet[T]() + class SetAccumulator[T] extends AccumulatorV2[T, java.util.Set[T]] { + private val _set = Collections.synchronizedSet(new java.util.HashSet[T]()) override def isZero: Boolean = _set.isEmpty - override def copy(): AccumulatorV2[T, HashSet[T]] = { + override def copy(): AccumulatorV2[T, java.util.Set[T]] = { val newAcc = new SetAccumulator[T]() - newAcc._set ++= _set + newAcc._set.addAll(_set) newAcc } override def reset(): Unit = _set.clear() - override def add(v: T): Unit = _set += v - override def merge(other: AccumulatorV2[T, HashSet[T]]): Unit = _set ++= other.value - override def value: HashSet[T] = _set + override def add(v: T): Unit = _set.add(v) + override def merge(other: AccumulatorV2[T, java.util.Set[T]]): Unit = { + _set.addAll(other.value) + } + override def value: java.util.Set[T] = _set } /** @@ -138,7 +142,9 @@ package object debug { debugPrint(s"== ${child.simpleString} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => - val actualDataTypes = metric.elementTypes.value.mkString("{", ",", "}") + // This is called on driver. All accumulator updates have a fixed value. So it's safe to use + // `asScala` which accesses the internal values using `java.util.Iterator`. + val actualDataTypes = metric.elementTypes.value.asScala.mkString("{", ",", "}") debugPrint(s" ${attr.name} ${attr.dataType}: $actualDataTypes") } } From 3b80d1f2f045d924b312f3332e93471d95940d98 Mon Sep 17 00:00:00 2001 From: Eric Liang Date: Wed, 14 Sep 2016 13:37:35 -0700 Subject: [PATCH 40/83] [SPARK-17472] [PYSPARK] Better error message for serialization failures of large objects in Python ## What changes were proposed in this pull request? For large objects, pickle does not raise useful error messages. However, we can wrap them to be slightly more user friendly: Example 1: ``` def run(): import numpy.random as nr b = nr.bytes(8 * 1000000000) sc.parallelize(range(1000), 1000).map(lambda x: len(b)).count() run() ``` Before: ``` error: 'i' format requires -2147483648 <= number <= 2147483647 ``` After: ``` pickle.PicklingError: Object too large to serialize: 'i' format requires -2147483648 <= number <= 2147483647 ``` Example 2: ``` def run(): import numpy.random as nr b = sc.broadcast(nr.bytes(8 * 1000000000)) sc.parallelize(range(1000), 1000).map(lambda x: len(b.value)).count() run() ``` Before: ``` SystemError: error return without exception set ``` After: ``` cPickle.PicklingError: Could not serialize broadcast: SystemError: error return without exception set ``` ## How was this patch tested? Manually tried out these cases cc davies Author: Eric Liang Closes #15026 from ericl/spark-17472. --- python/pyspark/broadcast.py | 11 ++++++++++- python/pyspark/cloudpickle.py | 10 ++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index a0b819220e6d3..74dee1420754a 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -20,6 +20,8 @@ import gc from tempfile import NamedTemporaryFile +from pyspark.cloudpickle import print_exec + if sys.version < '3': import cPickle as pickle else: @@ -75,7 +77,14 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None): self._path = path def dump(self, value, f): - pickle.dump(value, f, 2) + try: + pickle.dump(value, f, 2) + except pickle.PickleError: + raise + except Exception as e: + msg = "Could not serialize broadcast: " + e.__class__.__name__ + ": " + e.message + print_exec(sys.stderr) + raise pickle.PicklingError(msg) f.close() return f.name diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index 822ae46e45111..da2b2f3757967 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -109,6 +109,16 @@ def dump(self, obj): if 'recursion' in e.args[0]: msg = """Could not pickle object as excessively deep recursion required.""" raise pickle.PicklingError(msg) + except pickle.PickleError: + raise + except Exception as e: + if "'i' format requires" in e.message: + msg = "Object too large to serialize: " + e.message + else: + msg = "Could not serialize object: " + e.__class__.__name__ + ": " + e.message + print_exec(sys.stderr) + raise pickle.PicklingError(msg) + def save_memoryview(self, obj): """Fallback to save_string""" From 795d83e98195f89f1fa6dcc020ad85eeb8db8bfd Mon Sep 17 00:00:00 2001 From: Xing SHI Date: Wed, 14 Sep 2016 13:46:46 -0700 Subject: [PATCH 41/83] [SPARK-17465][SPARK CORE] Inappropriate memory management in `org.apache.spark.storage.MemoryStore` may lead to memory leak The expression like `if (memoryMap(taskAttemptId) == 0) memoryMap.remove(taskAttemptId)` in method `releaseUnrollMemoryForThisTask` and `releasePendingUnrollMemoryForThisTask` should be called after release memory operation, whatever `memoryToRelease` is > 0 or not. If the memory of a task has been set to 0 when calling a `releaseUnrollMemoryForThisTask` or a `releasePendingUnrollMemoryForThisTask` method, the key in the memory map corresponding to that task will never be removed from the hash map. See the details in [SPARK-17465](https://issues.apache.org/jira/browse/SPARK-17465). Author: Xing SHI Closes #15022 from saturday-shi/SPARK-17465. --- .../scala/org/apache/spark/storage/memory/MemoryStore.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index baa3fde2d05f1..ec1b0f7149271 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -593,11 +593,11 @@ private[spark] class MemoryStore( val memoryToRelease = math.min(memory, unrollMemoryMap(taskAttemptId)) if (memoryToRelease > 0) { unrollMemoryMap(taskAttemptId) -= memoryToRelease - if (unrollMemoryMap(taskAttemptId) == 0) { - unrollMemoryMap.remove(taskAttemptId) - } memoryManager.releaseUnrollMemory(memoryToRelease, memoryMode) } + if (unrollMemoryMap(taskAttemptId) == 0) { + unrollMemoryMap.remove(taskAttemptId) + } } } } From f1a022363b14dbd3f3a92ed9a53c2d7232000ed6 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 15 Sep 2016 14:43:10 +0800 Subject: [PATCH 42/83] [SPARK-17440][SPARK-17441] Fixed Multiple Bugs in ALTER TABLE ### What changes were proposed in this pull request? For the following `ALTER TABLE` DDL, we should issue an exception when the target table is a `VIEW`: ```SQL ALTER TABLE viewName SET LOCATION '/path/to/your/lovely/heart' ALTER TABLE viewName SET SERDE 'whatever' ALTER TABLE viewName SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y') ALTER TABLE viewName ADD IF NOT EXISTS PARTITION (a='4', b='8') ALTER TABLE viewName DROP IF EXISTS PARTITION (a='2') ALTER TABLE viewName RECOVER PARTITIONS ALTER TABLE viewName PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p') ``` In addition, `ALTER TABLE RENAME PARTITION` is unable to handle data source tables, just like the other `ALTER PARTITION` commands. We should issue an exception instead. ### How was this patch tested? Added a few test cases. Author: gatorsmile Closes #15004 from gatorsmile/altertable. --- .../spark/sql/execution/command/ddl.scala | 45 +++++++++---- .../spark/sql/execution/command/tables.scala | 4 +- .../sql/execution/command/DDLSuite.scala | 63 +++++++++++++---- .../sql/hive/execution/HiveDDLSuite.scala | 67 ++++++++++--------- 4 files changed, 120 insertions(+), 59 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 dcda2f8d1c52a..c0ccdca98e05b 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 @@ -230,8 +230,8 @@ case class AlterTableSetPropertiesCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - DDLUtils.verifyAlterTableType(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView) // This overrides old properties val newTable = table.copy(properties = table.properties ++ properties) catalog.alterTable(newTable) @@ -258,8 +258,8 @@ case class AlterTableUnsetPropertiesCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - DDLUtils.verifyAlterTableType(catalog, tableName, isView) val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView) if (!ifExists) { propKeys.foreach { k => if (!table.properties.contains(k)) { @@ -299,6 +299,7 @@ case class AlterTableSerDePropertiesCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView = false) // For datasource tables, disallow setting serde or specifying partition if (partSpec.isDefined && DDLUtils.isDatasourceTable(table)) { throw new AnalysisException("Operation not allowed: ALTER TABLE SET " + @@ -348,6 +349,7 @@ case class AlterTableAddPartitionCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView = false) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( "ALTER TABLE ADD PARTITION is not allowed for tables defined using the datasource API") @@ -377,7 +379,14 @@ case class AlterTableRenamePartitionCommand( extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - sparkSession.sessionState.catalog.renamePartitions( + val catalog = sparkSession.sessionState.catalog + val table = catalog.getTableMetadata(tableName) + if (DDLUtils.isDatasourceTable(table)) { + throw new AnalysisException( + "ALTER TABLE RENAME PARTITION is not allowed for tables defined using the datasource API") + } + DDLUtils.verifyAlterTableType(catalog, table, isView = false) + catalog.renamePartitions( tableName, Seq(oldPartition), Seq(newPartition)) Seq.empty[Row] } @@ -408,6 +417,7 @@ case class AlterTableDropPartitionCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView = false) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( "ALTER TABLE DROP PARTITIONS is not allowed for tables defined using the datasource API") @@ -469,6 +479,7 @@ case class AlterTableRecoverPartitionsCommand( s"Operation not allowed: $cmd on temporary tables: $tableName") } val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView = false) if (DDLUtils.isDatasourceTable(table)) { throw new AnalysisException( s"Operation not allowed: $cmd on datasource tables: $tableName") @@ -644,6 +655,7 @@ case class AlterTableSetLocationCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog val table = catalog.getTableMetadata(tableName) + DDLUtils.verifyAlterTableType(catalog, table, isView = false) partitionSpec match { case Some(spec) => // Partition spec is specified, so we set the location only for this partition @@ -682,19 +694,26 @@ object DDLUtils { /** * If the command ALTER VIEW is to alter a table or ALTER TABLE is to alter a view, * issue an exception [[AnalysisException]]. + * + * Note: temporary views can be altered by both ALTER VIEW and ALTER TABLE commands, + * since temporary views can be also created by CREATE TEMPORARY TABLE. In the future, + * when we decided to drop the support, we should disallow users to alter temporary views + * by ALTER TABLE. */ def verifyAlterTableType( catalog: SessionCatalog, - tableIdentifier: TableIdentifier, + tableMetadata: CatalogTable, isView: Boolean): Unit = { - catalog.getTableMetadataOption(tableIdentifier).map(_.tableType match { - case CatalogTableType.VIEW if !isView => - throw new AnalysisException( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") - case o if o != CatalogTableType.VIEW && isView => - throw new AnalysisException( - s"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead") - case _ => - }) + if (!catalog.isTemporaryTable(tableMetadata.identifier)) { + tableMetadata.tableType match { + case CatalogTableType.VIEW if !isView => + throw new AnalysisException( + "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead") + case o if o != CatalogTableType.VIEW && isView => + throw new AnalysisException( + s"Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead") + case _ => + } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 9fbcd48b4a911..60e6b5db62a31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -158,7 +158,8 @@ case class AlterTableRenameCommand( override def run(sparkSession: SparkSession): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - DDLUtils.verifyAlterTableType(catalog, oldName, isView) + val table = catalog.getTableMetadata(oldName) + DDLUtils.verifyAlterTableType(catalog, table, isView) // If this is a temp view, just rename the view. // Otherwise, if this is a real table, we also need to uncache and invalidate the table. val isTemporary = catalog.isTemporaryTable(oldName) @@ -177,7 +178,6 @@ case class AlterTableRenameCommand( } } // For datasource tables, we also need to update the "path" serde property - val table = catalog.getTableMetadata(oldName) if (DDLUtils.isDatasourceTable(table) && table.tableType == CatalogTableType.MANAGED) { val newPath = catalog.defaultTablePath(newTblName) val newTable = table.withNewStorage( 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 95672e01f5546..4a171808c05ba 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 @@ -696,6 +696,18 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { assert(spark.table("teachers").collect().toSeq == df.collect().toSeq) } + test("rename temporary table") { + withTempView("tab1", "tab2") { + spark.range(10).createOrReplaceTempView("tab1") + sql("ALTER TABLE tab1 RENAME TO tab2") + checkAnswer(spark.table("tab2"), spark.range(10).toDF()) + intercept[NoSuchTableException] { spark.table("tab1") } + sql("ALTER VIEW tab2 RENAME TO tab1") + checkAnswer(spark.table("tab1"), spark.range(10).toDF()) + intercept[NoSuchTableException] { spark.table("tab2") } + } + } + test("rename temporary table - destination table already exists") { withTempView("tab1", "tab2") { sql( @@ -880,25 +892,16 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { test("alter table: rename partition") { val catalog = spark.sessionState.catalog val tableIdent = TableIdentifier("tab1", Some("dbx")) - val part1 = Map("a" -> "1", "b" -> "q") - val part2 = Map("a" -> "2", "b" -> "c") - val part3 = Map("a" -> "3", "b" -> "p") - createDatabase(catalog, "dbx") - createTable(catalog, tableIdent) - createTablePartition(catalog, part1, tableIdent) - createTablePartition(catalog, part2, tableIdent) - createTablePartition(catalog, part3, tableIdent) - assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == - Set(part1, part2, part3)) + createPartitionedTable(tableIdent, isDatasourceTable = false) sql("ALTER TABLE dbx.tab1 PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p')") - sql("ALTER TABLE dbx.tab1 PARTITION (a='2', b='c') RENAME TO PARTITION (a='200', b='c')") + sql("ALTER TABLE dbx.tab1 PARTITION (a='2', b='c') RENAME TO PARTITION (a='20', b='c')") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == - Set(Map("a" -> "100", "b" -> "p"), Map("a" -> "200", "b" -> "c"), part3)) + Set(Map("a" -> "100", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) // rename without explicitly specifying database catalog.setCurrentDatabase("dbx") sql("ALTER TABLE tab1 PARTITION (a='100', b='p') RENAME TO PARTITION (a='10', b='p')") assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == - Set(Map("a" -> "10", "b" -> "p"), Map("a" -> "200", "b" -> "c"), part3)) + Set(Map("a" -> "10", "b" -> "p"), Map("a" -> "20", "b" -> "c"), Map("a" -> "3", "b" -> "p"))) // table to alter does not exist intercept[NoSuchTableException] { sql("ALTER TABLE does_not_exist PARTITION (c='3') RENAME TO PARTITION (c='333')") @@ -909,6 +912,38 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } } + test("alter table: rename partition (datasource table)") { + createPartitionedTable(TableIdentifier("tab1", Some("dbx")), isDatasourceTable = true) + val e = intercept[AnalysisException] { + sql("ALTER TABLE dbx.tab1 PARTITION (a='1', b='q') RENAME TO PARTITION (a='100', b='p')") + }.getMessage + assert(e.contains( + "ALTER TABLE RENAME PARTITION is not allowed for tables defined using the datasource API")) + // table to alter does not exist + intercept[NoSuchTableException] { + sql("ALTER TABLE does_not_exist PARTITION (c='3') RENAME TO PARTITION (c='333')") + } + } + + private def createPartitionedTable( + tableIdent: TableIdentifier, + isDatasourceTable: Boolean): Unit = { + val catalog = spark.sessionState.catalog + val part1 = Map("a" -> "1", "b" -> "q") + val part2 = Map("a" -> "2", "b" -> "c") + val part3 = Map("a" -> "3", "b" -> "p") + createDatabase(catalog, "dbx") + createTable(catalog, tableIdent) + createTablePartition(catalog, part1, tableIdent) + createTablePartition(catalog, part2, tableIdent) + createTablePartition(catalog, part3, tableIdent) + assert(catalog.listPartitions(tableIdent).map(_.spec).toSet == + Set(part1, part2, part3)) + if (isDatasourceTable) { + convertToDatasourceTable(catalog, tableIdent) + } + } + test("show tables") { withTempView("show1a", "show2b") { sql( @@ -1255,7 +1290,7 @@ class DDLSuite extends QueryTest with SharedSQLContext with BeforeAndAfterEach { } // table to alter does not exist intercept[AnalysisException] { - sql("ALTER TABLE does_not_exist SET SERDEPROPERTIES ('x' = 'y')") + sql("ALTER TABLE does_not_exist PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") } } 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 3cba5b2a097f1..aa35a335facbf 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 @@ -305,6 +305,16 @@ class HiveDDLSuite } } + private def assertErrorForAlterTableOnView(sqlText: String): Unit = { + val message = intercept[AnalysisException](sql(sqlText)).getMessage + assert(message.contains("Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + } + + private def assertErrorForAlterViewOnTable(sqlText: String): Unit = { + val message = intercept[AnalysisException](sql(sqlText)).getMessage + assert(message.contains("Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + } + test("alter views and alter table - misuse") { val tabName = "tab1" withTable(tabName) { @@ -317,45 +327,42 @@ class HiveDDLSuite assert(catalog.tableExists(TableIdentifier(tabName))) assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) - var message = intercept[AnalysisException] { - sql(s"ALTER VIEW $tabName RENAME TO $newViewName") - }.getMessage - assert(message.contains( - "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName RENAME TO $newViewName") - message = intercept[AnalysisException] { - sql(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')") - }.getMessage - assert(message.contains( - "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName RENAME TO $newViewName") - message = intercept[AnalysisException] { - sql(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')") - }.getMessage - assert(message.contains( - "Cannot alter a table with ALTER VIEW. Please use ALTER TABLE instead")) + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName SET TBLPROPERTIES ('p' = 'an')") - message = intercept[AnalysisException] { - sql(s"ALTER TABLE $oldViewName RENAME TO $newViewName") - }.getMessage - assert(message.contains( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") - message = intercept[AnalysisException] { - sql(s"ALTER TABLE $oldViewName SET TBLPROPERTIES ('p' = 'an')") - }.getMessage - assert(message.contains( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + assertErrorForAlterViewOnTable(s"ALTER VIEW $tabName UNSET TBLPROPERTIES ('p')") - message = intercept[AnalysisException] { - sql(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") - }.getMessage - assert(message.contains( - "Cannot alter a view with ALTER TABLE. Please use ALTER VIEW instead")) + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName UNSET TBLPROPERTIES ('p')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET LOCATION '/path/to/home'") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDE 'whatever'") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName SET SERDEPROPERTIES ('x' = 'y')") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName PARTITION (a=1, b=2) SET SERDEPROPERTIES ('x' = 'y')") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName ADD IF NOT EXISTS PARTITION (a='4', b='8')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName DROP IF EXISTS PARTITION (a='2')") + + assertErrorForAlterTableOnView(s"ALTER TABLE $oldViewName RECOVER PARTITIONS") + + assertErrorForAlterTableOnView( + s"ALTER TABLE $oldViewName PARTITION (a='1') RENAME TO PARTITION (a='100')") assert(catalog.tableExists(TableIdentifier(tabName))) assert(catalog.tableExists(TableIdentifier(oldViewName))) + assert(!catalog.tableExists(TableIdentifier(newViewName))) } } } From de885a822e7db2e6e9b9b950f78f9bc8e11c2e8c Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 15 Sep 2016 09:30:15 +0100 Subject: [PATCH 43/83] [SPARK-17507][ML][MLLIB] check weight vector size in ANN ## What changes were proposed in this pull request? as the TODO described, check weight vector size and if wrong throw exception. ## How was this patch tested? existing tests. Author: WeichenXu Closes #15060 from WeichenXu123/check_input_weight_size_of_ann. --- .../src/main/scala/org/apache/spark/ml/ann/Layer.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 88909a9fb953f..e7e0dae0b5a01 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -545,7 +545,9 @@ private[ann] object FeedForwardModel { * @return model */ def apply(topology: FeedForwardTopology, weights: Vector): FeedForwardModel = { - // TODO: check that weights size is equal to sum of layers sizes + val expectedWeightSize = topology.layers.map(_.weightSize).sum + require(weights.size == expectedWeightSize, + s"Expected weight vector of size ${expectedWeightSize} but got size ${weights.size}.") new FeedForwardModel(weights, topology) } @@ -559,11 +561,7 @@ private[ann] object FeedForwardModel { def apply(topology: FeedForwardTopology, seed: Long = 11L): FeedForwardModel = { val layers = topology.layers val layerModels = new Array[LayerModel](layers.length) - var totalSize = 0 - for (i <- 0 until topology.layers.length) { - totalSize += topology.layers(i).weightSize - } - val weights = BDV.zeros[Double](totalSize) + val weights = BDV.zeros[Double](topology.layers.map(_.weightSize).sum) var offset = 0 val random = new XORShiftRandom(seed) for (i <- 0 until layers.length) { From 5ee52aacea8bceb81712f8cc6eef4a230d7c78f3 Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Thu, 15 Sep 2016 09:37:12 +0100 Subject: [PATCH 44/83] [SPARK-17524][TESTS] Use specified spark.buffer.pageSize ## What changes were proposed in this pull request? This PR has the appendRowUntilExceedingPageSize test in RowBasedKeyValueBatchSuite use whatever spark.buffer.pageSize value a user has specified to prevent a test failure for anyone testing Apache Spark on a box with a reduced page size. The test is currently hardcoded to use the default page size which is 64 MB so this minor PR is a test improvement ## How was this patch tested? Existing unit tests with 1 MB page size and with 64 MB (the default) page size Author: Adam Roberts Closes #15079 from a-roberts/patch-5. --- .../catalyst/expressions/RowBasedKeyValueBatchSuite.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java index 0dd129cea7b3f..fb3dbe8ed1996 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java @@ -338,15 +338,17 @@ public void appendRowUntilExceedingCapacity() throws Exception { @Test public void appendRowUntilExceedingPageSize() throws Exception { + // Use default size or spark.buffer.pageSize if specified + int pageSizeToUse = (int) memoryManager.pageSizeBytes(); RowBasedKeyValueBatch batch = RowBasedKeyValueBatch.allocate(keySchema, - valueSchema, taskMemoryManager, 64 * 1024 * 1024); //enough capacity + valueSchema, taskMemoryManager, pageSizeToUse); //enough capacity try { UnsafeRow key = makeKeyRow(1, "A"); UnsafeRow value = makeValueRow(1, 1); int recordLength = 8 + key.getSizeInBytes() + value.getSizeInBytes() + 8; int totalSize = 4; int numRows = 0; - while (totalSize + recordLength < 64 * 1024 * 1024) { // default page size + while (totalSize + recordLength < pageSizeToUse) { appendRow(batch, key, value); totalSize += recordLength; numRows++; From 037565fa44eebdf3336e69830517a163f88a94d6 Mon Sep 17 00:00:00 2001 From: codlife <1004910847@qq.com> Date: Thu, 15 Sep 2016 09:38:13 +0100 Subject: [PATCH 45/83] [SPARK-17521] Error when I use sparkContext.makeRDD(Seq()) ## What changes were proposed in this pull request? when i use sc.makeRDD below ``` val data3 = sc.makeRDD(Seq()) println(data3.partitions.length) ``` I got an error: Exception in thread "main" java.lang.IllegalArgumentException: Positive number of slices required We can fix this bug just modify the last line ,do a check of seq.size ``` def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap new ParallelCollectionRDD[T](this, seq.map(_._1), math.max(seq.size, defaultParallelism), indexToPrefs) } ``` ## How was this patch tested? manual tests (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Author: codlife <1004910847@qq.com> Author: codlife Closes #15077 from codlife/master. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e32e4aa5b8312..35b6334832393 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -795,7 +795,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def makeRDD[T: ClassTag](seq: Seq[(T, Seq[String])]): RDD[T] = withScope { assertNotStopped() val indexToPrefs = seq.zipWithIndex.map(t => (t._2, t._1._2)).toMap - new ParallelCollectionRDD[T](this, seq.map(_._1), seq.size, indexToPrefs) + new ParallelCollectionRDD[T](this, seq.map(_._1), math.max(seq.size, 1), indexToPrefs) } /** From 6f0e76029f5a7fc338aaa06a5f85b904d50aa79f Mon Sep 17 00:00:00 2001 From: cenyuhai Date: Thu, 15 Sep 2016 09:58:53 +0100 Subject: [PATCH 46/83] [SPARK-17406][WEB UI] limit timeline executor events ## What changes were proposed in this pull request? The job page will be too slow to open when there are thousands of executor events(added or removed). I found that in ExecutorsTab file, executorIdToData will not remove elements, it will increase all the time.Before this pr, it looks like [timeline1.png](https://issues.apache.org/jira/secure/attachment/12827112/timeline1.png). After this pr, it looks like [timeline2.png](https://issues.apache.org/jira/secure/attachment/12827113/timeline2.png)(we can set how many executor events will be displayed) Author: cenyuhai Closes #14969 from cenyuhai/SPARK-17406. --- .../apache/spark/ui/exec/ExecutorsPage.scala | 41 +++---- .../apache/spark/ui/exec/ExecutorsTab.scala | 112 +++++++++++------- .../apache/spark/ui/jobs/AllJobsPage.scala | 66 +++++------ .../apache/spark/ui/jobs/ExecutorTable.scala | 3 +- .../org/apache/spark/ui/jobs/JobPage.scala | 67 ++++++----- .../org/apache/spark/ui/jobs/StagePage.scala | 4 +- .../org/apache/spark/ui/jobs/UIData.scala | 5 - project/MimaExcludes.scala | 12 ++ 8 files changed, 162 insertions(+), 148 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 982e8915a8ded..7953d77fd7ece 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -17,14 +17,12 @@ package org.apache.spark.ui.exec -import java.net.URLEncoder import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.status.api.v1.ExecutorSummary -import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} -import org.apache.spark.util.Utils +import org.apache.spark.ui.{UIUtils, WebUIPage} // This isn't even used anymore -- but we need to keep it b/c of a MiMa false positive private[ui] case class ExecutorSummaryInfo( @@ -83,18 +81,7 @@ private[spark] object ExecutorsPage { val memUsed = status.memUsed val maxMem = status.maxMem val diskUsed = status.diskUsed - val totalCores = listener.executorToTotalCores.getOrElse(execId, 0) - val maxTasks = listener.executorToTasksMax.getOrElse(execId, 0) - val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) - val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) - val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) - val totalTasks = activeTasks + failedTasks + completedTasks - val totalDuration = listener.executorToDuration.getOrElse(execId, 0L) - val totalGCTime = listener.executorToJvmGCTime.getOrElse(execId, 0L) - val totalInputBytes = listener.executorToInputBytes.getOrElse(execId, 0L) - val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0L) - val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0L) - val executorLogs = listener.executorToLogUrls.getOrElse(execId, Map.empty) + val taskSummary = listener.executorToTaskSummary.getOrElse(execId, ExecutorTaskSummary(execId)) new ExecutorSummary( execId, @@ -103,19 +90,19 @@ private[spark] object ExecutorsPage { rddBlocks, memUsed, diskUsed, - totalCores, - maxTasks, - activeTasks, - failedTasks, - completedTasks, - totalTasks, - totalDuration, - totalGCTime, - totalInputBytes, - totalShuffleRead, - totalShuffleWrite, + taskSummary.totalCores, + taskSummary.tasksMax, + taskSummary.tasksActive, + taskSummary.tasksFailed, + taskSummary.tasksComplete, + taskSummary.tasksActive + taskSummary.tasksFailed + taskSummary.tasksComplete, + taskSummary.duration, + taskSummary.jvmGCTime, + taskSummary.inputBytes, + taskSummary.shuffleRead, + taskSummary.shuffleWrite, maxMem, - executorLogs + taskSummary.executorLogs ) } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index 676f4457510c2..678571fd4f5ac 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -17,14 +17,13 @@ package org.apache.spark.ui.exec -import scala.collection.mutable.HashMap +import scala.collection.mutable.{LinkedHashMap, ListBuffer} import org.apache.spark.{ExceptionFailure, Resubmitted, SparkConf, SparkContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageStatus, StorageStatusListener} import org.apache.spark.ui.{SparkUI, SparkUITab} -import org.apache.spark.ui.jobs.UIData.ExecutorUIData private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "executors") { val listener = parent.executorsListener @@ -38,6 +37,25 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec } } +private[ui] case class ExecutorTaskSummary( + var executorId: String, + var totalCores: Int = 0, + var tasksMax: Int = 0, + var tasksActive: Int = 0, + var tasksFailed: Int = 0, + var tasksComplete: Int = 0, + var duration: Long = 0L, + var jvmGCTime: Long = 0L, + var inputBytes: Long = 0L, + var inputRecords: Long = 0L, + var outputBytes: Long = 0L, + var outputRecords: Long = 0L, + var shuffleRead: Long = 0L, + var shuffleWrite: Long = 0L, + var executorLogs: Map[String, String] = Map.empty, + var isAlive: Boolean = true +) + /** * :: DeveloperApi :: * A SparkListener that prepares information to be displayed on the ExecutorsTab @@ -45,21 +63,11 @@ private[ui] class ExecutorsTab(parent: SparkUI) extends SparkUITab(parent, "exec @DeveloperApi class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: SparkConf) extends SparkListener { - val executorToTotalCores = HashMap[String, Int]() - val executorToTasksMax = HashMap[String, Int]() - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToJvmGCTime = HashMap[String, Long]() - val executorToInputBytes = HashMap[String, Long]() - val executorToInputRecords = HashMap[String, Long]() - val executorToOutputBytes = HashMap[String, Long]() - val executorToOutputRecords = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - val executorToLogUrls = HashMap[String, Map[String, String]]() - val executorIdToData = HashMap[String, ExecutorUIData]() + var executorToTaskSummary = LinkedHashMap[String, ExecutorTaskSummary]() + var executorEvents = new ListBuffer[SparkListenerEvent]() + + private val maxTimelineExecutors = conf.getInt("spark.ui.timeline.executors.maximum", 1000) + private val retainedDeadExecutors = conf.getInt("spark.ui.retainedDeadExecutors", 100) def activeStorageStatusList: Seq[StorageStatus] = storageStatusListener.storageStatusList @@ -67,18 +75,29 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = synchronized { val eid = executorAdded.executorId - executorToLogUrls(eid) = executorAdded.executorInfo.logUrlMap - executorToTotalCores(eid) = executorAdded.executorInfo.totalCores - executorToTasksMax(eid) = executorToTotalCores(eid) / conf.getInt("spark.task.cpus", 1) - executorIdToData(eid) = new ExecutorUIData(executorAdded.time) + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.executorLogs = executorAdded.executorInfo.logUrlMap + taskSummary.totalCores = executorAdded.executorInfo.totalCores + taskSummary.tasksMax = taskSummary.totalCores / conf.getInt("spark.task.cpus", 1) + executorEvents += executorAdded + if (executorEvents.size > maxTimelineExecutors) { + executorEvents.remove(0) + } + + val deadExecutors = executorToTaskSummary.filter(e => !e._2.isAlive) + if (deadExecutors.size > retainedDeadExecutors) { + val head = deadExecutors.head + executorToTaskSummary.remove(head._1) + } } override def onExecutorRemoved( executorRemoved: SparkListenerExecutorRemoved): Unit = synchronized { - val eid = executorRemoved.executorId - val uiData = executorIdToData(eid) - uiData.finishTime = Some(executorRemoved.time) - uiData.finishReason = Some(executorRemoved.reason) + executorEvents += executorRemoved + if (executorEvents.size > maxTimelineExecutors) { + executorEvents.remove(0) + } + executorToTaskSummary.get(executorRemoved.executorId).foreach(e => e.isAlive = false) } override def onApplicationStart(applicationStart: SparkListenerApplicationStart): Unit = { @@ -87,19 +106,25 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar s.blockManagerId.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER || s.blockManagerId.executorId == SparkContext.DRIVER_IDENTIFIER } - storageStatus.foreach { s => executorToLogUrls(s.blockManagerId.executorId) = logs.toMap } + storageStatus.foreach { s => + val eid = s.blockManagerId.executorId + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.executorLogs = logs.toMap + } } } override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { val eid = taskStart.taskInfo.executorId - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) + taskSummary.tasksActive += 1 } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { val info = taskEnd.taskInfo if (info != null) { val eid = info.executorId + val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) taskEnd.reason match { case Resubmitted => // Note: For resubmitted tasks, we continue to use the metrics that belong to the @@ -108,31 +133,26 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar // metrics added by each attempt, but this is much more complicated. return case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + taskSummary.tasksFailed += 1 case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + taskSummary.tasksComplete += 1 } - - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + if (taskSummary.tasksActive >= 1) { + taskSummary.tasksActive -= 1 + } + taskSummary.duration += info.duration // Update shuffle read/write val metrics = taskEnd.taskMetrics if (metrics != null) { - executorToInputBytes(eid) = - executorToInputBytes.getOrElse(eid, 0L) + metrics.inputMetrics.bytesRead - executorToInputRecords(eid) = - executorToInputRecords.getOrElse(eid, 0L) + metrics.inputMetrics.recordsRead - executorToOutputBytes(eid) = - executorToOutputBytes.getOrElse(eid, 0L) + metrics.outputMetrics.bytesWritten - executorToOutputRecords(eid) = - executorToOutputRecords.getOrElse(eid, 0L) + metrics.outputMetrics.recordsWritten - - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + metrics.shuffleReadMetrics.remoteBytesRead - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + metrics.shuffleWriteMetrics.bytesWritten - executorToJvmGCTime(eid) = executorToJvmGCTime.getOrElse(eid, 0L) + metrics.jvmGCTime + taskSummary.inputBytes += metrics.inputMetrics.bytesRead + taskSummary.inputRecords += metrics.inputMetrics.recordsRead + taskSummary.outputBytes += metrics.outputMetrics.bytesWritten + taskSummary.outputRecords += metrics.outputMetrics.recordsWritten + + taskSummary.shuffleRead += metrics.shuffleReadMetrics.remoteBytesRead + taskSummary.shuffleWrite += metrics.shuffleWriteMetrics.bytesWritten + taskSummary.jvmGCTime += metrics.jvmGCTime } } } 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 e5363ce8ca9dc..c04964ec66479 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 @@ -28,9 +28,9 @@ import scala.xml._ import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.scheduler._ import org.apache.spark.ui._ -import org.apache.spark.ui.jobs.UIData.{ExecutorUIData, JobUIData, StageUIData} +import org.apache.spark.ui.jobs.UIData.{JobUIData, StageUIData} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished jobs */ @@ -123,55 +123,55 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { } } - private def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + private def makeExecutorEvent(executorUIDatas: Seq[SparkListenerEvent]): + Seq[String] = { val events = ListBuffer[String]() executorUIDatas.foreach { - case (executorId, event) => + case a: SparkListenerExecutorAdded => val addedEvent = s""" |{ | 'className': 'executor added', | 'group': 'executors', - | 'start': new Date(${event.startTime}), + | 'start': new Date(${a.time}), | 'content': '
    Executor ${executorId} added
    ' + | 'data-title="Executor ${a.executorId}
    ' + + | 'Added at ${UIUtils.formatDate(new Date(a.time))}"' + + | 'data-html="true">Executor ${a.executorId} added
    ' |} """.stripMargin events += addedEvent + case e: SparkListenerExecutorRemoved => + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${e.time}), + | 'content': '
    Reason: ${e.reason.replace("\n", " ")}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${e.executorId} removed
    ' + |} + """.stripMargin + events += removedEvent - if (event.finishTime.isDefined) { - val removedEvent = - s""" - |{ - | 'className': 'executor removed', - | 'group': 'executors', - | 'start': new Date(${event.finishTime.get}), - | 'content': '
    Reason: ${event.finishReason.get.replace("\n", " ")}""" - } else { - "" - } - }"' + - | 'data-html="true">Executor ${executorId} removed
    ' - |} - """.stripMargin - events += removedEvent - } } events.toSeq } private def makeTimeline( jobs: Seq[JobUIData], - executors: HashMap[String, ExecutorUIData], + executors: Seq[SparkListenerEvent], startTime: Long): Seq[Node] = { val jobEventJsonAsStrSeq = makeJobEvent(jobs) @@ -353,7 +353,7 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { var content = summary val executorListener = parent.executorListener content ++= makeTimeline(activeJobs ++ completedJobs ++ failedJobs, - executorListener.executorIdToData, startTime) + executorListener.executorEvents, startTime) if (shouldShowActiveJobs) { content ++=

    Active Jobs ({activeJobs.size})

    ++ diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 133c3b1b9aca8..9fb3f35fd9685 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -118,7 +118,8 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage
    {k}
    { - val logs = parent.executorsListener.executorToLogUrls.getOrElse(k, Map.empty) + val logs = parent.executorsListener.executorToTaskSummary.get(k) + .map(_.executorLogs).getOrElse(Map.empty) logs.map { case (logName, logUrl) => } 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 0ec42d68d3dcc..2f7f8976a8899 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 @@ -20,15 +20,14 @@ package org.apache.spark.ui.jobs import java.util.Date import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.{Buffer, HashMap, ListBuffer} +import scala.collection.mutable.{Buffer, ListBuffer} import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler.StageInfo +import org.apache.spark.scheduler._ import org.apache.spark.ui.{ToolTips, UIUtils, WebUIPage} -import org.apache.spark.ui.jobs.UIData.ExecutorUIData /** Page showing statistics and stage list for a given job */ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { @@ -93,55 +92,55 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { } } - def makeExecutorEvent(executorUIDatas: HashMap[String, ExecutorUIData]): Seq[String] = { + def makeExecutorEvent(executorUIDatas: Seq[SparkListenerEvent]): Seq[String] = { val events = ListBuffer[String]() executorUIDatas.foreach { - case (executorId, event) => + case a: SparkListenerExecutorAdded => val addedEvent = s""" |{ | 'className': 'executor added', | 'group': 'executors', - | 'start': new Date(${event.startTime}), + | 'start': new Date(${a.time}), | 'content': '
    Executor ${executorId} added
    ' + | 'data-title="Executor ${a.executorId}
    ' + + | 'Added at ${UIUtils.formatDate(new Date(a.time))}"' + + | 'data-html="true">Executor ${a.executorId} added
    ' |} """.stripMargin events += addedEvent - if (event.finishTime.isDefined) { - val removedEvent = - s""" - |{ - | 'className': 'executor removed', - | 'group': 'executors', - | 'start': new Date(${event.finishTime.get}), - | 'content': '
    Reason: ${event.finishReason.get.replace("\n", " ")}""" - } else { - "" - } - }"' + - | 'data-html="true">Executor ${executorId} removed
    ' - |} - """.stripMargin - events += removedEvent - } + case e: SparkListenerExecutorRemoved => + val removedEvent = + s""" + |{ + | 'className': 'executor removed', + | 'group': 'executors', + | 'start': new Date(${e.time}), + | 'content': '
    Reason: ${e.reason.replace("\n", " ")}""" + } else { + "" + } + }"' + + | 'data-html="true">Executor ${e.executorId} removed
    ' + |} + """.stripMargin + events += removedEvent + } events.toSeq } private def makeTimeline( stages: Seq[StageInfo], - executors: HashMap[String, ExecutorUIData], + executors: Seq[SparkListenerEvent], appStartTime: Long): Seq[Node] = { val stageEventJsonAsStrSeq = makeStageEvent(stages) @@ -319,7 +318,7 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val operationGraphListener = parent.operationGraphListener content ++= makeTimeline(activeStages ++ completedStages ++ failedStages, - executorListener.executorIdToData, appStartTime) + executorListener.executorEvents, appStartTime) content ++= UIUtils.showDagVizForJob( jobId, operationGraphListener.getOperationGraphForJob(jobId)) 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 de787f257737d..c322ae0972ad7 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 @@ -1017,8 +1017,8 @@ private[ui] class TaskDataSource( None } - val logs = executorsListener.executorToLogUrls.getOrElse(info.executorId, Map.empty) - + val logs = executorsListener.executorToTaskSummary.get(info.executorId) + .map(_.executorLogs).getOrElse(Map.empty) new TaskTableRowData( info.index, info.taskId, diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 74bca9931acf7..c729f03b3c383 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -177,11 +177,6 @@ private[spark] object UIData { } } - class ExecutorUIData( - val startTime: Long, - var finishTime: Option[Long] = None, - var finishReason: Option[String] = None) - case class TaskMetricsUIData( executorDeserializeTime: Long, executorRunTime: Long, diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index fbd78aeb20dd6..37fff2efa4eae 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -426,6 +426,18 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorIdToData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksActive"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksComplete"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksFailed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleWrite"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToDuration"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToLogUrls"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputRecords"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), From 74bf9a2d34556ebd1921c6eb23bca0831670c7ac Mon Sep 17 00:00:00 2001 From: John Muller Date: Thu, 15 Sep 2016 10:00:28 +0100 Subject: [PATCH 47/83] [SPARK-17536][SQL] Minor performance improvement to JDBC batch inserts ## What changes were proposed in this pull request? Optimize a while loop during batch inserts ## How was this patch tested? Unit tests were done, specifically "mvn test" for sql Author: John Muller Closes #15098 from blue666man/SPARK-17536. --- .../apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala | 2 +- 1 file changed, 1 insertion(+), 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 132472ad0ce87..b09fd511a9074 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 @@ -590,12 +590,12 @@ object JdbcUtils extends Logging { val stmt = insertStatement(conn, table, rddSchema, dialect) val setters: Array[JDBCValueSetter] = rddSchema.fields.map(_.dataType) .map(makeSetter(conn, dialect, _)).toArray + val numFields = rddSchema.fields.length try { var rowCount = 0 while (iterator.hasNext) { val row = iterator.next() - val numFields = rddSchema.fields.length var i = 0 while (i < numFields) { if (row.isNullAt(i)) { From 396a6ce38dc9f582f2c41deccc4ff250d421132e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 15 Sep 2016 13:54:41 +0100 Subject: [PATCH 48/83] [SPARK-17406][BUILD][HOTFIX] MiMa excludes fix ## What changes were proposed in this pull request? Following https://github.com/apache/spark/pull/14969 for some reason the MiMa excludes weren't complete, but still passed the PR builder. This adds 3 more excludes from https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.2/1749/consoleFull It also moves the excludes to their own Seq in the build, as they probably should have been. Even though this is merged to 2.1.x only / master, I left the exclude in for 2.0.x in case we back port. It's a private API so is always a false positive. ## How was this patch tested? Jenkins build Author: Sean Owen Closes #15110 from srowen/SPARK-17406.2. --- project/MimaExcludes.scala | 29 +++++++++++++++++------------ 1 file changed, 17 insertions(+), 12 deletions(-) diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 37fff2efa4eae..1bdcf9a623dc9 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -426,18 +426,6 @@ object MimaExcludes { ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.storage.StorageStatusListener.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.streaming.scheduler.BatchInfo.streamIdToNumRecords"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.storageStatusList"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorIdToData"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksActive"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksComplete"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputRecords"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleRead"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksFailed"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleWrite"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToDuration"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputBytes"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToLogUrls"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputBytes"), - ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputRecords"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.this"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.storage.StorageListener.storageStatusList"), ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ExceptionFailure.apply"), @@ -807,6 +795,23 @@ object MimaExcludes { // SPARK-17096: Improve exception string reported through the StreamingQueryListener ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.stackTrace"), ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.streaming.StreamingQueryListener#QueryTerminated.this") + ) ++ Seq( + // SPARK-17406 limit timeline executor events + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorIdToData"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksActive"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksComplete"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleRead"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksFailed"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToShuffleWrite"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToDuration"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToInputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToLogUrls"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputBytes"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToOutputRecords"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTotalCores"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToTasksMax"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.ui.exec.ExecutorsListener.executorToJvmGCTime") ) } From 4b293405578d96a88f70f5fddff9260dd01dd3a3 Mon Sep 17 00:00:00 2001 From: Tejas Patil Date: Thu, 15 Sep 2016 10:23:41 -0700 Subject: [PATCH 49/83] [SPARK-17451][CORE] CoarseGrainedExecutorBackend should inform driver before self-kill ## What changes were proposed in this pull request? Jira : https://issues.apache.org/jira/browse/SPARK-17451 `CoarseGrainedExecutorBackend` in some failure cases exits the JVM. While this does not have any issue, from the driver UI there is no specific reason captured for this. In this PR, I am adding functionality to `exitExecutor` to notify driver that the executor is exiting. ## How was this patch tested? Ran the change over a test env and took down shuffle service before the executor could register to it. In the driver logs, where the job failure reason is mentioned (ie. `Job aborted due to stage ...` it gives the correct reason: Before: `ExecutorLostFailure (executor ZZZZZZZZZ exited caused by one of the running tasks) Reason: Remote RPC client disassociated. Likely due to containers exceeding thresholds, or network issues. Check driver logs for WARN messages.` After: `ExecutorLostFailure (executor ZZZZZZZZZ exited caused by one of the running tasks) Reason: Unable to create executor due to java.util.concurrent.TimeoutException: Timeout waiting for task.` Author: Tejas Patil Closes #15013 from tejasapatil/SPARK-17451_inform_driver. --- .../CoarseGrainedExecutorBackend.scala | 26 ++++++++++++++----- .../apache/spark/storage/BlockManager.scala | 3 +++ 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 391b97d73e026..7eec4ae64f296 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -31,7 +31,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.internal.Logging import org.apache.spark.rpc._ -import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ import org.apache.spark.serializer.SerializerInstance import org.apache.spark.util.{ThreadUtils, Utils} @@ -65,7 +65,7 @@ private[spark] class CoarseGrainedExecutorBackend( case Success(msg) => // Always receive `true`. Just ignore it case Failure(e) => - exitExecutor(1, s"Cannot register with driver: $driverUrl", e) + exitExecutor(1, s"Cannot register with driver: $driverUrl", e, notifyDriver = false) }(ThreadUtils.sameThread) } @@ -129,7 +129,8 @@ private[spark] class CoarseGrainedExecutorBackend( if (stopping.get()) { logInfo(s"Driver from $remoteAddress disconnected during shutdown") } else if (driver.exists(_.address == remoteAddress)) { - exitExecutor(1, s"Driver $remoteAddress disassociated! Shutting down.") + exitExecutor(1, s"Driver $remoteAddress disassociated! Shutting down.", null, + notifyDriver = false) } else { logWarning(s"An unknown ($remoteAddress) driver disconnected.") } @@ -148,12 +149,25 @@ private[spark] class CoarseGrainedExecutorBackend( * executor exits differently. For e.g. when an executor goes down, * back-end may not want to take the parent process down. */ - protected def exitExecutor(code: Int, reason: String, throwable: Throwable = null) = { + protected def exitExecutor(code: Int, + reason: String, + throwable: Throwable = null, + notifyDriver: Boolean = true) = { + val message = "Executor self-exiting due to : " + reason if (throwable != null) { - logError(reason, throwable) + logError(message, throwable) } else { - logError(reason) + logError(message) } + + if (notifyDriver && driver.nonEmpty) { + driver.get.ask[Boolean]( + RemoveExecutor(executorId, new ExecutorLossReason(reason)) + ).onFailure { case e => + logWarning(s"Unable to notify the driver due to " + e.getMessage, e) + }(ThreadUtils.sameThread) + } + System.exit(code) } } 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 a724fdf009789..c172ac2cdc0e3 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -199,6 +199,9 @@ private[spark] class BlockManager( logError(s"Failed to connect to external shuffle server, will retry ${MAX_ATTEMPTS - i}" + s" more times after waiting $SLEEP_TIME_SECS seconds...", e) Thread.sleep(SLEEP_TIME_SECS * 1000) + case NonFatal(e) => + throw new SparkException("Unable to register with external shuffle server due to : " + + e.getMessage, e) } } } From 1430e3b54c8c4a80d86aa579f042a9d4b32cb00a Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Thu, 15 Sep 2016 10:40:10 -0700 Subject: [PATCH 50/83] [SPARK-17379][BUILD] Upgrade netty-all to 4.0.41 final for bug fixes ## What changes were proposed in this pull request? Upgrade netty-all to latest in the 4.0.x line which is 4.0.41, mentions several bug fixes and performance improvements we may find useful, see netty.io/news/2016/08/29/4-0-41-Final-4-1-5-Final.html. Initially tried to use 4.1.5 but noticed it's not backwards compatible. ## How was this patch tested? Existing unit tests against branch-1.6 and branch-2.0 using IBM Java 8 on Intel, Power and Z architectures Author: Adam Roberts Closes #14961 from a-roberts/netty. --- .../java/org/apache/spark/network/util/TransportConf.java | 5 +++++ dev/deps/spark-deps-hadoop-2.2 | 2 +- dev/deps/spark-deps-hadoop-2.3 | 2 +- dev/deps/spark-deps-hadoop-2.4 | 2 +- dev/deps/spark-deps-hadoop-2.6 | 2 +- dev/deps/spark-deps-hadoop-2.7 | 2 +- pom.xml | 2 +- 7 files changed, 11 insertions(+), 6 deletions(-) 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 0efc400aa388c..7d5baa9a9c8f8 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 @@ -23,6 +23,11 @@ * A central location that tracks all the settings we expose to users. */ public class TransportConf { + + static { + // Set this due to Netty PR #5661 for Netty 4.0.37+ to work + System.setProperty("io.netty.maxDirectMemory", "0"); + } private final String SPARK_NETWORK_IO_MODE_KEY; private final String SPARK_NETWORK_IO_PREFERDIRECTBUFS_KEY; diff --git a/dev/deps/spark-deps-hadoop-2.2 b/dev/deps/spark-deps-hadoop-2.2 index 81adde6a13a14..a7259e25bfec6 100644 --- a/dev/deps/spark-deps-hadoop-2.2 +++ b/dev/deps/spark-deps-hadoop-2.2 @@ -124,7 +124,7 @@ metrics-json-3.1.2.jar metrics-jvm-3.1.2.jar minlog-1.3.0.jar netty-3.8.0.Final.jar -netty-all-4.0.29.Final.jar +netty-all-4.0.41.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.3 b/dev/deps/spark-deps-hadoop-2.3 index 75ab6286dec3c..6986ab572b947 100644 --- a/dev/deps/spark-deps-hadoop-2.3 +++ b/dev/deps/spark-deps-hadoop-2.3 @@ -131,7 +131,7 @@ metrics-jvm-3.1.2.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar -netty-all-4.0.29.Final.jar +netty-all-4.0.41.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.4 b/dev/deps/spark-deps-hadoop-2.4 index 897d802a9d6a1..75cccb352b9cf 100644 --- a/dev/deps/spark-deps-hadoop-2.4 +++ b/dev/deps/spark-deps-hadoop-2.4 @@ -131,7 +131,7 @@ metrics-jvm-3.1.2.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar -netty-all-4.0.29.Final.jar +netty-all-4.0.41.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index f95ddb1c3065d..ef7b8a7d8da26 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -139,7 +139,7 @@ metrics-jvm-3.1.2.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar -netty-all-4.0.29.Final.jar +netty-all-4.0.41.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 8df02c032bf21..d464c97ed1d67 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -140,7 +140,7 @@ metrics-jvm-3.1.2.jar minlog-1.3.0.jar mx4j-3.0.2.jar netty-3.8.0.Final.jar -netty-all-4.0.29.Final.jar +netty-all-4.0.41.Final.jar objenesis-2.1.jar opencsv-2.3.jar oro-2.0.8.jar diff --git a/pom.xml b/pom.xml index 3b3ad39b47571..ef83c184d0237 100644 --- a/pom.xml +++ b/pom.xml @@ -551,7 +551,7 @@ io.netty netty-all - 4.0.29.Final + 4.0.41.Final io.netty From 129e87bb97288503b616b65a152d948b213d6bd5 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 15 Sep 2016 11:22:58 -0700 Subject: [PATCH 51/83] [SPARK-17547] Ensure temp shuffle data file is cleaned up after error SPARK-8029 (#9610) modified shuffle writers to first stage their data to a temporary file in the same directory as the final destination file and then to atomically rename this temporary file at the end of the write job. However, this change introduced the potential for the temporary output file to be leaked if an exception occurs during the write because the shuffle writers' existing error cleanup code doesn't handle deletion of the temp file. This patch avoids this potential cause of disk-space leaks by adding `finally` blocks to ensure that temp files are always deleted if they haven't been renamed. Author: Josh Rosen Closes #15104 from JoshRosen/cleanup-tmp-data-file-in-shuffle-writer. --- .../sort/BypassMergeSortShuffleWriter.java | 10 ++- .../shuffle/sort/UnsafeShuffleWriter.java | 18 +++-- .../shuffle/IndexShuffleBlockResolver.scala | 80 ++++++++++--------- .../shuffle/sort/SortShuffleWriter.scala | 14 +++- 4 files changed, 73 insertions(+), 49 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 0fcc56d50ae6a..4a15559e55cbd 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 @@ -160,8 +160,14 @@ public void write(Iterator> records) throws IOException { File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); File tmp = Utils.tempFileWith(output); - partitionLengths = writePartitionedFile(tmp); - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + try { + partitionLengths = writePartitionedFile(tmp); + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } + } mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 63d376b44fb11..f235c434be7b1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -210,15 +210,21 @@ void closeAndWriteOutput() throws IOException { final File output = shuffleBlockResolver.getDataFile(shuffleId, mapId); final File tmp = Utils.tempFileWith(output); try { - partitionLengths = mergeSpills(spills, tmp); - } finally { - for (SpillInfo spill : spills) { - if (spill.file.exists() && ! spill.file.delete()) { - logger.error("Error while deleting spill file {}", spill.file.getPath()); + try { + partitionLengths = mergeSpills(spills, tmp); + } finally { + for (SpillInfo spill : spills) { + if (spill.file.exists() && ! spill.file.delete()) { + logger.error("Error while deleting spill file {}", spill.file.getPath()); + } } } + shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); + } finally { + if (tmp.exists() && !tmp.delete()) { + logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); + } } - shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, tmp); mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } diff --git a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala index 94d8c0d0fd3e4..8d6396bededa9 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockResolver.scala @@ -139,48 +139,54 @@ private[spark] class IndexShuffleBlockResolver( dataTmp: File): Unit = { val indexFile = getIndexFile(shuffleId, mapId) val indexTmp = Utils.tempFileWith(indexFile) - val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) - Utils.tryWithSafeFinally { - // We take in lengths of each block, need to convert it to offsets. - var offset = 0L - out.writeLong(offset) - for (length <- lengths) { - offset += length + try { + val out = new DataOutputStream(new BufferedOutputStream(new FileOutputStream(indexTmp))) + Utils.tryWithSafeFinally { + // We take in lengths of each block, need to convert it to offsets. + var offset = 0L out.writeLong(offset) + for (length <- lengths) { + offset += length + out.writeLong(offset) + } + } { + out.close() } - } { - out.close() - } - val dataFile = getDataFile(shuffleId, mapId) - // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure - // the following check and rename are atomic. - synchronized { - val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) - if (existingLengths != null) { - // Another attempt for the same task has already written our map outputs successfully, - // so just use the existing partition lengths and delete our temporary map outputs. - System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) - if (dataTmp != null && dataTmp.exists()) { - dataTmp.delete() - } - indexTmp.delete() - } else { - // This is the first successful attempt in writing the map outputs for this task, - // so override any existing index and data files with the ones we wrote. - if (indexFile.exists()) { - indexFile.delete() - } - if (dataFile.exists()) { - dataFile.delete() - } - if (!indexTmp.renameTo(indexFile)) { - throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) - } - if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { - throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) + val dataFile = getDataFile(shuffleId, mapId) + // There is only one IndexShuffleBlockResolver per executor, this synchronization make sure + // the following check and rename are atomic. + synchronized { + val existingLengths = checkIndexAndDataFile(indexFile, dataFile, lengths.length) + if (existingLengths != null) { + // Another attempt for the same task has already written our map outputs successfully, + // so just use the existing partition lengths and delete our temporary map outputs. + System.arraycopy(existingLengths, 0, lengths, 0, lengths.length) + if (dataTmp != null && dataTmp.exists()) { + dataTmp.delete() + } + indexTmp.delete() + } else { + // This is the first successful attempt in writing the map outputs for this task, + // so override any existing index and data files with the ones we wrote. + if (indexFile.exists()) { + indexFile.delete() + } + if (dataFile.exists()) { + dataFile.delete() + } + if (!indexTmp.renameTo(indexFile)) { + throw new IOException("fail to rename file " + indexTmp + " to " + indexFile) + } + if (dataTmp != null && dataTmp.exists() && !dataTmp.renameTo(dataFile)) { + throw new IOException("fail to rename file " + dataTmp + " to " + dataFile) + } } } + } finally { + if (indexTmp.exists() && !indexTmp.delete()) { + logError(s"Failed to delete temporary index file at ${indexTmp.getAbsolutePath}") + } } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index cc01e6aa7ea91..636b88e792bf3 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -67,10 +67,16 @@ private[spark] class SortShuffleWriter[K, V, C]( // (see SPARK-3570). val output = shuffleBlockResolver.getDataFile(dep.shuffleId, mapId) val tmp = Utils.tempFileWith(output) - val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) - val partitionLengths = sorter.writePartitionedFile(blockId, tmp) - shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + try { + val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) + val partitionLengths = sorter.writePartitionedFile(blockId, tmp) + shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) + } finally { + if (tmp.exists() && !tmp.delete()) { + logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") + } + } } /** Close this writer, passing along whether the map completed */ From be726e2cf66ff327d675f1c662b574b9625cb5df Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Thu, 15 Sep 2016 20:24:15 +0200 Subject: [PATCH 52/83] [SPARK-17114][SQL] Fix aggregates grouped by literals with empty input ## What changes were proposed in this pull request? This PR fixes an issue with aggregates that have an empty input, and use a literals as their grouping keys. These aggregates are currently interpreted as aggregates **without** grouping keys, this triggers the ungrouped code path (which aways returns a single row). This PR fixes the `RemoveLiteralFromGroupExpressions` optimizer rule, which changes the semantics of the Aggregate by eliminating all literal grouping keys. ## How was this patch tested? Added tests to `SQLQueryTestSuite`. Author: Herman van Hovell Closes #15101 from hvanhovell/SPARK-17114-3. --- .../sql/catalyst/optimizer/Optimizer.scala | 11 +++- .../optimizer/AggregateOptimizeSuite.scala | 10 +++- .../resources/sql-tests/inputs/group-by.sql | 17 +++++++ .../sql-tests/results/group-by.sql.out | 51 +++++++++++++++++++ 4 files changed, 86 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/group-by.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/group-by.sql.out 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 d2f0c97989213..0df16b7a56c56 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 @@ -1098,9 +1098,16 @@ object ReplaceExceptWithAntiJoin extends Rule[LogicalPlan] { */ object RemoveLiteralFromGroupExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case a @ Aggregate(grouping, _, _) => + case a @ Aggregate(grouping, _, _) if grouping.nonEmpty => val newGrouping = grouping.filter(!_.foldable) - a.copy(groupingExpressions = newGrouping) + if (newGrouping.nonEmpty) { + a.copy(groupingExpressions = newGrouping) + } else { + // All grouping expressions are literals. We should not drop them all, because this can + // change the return semantics when the input of the Aggregate is empty (SPARK-17114). We + // instead replace this by single, easy to hash/sort, literal expression. + a.copy(groupingExpressions = Seq(Literal(0, IntegerType))) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala index 4c26c184b7b5b..aecf59aee6a9b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor class AggregateOptimizeSuite extends PlanTest { - val conf = new SimpleCatalystConf(caseSensitiveAnalysis = false) + val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, groupByOrdinal = false) val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf) val analyzer = new Analyzer(catalog, conf) @@ -49,6 +49,14 @@ class AggregateOptimizeSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("do not remove all grouping expressions if they are all literals") { + val query = testRelation.groupBy(Literal("1"), Literal(1) + Literal(2))(sum('b)) + val optimized = Optimize.execute(analyzer.execute(query)) + val correctAnswer = analyzer.execute(testRelation.groupBy(Literal(0))(sum('b))) + + comparePlans(optimized, correctAnswer) + } + test("Remove aliased literals") { val query = testRelation.select('a, Literal(1).as('y)).groupBy('a, 'y)(sum('b)) val optimized = Optimize.execute(analyzer.execute(query)) 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 new file mode 100644 index 0000000000000..6741703d9d82c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -0,0 +1,17 @@ +-- Temporary data. +create temporary view myview as values 128, 256 as v(int_col); + +-- group by should produce all input rows, +select int_col, count(*) from myview group by int_col; + +-- group by should produce a single row. +select 'foo', count(*) from myview group by 1; + +-- group-by should not produce any rows (whole stage code generation). +select 'foo' from myview where int_col == 0 group by 1; + +-- group-by should not produce any rows (hash aggregate). +select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1; + +-- group-by should not produce any rows (sort aggregate). +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1; 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 new file mode 100644 index 0000000000000..9127bd4dd4c6f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -0,0 +1,51 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view myview as values 128, 256 as v(int_col) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +select int_col, count(*) from myview group by int_col +-- !query 1 schema +struct +-- !query 1 output +128 1 +256 1 + + +-- !query 2 +select 'foo', count(*) from myview group by 1 +-- !query 2 schema +struct +-- !query 2 output +foo 2 + + +-- !query 3 +select 'foo' from myview where int_col == 0 group by 1 +-- !query 3 schema +struct +-- !query 3 output + + + +-- !query 4 +select 'foo', approx_count_distinct(int_col) from myview where int_col == 0 group by 1 +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +select 'foo', max(struct(int_col)) from myview where int_col == 0 group by 1 +-- !query 5 schema +struct> +-- !query 5 output + From 17f7ce3b21d26c6d17f42b8433e4d1f9bf4f1a8a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=B2=91=E7=8E=89=E6=B5=B7?= <261810726@qq.com> Date: Thu, 15 Sep 2016 20:45:00 +0200 Subject: [PATCH 53/83] [SPARK-17429][SQL] use ImplicitCastInputTypes with function Length MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? select length(11); select length(2.0); these sql will return errors, but hive is ok. this PR will support casting input types implicitly for function length the correct result is: select length(11) return 2 select length(2.0) return 3 Author: 岑玉海 <261810726@qq.com> Author: cenyuhai Closes #15014 from cenyuhai/SPARK-17429. --- .../sql/catalyst/expressions/stringExpressions.scala | 2 +- .../org/apache/spark/sql/StringFunctionsSuite.scala | 10 ++++++---- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index a8c23a8b0c536..1bcbb6cfc9246 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -1057,7 +1057,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) @ExpressionDescription( usage = "_FUNC_(str | binary) - Returns the length of str or number of bytes in binary data.", extended = "> SELECT _FUNC_('Spark SQL');\n 9") -case class Length(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Length(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, BinaryType)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index 1cc77464b93fc..bcc2351049953 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -330,7 +330,8 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { } test("string / binary length function") { - val df = Seq(("123", Array[Byte](1, 2, 3, 4), 123)).toDF("a", "b", "c") + val df = Seq(("123", Array[Byte](1, 2, 3, 4), 123, 2.0f, 3.015)) + .toDF("a", "b", "c", "d", "e") checkAnswer( df.select(length($"a"), length($"b")), Row(3, 4)) @@ -339,9 +340,10 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("length(a)", "length(b)"), Row(3, 4)) - intercept[AnalysisException] { - df.selectExpr("length(c)") // int type of the argument is unacceptable - } + checkAnswer( + df.selectExpr("length(c)", "length(d)", "length(e)"), + Row(3, 3, 5) + ) } test("initcap function") { From 609e6ce0b6637687ed09bf88690182e05609db8f Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Thu, 15 Sep 2016 20:53:48 +0200 Subject: [PATCH 54/83] [SPARK-17364][SQL] Antlr lexer wrongly treats full qualified identifier as a decimal number token when parsing SQL string ## What changes were proposed in this pull request? The Antlr lexer we use to tokenize a SQL string may wrongly tokenize a fully qualified identifier as a decimal number token. For example, table identifier `default.123_table` is wrongly tokenized as ``` default // Matches lexer rule IDENTIFIER .123 // Matches lexer rule DECIMAL_VALUE _TABLE // Matches lexer rule IDENTIFIER ``` The correct tokenization for `default.123_table` should be: ``` default // Matches lexer rule IDENTIFIER, . // Matches a single dot 123_TABLE // Matches lexer rule IDENTIFIER ``` This PR fix the Antlr grammar so that it can tokenize fully qualified identifier correctly: 1. Fully qualified table name can be parsed correctly. For example, `select * from database.123_suffix`. 2. Fully qualified column name can be parsed correctly, for example `select a.123_suffix from a`. ### Before change #### Case 1: Failed to parse fully qualified column name ``` scala> spark.sql("select a.123_column from a").show org.apache.spark.sql.catalyst.parser.ParseException: extraneous input '.123' expecting {, ... , IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 8) == SQL == select a.123_column from a --------^^^ ``` #### Case 2: Failed to parse fully qualified table name ``` scala> spark.sql("select * from default.123_table") org.apache.spark.sql.catalyst.parser.ParseException: extraneous input '.123' expecting {, ... IDENTIFIER, BACKQUOTED_IDENTIFIER}(line 1, pos 21) == SQL == select * from default.123_table ---------------------^^^ ``` ### After Change #### Case 1: fully qualified column name, no ParseException thrown ``` scala> spark.sql("select a.123_column from a").show ``` #### Case 2: fully qualified table name, no ParseException thrown ``` scala> spark.sql("select * from default.123_table") ``` ## How was this patch tested? Unit test. Author: Sean Zhong Closes #15006 from clockfly/SPARK-17364. --- .../spark/sql/catalyst/parser/SqlBase.g4 | 44 +++++++++++++++---- .../parser/ExpressionParserSuite.scala | 15 ++++++- .../parser/TableIdentifierParserSuite.scala | 13 ++++++ 3 files changed, 63 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index b475abdce2da9..7023c0c8c493f 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -16,6 +16,30 @@ grammar SqlBase; +@members { + /** + * Verify whether current token is a valid decimal token (which contains dot). + * Returns true if the character that follows the token is not a digit or letter or underscore. + * + * For example: + * For char stream "2.3", "2." is not a valid decimal token, because it is followed by digit '3'. + * For char stream "2.3_", "2.3" is not a valid decimal token, because it is followed by '_'. + * For char stream "2.3W", "2.3" is not a valid decimal token, because it is followed by 'W'. + * For char stream "12.0D 34.E2+0.12 " 12.0D is a valid decimal token because it is folllowed + * by a space. 34.E2 is a valid decimal token because it is followed by symbol '+' + * which is not a digit or letter or underscore. + */ + public boolean isValidDecimal() { + int nextChar = _input.LA(1); + if (nextChar >= 'A' && nextChar <= 'Z' || nextChar >= '0' && nextChar <= '9' || + nextChar == '_') { + return false; + } else { + return true; + } + } +} + tokens { DELIMITER } @@ -920,23 +944,22 @@ INTEGER_VALUE ; DECIMAL_VALUE - : DIGIT+ '.' DIGIT* - | '.' DIGIT+ + : DECIMAL_DIGITS {isValidDecimal()}? ; SCIENTIFIC_DECIMAL_VALUE - : DIGIT+ ('.' DIGIT*)? EXPONENT - | '.' DIGIT+ EXPONENT + : DIGIT+ EXPONENT + | DECIMAL_DIGITS EXPONENT {isValidDecimal()}? ; DOUBLE_LITERAL - : - (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'D' + : DIGIT+ EXPONENT? 'D' + | DECIMAL_DIGITS EXPONENT? 'D' {isValidDecimal()}? ; BIGDECIMAL_LITERAL - : - (INTEGER_VALUE | DECIMAL_VALUE | SCIENTIFIC_DECIMAL_VALUE) 'BD' + : DIGIT+ EXPONENT? 'BD' + | DECIMAL_DIGITS EXPONENT? 'BD' {isValidDecimal()}? ; IDENTIFIER @@ -947,6 +970,11 @@ BACKQUOTED_IDENTIFIER : '`' ( ~'`' | '``' )* '`' ; +fragment DECIMAL_DIGITS + : DIGIT+ '.' DIGIT* + | '.' DIGIT+ + ; + fragment EXPONENT : 'E' [+-]? DIGIT+ ; 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 4e399eef1fed8..f319215f05681 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} -import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.PlanTest @@ -518,4 +518,17 @@ class ExpressionParserSuite extends PlanTest { assertEqual("current_date", CurrentDate()) assertEqual("current_timestamp", CurrentTimestamp()) } + + test("SPARK-17364, fully qualified column name which starts with number") { + assertEqual("123_", UnresolvedAttribute("123_")) + assertEqual("1a.123_", UnresolvedAttribute("1a.123_")) + // ".123" should not be treated as token of type DECIMAL_VALUE + assertEqual("a.123A", UnresolvedAttribute("a.123A")) + // ".123E3" should not be treated as token of type SCIENTIFIC_DECIMAL_VALUE + assertEqual("a.123E3_column", UnresolvedAttribute("a.123E3_column")) + // ".123D" should not be treated as token of type DOUBLE_LITERAL + assertEqual("a.123D_column", UnresolvedAttribute("a.123D_column")) + // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL + assertEqual("a.123BD_column", UnresolvedAttribute("a.123BD_column")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index dadb8a8def43b..793be8953d07a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -91,4 +91,17 @@ class TableIdentifierParserSuite extends SparkFunSuite { assert(TableIdentifier(nonReserved) === parseTableIdentifier(nonReserved)) } } + + test("SPARK-17364 table identifier - contains number") { + assert(parseTableIdentifier("123_") == TableIdentifier("123_")) + assert(parseTableIdentifier("1a.123_") == TableIdentifier("123_", Some("1a"))) + // ".123" should not be treated as token of type DECIMAL_VALUE + assert(parseTableIdentifier("a.123A") == TableIdentifier("123A", Some("a"))) + // ".123E3" should not be treated as token of type SCIENTIFIC_DECIMAL_VALUE + assert(parseTableIdentifier("a.123E3_LIST") == TableIdentifier("123E3_LIST", Some("a"))) + // ".123D" should not be treated as token of type DOUBLE_LITERAL + assert(parseTableIdentifier("a.123D_LIST") == TableIdentifier("123D_LIST", Some("a"))) + // ".123BD" should not be treated as token of type BIGDECIMAL_LITERAL + assert(parseTableIdentifier("a.123BD_LIST") == TableIdentifier("123BD_LIST", Some("a"))) + } } From 2dc869f384612317bb252ec60cd4b65dbe7d8f89 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Thu, 15 Sep 2016 11:54:17 -0700 Subject: [PATCH 55/83] [SPARK-17484] Prevent invalid block locations from being reported after put() exceptions ## What changes were proposed in this pull request? If a BlockManager `put()` call failed after the BlockManagerMaster was notified of a block's availability then incomplete cleanup logic in a `finally` block would never send a second block status method to inform the master of the block's unavailability. This, in turn, leads to fetch failures and used to be capable of causing complete job failures before #15037 was fixed. This patch addresses this issue via multiple small changes: - The `finally` block now calls `removeBlockInternal` when cleaning up from a failed `put()`; in addition to removing the `BlockInfo` entry (which was _all_ that the old cleanup logic did), this code (redundantly) tries to remove the block from the memory and disk stores (as an added layer of defense against bugs lower down in the stack) and optionally notifies the master of block removal (which now happens during exception-triggered cleanup). - When a BlockManager receives a request for a block that it does not have it will now notify the master to update its block locations. This ensures that bad metadata pointing to non-existent blocks will eventually be fixed. Note that I could have implemented this logic in the block manager client (rather than in the remote server), but that would introduce the problem of distinguishing between transient and permanent failures; on the server, however, we know definitively that the block isn't present. - Catch `NonFatal` instead of `Exception` to avoid swallowing `InterruptedException`s thrown from synchronous block replication calls. This patch depends upon the refactorings in #15036, so that other patch will also have to be backported when backporting this fix. For more background on this issue, including example logs from a real production failure, see [SPARK-17484](https://issues.apache.org/jira/browse/SPARK-17484). ## How was this patch tested? Two new regression tests in BlockManagerSuite. Author: Josh Rosen Closes #15085 from JoshRosen/SPARK-17484. --- .../apache/spark/storage/BlockManager.scala | 37 +++++++++++++++---- .../spark/storage/BlockManagerSuite.scala | 34 +++++++++++++++++ 2 files changed, 63 insertions(+), 8 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 c172ac2cdc0e3..aa29acfd70461 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -283,7 +283,12 @@ private[spark] class BlockManager( } else { getLocalBytes(blockId) match { case Some(buffer) => new BlockManagerManagedBuffer(blockInfoManager, blockId, buffer) - case None => throw new BlockNotFoundException(blockId.toString) + case None => + // If this block manager receives a request for a block that it doesn't have then it's + // likely that the master has outdated block statuses for this block. Therefore, we send + // an RPC so that this block is marked as being unavailable from this block manager. + reportBlockStatus(blockId, BlockStatus.empty) + throw new BlockNotFoundException(blockId.toString) } } } @@ -859,22 +864,38 @@ private[spark] class BlockManager( } val startTimeMs = System.currentTimeMillis - var blockWasSuccessfullyStored: Boolean = false + var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) - blockWasSuccessfullyStored = res.isEmpty - res - } finally { - if (blockWasSuccessfullyStored) { + exceptionWasThrown = false + if (res.isEmpty) { + // the block was successfully stored if (keepReadLock) { blockInfoManager.downgradeLock(blockId) } else { blockInfoManager.unlock(blockId) } } else { - blockInfoManager.removeBlock(blockId) + removeBlockInternal(blockId, tellMaster = false) logWarning(s"Putting block $blockId failed") } + res + } finally { + // This cleanup is performed in a finally block rather than a `catch` to avoid having to + // catch and properly re-throw InterruptedException. + if (exceptionWasThrown) { + logWarning(s"Putting block $blockId failed due to an exception") + // If an exception was thrown then it's possible that the code in `putBody` has already + // notified the master about the availability of this block, so we need to send an update + // to remove this block location. + removeBlockInternal(blockId, tellMaster = tellMaster) + // The `putBody` code may have also added a new block status to TaskMetrics, so we need + // to cancel that out by overwriting it with an empty block status. We only do this if + // the finally block was entered via an exception because doing this unconditionally would + // cause us to send empty block statuses for every block that failed to be cached due to + // a memory shortage (which is an expected failure, unlike an uncaught exception). + addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) + } } if (level.replication > 1) { logDebug("Putting block %s with replication took %s" @@ -1173,7 +1194,7 @@ private[spark] class BlockManager( done = true // specified number of peers have been replicated to } } catch { - case e: Exception => + case NonFatal(e) => logWarning(s"Failed to replicate $blockId to $peer, failure #$failures", e) failures += 1 replicationFailed = true 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 fdf28b7dcbcf4..6d53d2e5f0ca6 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -861,6 +861,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memoryManager.setMemoryStore(store.memoryStore) + store.initialize("app-id") // The put should fail since a1 is not serializable. class UnserializableClass @@ -1206,6 +1207,39 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE verify(mockBlockManagerMaster, times(2)).getLocations("item") } + test("SPARK-17484: block status is properly updated following an exception in put()") { + val mockBlockTransferService = new MockBlockTransferService(maxFailures = 10) { + override def uploadBlock( + hostname: String, + port: Int, execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + throw new InterruptedException("Intentional interrupt") + } + } + store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + intercept[InterruptedException] { + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) + } + assert(store.getLocalBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + } + + test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { + store = makeBlockManager(8000, "executor1") + store2 = makeBlockManager(8000, "executor2") + store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(master.getLocations("item").nonEmpty) + store.removeBlock("item", tellMaster = false) + assert(master.getLocations("item").nonEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + assert(master.getLocations("item").isEmpty) + } + class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 From 2bb1a196ef6e57aeb360d5dd2245c64298d0cfae Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Thu, 15 Sep 2016 21:45:29 +0200 Subject: [PATCH 56/83] [SPARK-17458][SQL] Alias specified for aggregates in a pivot are not honored ## What changes were proposed in this pull request? This change preserves aliases that are given for pivot aggregations ## How was this patch tested? New unit test Author: Andrew Ray Closes #15111 from aray/SPARK-17458. --- .../apache/spark/sql/catalyst/analysis/Analyzer.scala | 10 +++++++++- .../org/apache/spark/sql/DataFramePivotSuite.scala | 11 +++++++++++ 2 files changed, 20 insertions(+), 1 deletion(-) 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 92bf8e0536fc4..5210f42c557b6 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 @@ -373,7 +373,15 @@ class Analyzer( case Pivot(groupByExprs, pivotColumn, pivotValues, aggregates, child) => val singleAgg = aggregates.size == 1 def outputName(value: Literal, aggregate: Expression): String = { - if (singleAgg) value.toString else value + "_" + aggregate.sql + if (singleAgg) { + value.toString + } else { + val suffix = aggregate match { + case n: NamedExpression => n.name + case _ => aggregate.sql + } + value + "_" + suffix + } } if (aggregates.forall(a => PivotFirst.supportsDataType(a.dataType))) { // Since evaluating |pivotValues| if statements for each input row can get slow this is an 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 d5cb5e15688e8..1bbe1354d55f4 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 @@ -197,4 +197,15 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext{ Row(2013, Seq(48000.0, 7.0), Seq(30000.0, 7.0)) :: Nil ) } + + test("pivot preserves aliases if given") { + assertResult( + Array("year", "dotNET_foo", "dotNET_avg(`earnings`)", "Java_foo", "Java_avg(`earnings`)") + )( + courseSales.groupBy($"year") + .pivot("course", Seq("dotNET", "Java")) + .agg(sum($"earnings").as("foo"), avg($"earnings")).columns + ) + } + } From 8098b2962fa32ddd9fd8a1eb1afb4eaae4a7fa41 Mon Sep 17 00:00:00 2001 From: Jagadeesan Date: Fri, 16 Sep 2016 10:18:45 +0100 Subject: [PATCH 57/83] =?UTF-8?q?[SPARK-17543]=20Missing=20log4j=20config?= =?UTF-8?q?=20file=20for=20tests=20in=20common/network-=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? The Maven module `common/network-shuffle` does not have a log4j configuration file for its test cases. So, added `log4j.properties` in the directory `src/test/resources`. …shuffle] Author: Jagadeesan Closes #15108 from jagadeesanas2/SPARK-17543. --- .../src/test/resources/log4j.properties | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) create mode 100644 common/network-shuffle/src/test/resources/log4j.properties diff --git a/common/network-shuffle/src/test/resources/log4j.properties b/common/network-shuffle/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..e73978908b683 --- /dev/null +++ b/common/network-shuffle/src/test/resources/log4j.properties @@ -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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=DEBUG, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n From fdb9154008acc3d78d6bbaac9bd976a2b3404259 Mon Sep 17 00:00:00 2001 From: Adam Roberts Date: Fri, 16 Sep 2016 10:20:50 +0100 Subject: [PATCH 58/83] [SPARK-17534][TESTS] Increase timeouts for DirectKafkaStreamSuite tests **## What changes were proposed in this pull request?** There are two tests in this suite that are particularly flaky on the following hardware: 2x Intel(R) Xeon(R) CPU E5-2697 v2 2.70GHz and 16 GB of RAM, 1 TB HDD This simple PR increases the timeout times and batch duration so they can reliably pass **## How was this patch tested?** Existing unit tests with the two core box where I was seeing the failures often Author: Adam Roberts Closes #15094 from a-roberts/patch-6. --- .../spark/streaming/kafka010/DirectKafkaStreamSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index b1d90b8a82d59..e04f35eceb1b4 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -108,7 +108,7 @@ class DirectKafkaStreamSuite val expectedTotal = (data.values.sum * topics.size) - 2 val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - ssc = new StreamingContext(sparkConf, Milliseconds(200)) + ssc = new StreamingContext(sparkConf, Milliseconds(1000)) val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( ssc, @@ -150,7 +150,7 @@ class DirectKafkaStreamSuite allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) } ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { assert(allReceived.size === expectedTotal, "didn't get expected number of messages, messages:\n" + allReceived.asScala.mkString("\n")) @@ -172,7 +172,7 @@ class DirectKafkaStreamSuite val expectedTotal = (data.values.sum * 2) - 3 val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - ssc = new StreamingContext(sparkConf, Milliseconds(200)) + ssc = new StreamingContext(sparkConf, Milliseconds(1000)) val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( ssc, @@ -214,7 +214,7 @@ class DirectKafkaStreamSuite allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) } ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { assert(allReceived.size === expectedTotal, "didn't get expected number of messages, messages:\n" + allReceived.asScala.mkString("\n")) From 0ef4313c2b042d4983dc6540fdb85a204637a4eb Mon Sep 17 00:00:00 2001 From: Sean Zhong Date: Fri, 16 Sep 2016 19:37:30 +0800 Subject: [PATCH 59/83] [SPARK-17426][SQL] Refactor `TreeNode.toJSON` to avoid OOM when converting unknown fields to JSON ## What changes were proposed in this pull request? This PR is a follow up of SPARK-17356. Current implementation of `TreeNode.toJSON` recursively converts all fields of TreeNode to JSON, even if the field is of type `Seq` or type Map. This may trigger out of memory exception in cases like: 1. the Seq or Map can be very big. Converting them to JSON may take huge memory, which may trigger out of memory error. 2. Some user space input may also be propagated to the Plan. The user space input can be of arbitrary type, and may also be self-referencing. Trying to print user space input to JSON may trigger out of memory error or stack overflow error. For a code example, please check the Jira description of SPARK-17426. In this PR, we refactor the `TreeNode.toJSON` so that we only convert a field to JSON string if the field is a safe type. ## How was this patch tested? Unit test. Author: Sean Zhong Closes #14990 from clockfly/json_oom2. --- .../spark/sql/catalyst/trees/TreeNode.scala | 218 +++---------- .../sql/catalyst/trees/TreeNodeSuite.scala | 294 +++++++++++++++++- .../org/apache/spark/sql/QueryTest.scala | 136 -------- 3 files changed, 333 insertions(+), 315 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 893af5146c5b3..83cb375525832 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 @@ -30,10 +30,15 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkContext import org.apache.spark.rdd.{EmptyRDD, RDD} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource} +import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.ScalaReflection._ import org.apache.spark.sql.catalyst.ScalaReflectionLock +import org.apache.spark.sql.catalyst.TableIdentifier 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.types._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -597,7 +602,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { // this child in all children. case (name, value: TreeNode[_]) if containsChild(value) => name -> JInt(children.indexOf(value)) - case (name, value: Seq[BaseType]) if value.toSet.subsetOf(containsChild) => + case (name, value: Seq[BaseType]) if value.forall(containsChild) => name -> JArray( value.map(v => JInt(children.indexOf(v.asInstanceOf[TreeNode[_]]))).toList ) @@ -621,194 +626,53 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { // SPARK-17356: In usage of mllib, Metadata may store a huge vector of data, transforming // it to JSON may trigger OutOfMemoryError. case m: Metadata => Metadata.empty.jsonValue + case clazz: Class[_] => JString(clazz.getName) case s: StorageLevel => ("useDisk" -> s.useDisk) ~ ("useMemory" -> s.useMemory) ~ ("useOffHeap" -> s.useOffHeap) ~ ("deserialized" -> s.deserialized) ~ ("replication" -> s.replication) case n: TreeNode[_] => n.jsonValue case o: Option[_] => o.map(parseToJson) - case t: Seq[_] => JArray(t.map(parseToJson).toList) - case m: Map[_, _] => - val fields = m.toList.map { case (k: String, v) => (k, parseToJson(v)) } - JObject(fields) - case r: RDD[_] => JNothing + // Recursive scan Seq[TreeNode], Seq[Partitioning], Seq[DataType] + case t: Seq[_] if t.forall(_.isInstanceOf[TreeNode[_]]) || + 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(Utils.truncatedString(t, "[", ", ", "]")) + case t: Seq[_] => JNull + case m: Map[_, _] => JNull // if it's a scala object, we can simply keep the full class path. // TODO: currently if the class name ends with "$", we think it's a scala object, there is // probably a better way to check it. case obj if obj.getClass.getName.endsWith("$") => "object" -> obj.getClass.getName - // returns null if the product type doesn't have a primary constructor, e.g. HiveFunctionWrapper - case p: Product => try { - val fieldNames = getConstructorParameterNames(p.getClass) - val fieldValues = p.productIterator.toSeq - assert(fieldNames.length == fieldValues.length) - ("product-class" -> JString(p.getClass.getName)) :: fieldNames.zip(fieldValues).map { - case (name, value) => name -> parseToJson(value) - }.toList - } catch { - case _: RuntimeException => null - } - case _ => JNull - } -} - -object TreeNode { - def fromJSON[BaseType <: TreeNode[BaseType]](json: String, sc: SparkContext): BaseType = { - val jsonAST = parse(json) - assert(jsonAST.isInstanceOf[JArray]) - reconstruct(jsonAST.asInstanceOf[JArray], sc).asInstanceOf[BaseType] - } - - private def reconstruct(treeNodeJson: JArray, sc: SparkContext): TreeNode[_] = { - assert(treeNodeJson.arr.forall(_.isInstanceOf[JObject])) - val jsonNodes = Stack(treeNodeJson.arr.map(_.asInstanceOf[JObject]): _*) - - def parseNextNode(): TreeNode[_] = { - val nextNode = jsonNodes.pop() - - val cls = Utils.classForName((nextNode \ "class").asInstanceOf[JString].s) - if (cls == classOf[Literal]) { - Literal.fromJSON(nextNode) - } else if (cls.getName.endsWith("$")) { - cls.getField("MODULE$").get(cls).asInstanceOf[TreeNode[_]] - } else { - val numChildren = (nextNode \ "num-children").asInstanceOf[JInt].num.toInt - - val children: Seq[TreeNode[_]] = (1 to numChildren).map(_ => parseNextNode()) - val fields = getConstructorParameters(cls) - - val parameters: Array[AnyRef] = fields.map { - case (fieldName, fieldType) => - parseFromJson(nextNode \ fieldName, fieldType, children, sc) - }.toArray - - val maybeCtor = cls.getConstructors.find { p => - val expectedTypes = p.getParameterTypes - expectedTypes.length == fields.length && expectedTypes.zip(fields.map(_._2)).forall { - case (cls, tpe) => cls == getClassFromType(tpe) - } - } - if (maybeCtor.isEmpty) { - sys.error(s"No valid constructor for ${cls.getName}") - } else { - try { - maybeCtor.get.newInstance(parameters: _*).asInstanceOf[TreeNode[_]] - } catch { - case e: java.lang.IllegalArgumentException => - throw new RuntimeException( - s""" - |Failed to construct tree node: ${cls.getName} - |ctor: ${maybeCtor.get} - |types: ${parameters.map(_.getClass).mkString(", ")} - |args: ${parameters.mkString(", ")} - """.stripMargin, e) - } - } - } - } - - parseNextNode() - } - - import universe._ - - private def parseFromJson( - value: JValue, - expectedType: Type, - children: Seq[TreeNode[_]], - sc: SparkContext): AnyRef = ScalaReflectionLock.synchronized { - if (value == JNull) return null - - expectedType match { - case t if t <:< definitions.BooleanTpe => - value.asInstanceOf[JBool].value: java.lang.Boolean - case t if t <:< definitions.ByteTpe => - value.asInstanceOf[JInt].num.toByte: java.lang.Byte - case t if t <:< definitions.ShortTpe => - value.asInstanceOf[JInt].num.toShort: java.lang.Short - case t if t <:< definitions.IntTpe => - value.asInstanceOf[JInt].num.toInt: java.lang.Integer - case t if t <:< definitions.LongTpe => - value.asInstanceOf[JInt].num.toLong: java.lang.Long - case t if t <:< definitions.FloatTpe => - value.asInstanceOf[JDouble].num.toFloat: java.lang.Float - case t if t <:< definitions.DoubleTpe => - value.asInstanceOf[JDouble].num: java.lang.Double - - case t if t <:< localTypeOf[java.lang.Boolean] => - value.asInstanceOf[JBool].value: java.lang.Boolean - case t if t <:< localTypeOf[BigInt] => value.asInstanceOf[JInt].num - case t if t <:< localTypeOf[java.lang.String] => value.asInstanceOf[JString].s - case t if t <:< localTypeOf[UUID] => UUID.fromString(value.asInstanceOf[JString].s) - case t if t <:< localTypeOf[DataType] => DataType.parseDataType(value) - case t if t <:< localTypeOf[Metadata] => Metadata.fromJObject(value.asInstanceOf[JObject]) - case t if t <:< localTypeOf[StorageLevel] => - val JBool(useDisk) = value \ "useDisk" - val JBool(useMemory) = value \ "useMemory" - val JBool(useOffHeap) = value \ "useOffHeap" - val JBool(deserialized) = value \ "deserialized" - val JInt(replication) = value \ "replication" - StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication.toInt) - case t if t <:< localTypeOf[TreeNode[_]] => value match { - case JInt(i) => children(i.toInt) - case arr: JArray => reconstruct(arr, sc) - case _ => throw new RuntimeException(s"$value is not a valid json value for tree node.") + case p: Product if shouldConvertToJson(p) => + try { + val fieldNames = getConstructorParameterNames(p.getClass) + val fieldValues = p.productIterator.toSeq + assert(fieldNames.length == fieldValues.length) + ("product-class" -> JString(p.getClass.getName)) :: fieldNames.zip(fieldValues).map { + case (name, value) => name -> parseToJson(value) + }.toList + } catch { + case _: RuntimeException => null } - case t if t <:< localTypeOf[Option[_]] => - if (value == JNothing) { - None - } else { - val TypeRef(_, _, Seq(optType)) = t - Option(parseFromJson(value, optType, children, sc)) - } - case t if t <:< localTypeOf[Seq[_]] => - val TypeRef(_, _, Seq(elementType)) = t - val JArray(elements) = value - elements.map(parseFromJson(_, elementType, children, sc)).toSeq - case t if t <:< localTypeOf[Map[_, _]] => - val TypeRef(_, _, Seq(keyType, valueType)) = t - val JObject(fields) = value - fields.map { - case (name, value) => name -> parseFromJson(value, valueType, children, sc) - }.toMap - case t if t <:< localTypeOf[RDD[_]] => - new EmptyRDD[Any](sc) - case _ if isScalaObject(value) => - val JString(clsName) = value \ "object" - val cls = Utils.classForName(clsName) - cls.getField("MODULE$").get(cls) - case t if t <:< localTypeOf[Product] => - val fields = getConstructorParameters(t) - val clsName = getClassNameFromType(t) - parseToProduct(clsName, fields, value, children, sc) - // There maybe some cases that the parameter type signature is not Product but the value is, - // e.g. `SpecifiedWindowFrame` with type signature `WindowFrame`, handle it here. - case _ if isScalaProduct(value) => - val JString(clsName) = value \ "product-class" - val fields = getConstructorParameters(Utils.classForName(clsName)) - parseToProduct(clsName, fields, value, children, sc) - case _ => sys.error(s"Do not support type $expectedType with json $value.") - } - } - - private def parseToProduct( - clsName: String, - fields: Seq[(String, Type)], - value: JValue, - children: Seq[TreeNode[_]], - sc: SparkContext): AnyRef = { - val parameters: Array[AnyRef] = fields.map { - case (fieldName, fieldType) => parseFromJson(value \ fieldName, fieldType, children, sc) - }.toArray - val ctor = Utils.classForName(clsName).getConstructors.maxBy(_.getParameterTypes.size) - ctor.newInstance(parameters: _*).asInstanceOf[AnyRef] - } - - private def isScalaObject(jValue: JValue): Boolean = (jValue \ "object") match { - case JString(str) if str.endsWith("$") => true - case _ => false + case _ => JNull } - private def isScalaProduct(jValue: JValue): Boolean = (jValue \ "product-class") match { - case _: JString => true + private def shouldConvertToJson(product: Product): Boolean = product match { + case exprId: ExprId => true + case field: StructField => true + case id: TableIdentifier => true + case join: JoinType => true + case id: FunctionIdentifier => true + case spec: BucketSpec => true + case catalog: CatalogTable => true + case boundary: FrameBoundary => true + case frame: WindowFrame => true + case partition: Partitioning => true + case resource: FunctionResource => true + case broadcast: BroadcastMode => true + case table: CatalogTableType => true + case storage: CatalogStorageFormat => true case _ => false } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 6246380dbeb9b..cb0426c7a98a1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -17,13 +17,29 @@ package org.apache.spark.sql.catalyst.trees +import java.math.BigInteger +import java.util.UUID + import scala.collection.mutable.ArrayBuffer +import org.json4s.jackson.JsonMethods +import org.json4s.jackson.JsonMethods._ +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType, FunctionResource, JarResource} +import org.apache.spark.sql.catalyst.dsl.expressions.DslString import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback -import org.apache.spark.sql.types.{IntegerType, NullType, StringType} +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.plans.{LeftOuter, NaturalJoin} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Union} +import org.apache.spark.sql.catalyst.plans.physical.{IdentityBroadcastMode, RoundRobinPartitioning, SinglePartition} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.types.{BooleanType, DoubleType, FloatType, IntegerType, Metadata, NullType, StringType, StructField, StructType} +import org.apache.spark.storage.StorageLevel case class Dummy(optKey: Option[Expression]) extends Expression with CodegenFallback { override def children: Seq[Expression] = optKey.toSeq @@ -45,6 +61,20 @@ case class ExpressionInMap(map: Map[String, Expression]) extends Expression with override lazy val resolved = true } +case class JsonTestTreeNode(arg: Any) extends LeafNode { + override def output: Seq[Attribute] = Seq.empty[Attribute] +} + +case class NameValue(name: String, value: Any) + +case object DummyObject + +case class SelfReferenceUDF( + var config: Map[String, Any] = Map.empty[String, Any]) extends Function1[String, Boolean] { + config += "self" -> this + def apply(key: String): Boolean = config.contains(key) +} + class TreeNodeSuite extends SparkFunSuite { test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } @@ -261,4 +291,264 @@ class TreeNodeSuite extends SparkFunSuite { assert(actual === expected) } } + + test("toJSON") { + def assertJSON(input: Any, json: JValue): Unit = { + val expected = + s""" + |[{ + | "class": "${classOf[JsonTestTreeNode].getName}", + | "num-children": 0, + | "arg": ${compact(render(json))} + |}] + """.stripMargin + compareJSON(JsonTestTreeNode(input).toJSON, expected) + } + + // Converts simple types to JSON + assertJSON(true, true) + assertJSON(33.toByte, 33) + assertJSON(44, 44) + assertJSON(55L, 55L) + assertJSON(3.0, 3.0) + assertJSON(4.0D, 4.0D) + assertJSON(BigInt(BigInteger.valueOf(88L)), 88L) + assertJSON(null, JNull) + assertJSON("text", "text") + assertJSON(Some("text"), "text") + compareJSON(JsonTestTreeNode(None).toJSON, + s"""[ + | { + | "class": "${classOf[JsonTestTreeNode].getName}", + | "num-children": 0 + | } + |] + """.stripMargin) + + val uuid = UUID.randomUUID() + assertJSON(uuid, uuid.toString) + + // Converts Spark Sql DataType to JSON + assertJSON(IntegerType, "integer") + assertJSON(Metadata.empty, JObject(Nil)) + assertJSON( + StorageLevel.NONE, + JObject( + "useDisk" -> false, + "useMemory" -> false, + "useOffHeap" -> false, + "deserialized" -> false, + "replication" -> 1) + ) + + // Converts TreeNode argument to JSON + assertJSON( + Literal(333), + List( + JObject( + "class" -> classOf[Literal].getName, + "num-children" -> 0, + "value" -> "333", + "dataType" -> "integer"))) + + // Converts Seq[String] to JSON + assertJSON(Seq("1", "2", "3"), "[1, 2, 3]") + + // Converts Seq[DataType] to JSON + assertJSON(Seq(IntegerType, DoubleType, FloatType), List("integer", "double", "float")) + + // Converts Seq[Partitioning] to JSON + assertJSON( + Seq(SinglePartition, RoundRobinPartitioning(numPartitions = 3)), + List( + JObject("object" -> JString(SinglePartition.getClass.getName)), + JObject( + "product-class" -> classOf[RoundRobinPartitioning].getName, + "numPartitions" -> 3))) + + // Converts case object to JSON + assertJSON(DummyObject, JObject("object" -> JString(DummyObject.getClass.getName))) + + // Converts ExprId to JSON + assertJSON( + ExprId(0, uuid), + JObject( + "product-class" -> classOf[ExprId].getName, + "id" -> 0, + "jvmId" -> uuid.toString)) + + // Converts StructField to JSON + assertJSON( + StructField("field", IntegerType), + JObject( + "product-class" -> classOf[StructField].getName, + "name" -> "field", + "dataType" -> "integer", + "nullable" -> true, + "metadata" -> JObject(Nil))) + + // Converts TableIdentifier to JSON + assertJSON( + TableIdentifier("table"), + JObject( + "product-class" -> classOf[TableIdentifier].getName, + "table" -> "table")) + + // Converts JoinType to JSON + assertJSON( + NaturalJoin(LeftOuter), + JObject( + "product-class" -> classOf[NaturalJoin].getName, + "tpe" -> JObject("object" -> JString(LeftOuter.getClass.getName)))) + + // Converts FunctionIdentifier to JSON + assertJSON( + FunctionIdentifier("function", None), + JObject( + "product-class" -> JString(classOf[FunctionIdentifier].getName), + "funcName" -> "function")) + + // Converts BucketSpec to JSON + assertJSON( + BucketSpec(1, Seq("bucket"), Seq("sort")), + JObject( + "product-class" -> classOf[BucketSpec].getName, + "numBuckets" -> 1, + "bucketColumnNames" -> "[bucket]", + "sortColumnNames" -> "[sort]")) + + // Converts FrameBoundary to JSON + assertJSON( + ValueFollowing(3), + JObject( + "product-class" -> classOf[ValueFollowing].getName, + "value" -> 3)) + + // Converts WindowFrame to JSON + assertJSON( + SpecifiedWindowFrame(RowFrame, UnboundedFollowing, CurrentRow), + JObject( + "product-class" -> classOf[SpecifiedWindowFrame].getName, + "frameType" -> JObject("object" -> JString(RowFrame.getClass.getName)), + "frameStart" -> JObject("object" -> JString(UnboundedFollowing.getClass.getName)), + "frameEnd" -> JObject("object" -> JString(CurrentRow.getClass.getName)))) + + // Converts Partitioning to JSON + assertJSON( + RoundRobinPartitioning(numPartitions = 3), + JObject( + "product-class" -> classOf[RoundRobinPartitioning].getName, + "numPartitions" -> 3)) + + // Converts FunctionResource to JSON + assertJSON( + FunctionResource(JarResource, "file:///"), + JObject( + "product-class" -> JString(classOf[FunctionResource].getName), + "resourceType" -> JObject("object" -> JString(JarResource.getClass.getName)), + "uri" -> "file:///")) + + // Converts BroadcastMode to JSON + assertJSON( + IdentityBroadcastMode, + JObject("object" -> JString(IdentityBroadcastMode.getClass.getName))) + + // Converts CatalogTable to JSON + assertJSON( + CatalogTable( + TableIdentifier("table"), + CatalogTableType.MANAGED, + CatalogStorageFormat.empty, + StructType(StructField("a", IntegerType, true) :: Nil), + createTime = 0L), + + JObject( + "product-class" -> classOf[CatalogTable].getName, + "identifier" -> JObject( + "product-class" -> classOf[TableIdentifier].getName, + "table" -> "table" + ), + "tableType" -> JObject( + "product-class" -> classOf[CatalogTableType].getName, + "name" -> "MANAGED" + ), + "storage" -> JObject( + "product-class" -> classOf[CatalogStorageFormat].getName, + "compressed" -> false, + "properties" -> JNull + ), + "schema" -> JObject( + "type" -> "struct", + "fields" -> List( + JObject( + "name" -> "a", + "type" -> "integer", + "nullable" -> true, + "metadata" -> JObject(Nil)))), + "partitionColumnNames" -> List.empty[String], + "owner" -> "", + "createTime" -> 0, + "lastAccessTime" -> -1, + "properties" -> JNull, + "unsupportedFeatures" -> List.empty[String])) + + // For unknown case class, returns JNull. + val bigValue = new Array[Int](10000) + assertJSON(NameValue("name", bigValue), JNull) + + // Converts Seq[TreeNode] to JSON recursively + assertJSON( + Seq(Literal(1), Literal(2)), + List( + List( + JObject( + "class" -> JString(classOf[Literal].getName), + "num-children" -> 0, + "value" -> "1", + "dataType" -> "integer")), + List( + JObject( + "class" -> JString(classOf[Literal].getName), + "num-children" -> 0, + "value" -> "2", + "dataType" -> "integer")))) + + // Other Seq is converted to JNull, to reduce the risk of out of memory + assertJSON(Seq(1, 2, 3), JNull) + + // All Map type is converted to JNull, to reduce the risk of out of memory + assertJSON(Map("key" -> "value"), JNull) + + // Unknown type is converted to JNull, to reduce the risk of out of memory + assertJSON(new Object {}, JNull) + + // Convert all TreeNode children to JSON + assertJSON( + Union(Seq(JsonTestTreeNode("0"), JsonTestTreeNode("1"))), + List( + JObject( + "class" -> classOf[Union].getName, + "num-children" -> 2, + "children" -> List(0, 1)), + JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "0"), + JObject( + "class" -> classOf[JsonTestTreeNode].getName, + "num-children" -> 0, + "arg" -> "1"))) + } + + test("toJSON should not throws java.lang.StackOverflowError") { + val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr)) + // Should not throw java.lang.StackOverflowError + udf.toJSON + } + + private def compareJSON(leftJson: String, rightJson: String): Unit = { + val left = JsonMethods.parse(leftJson) + val right = JsonMethods.parse(rightJson) + assert(left == right) + } } 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 d361f61764d1f..34fa626e00e31 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 @@ -120,7 +120,6 @@ abstract class QueryTest extends PlanTest { throw ae } } - checkJsonFormat(analyzedDS) assertEmptyMissingInput(analyzedDS) try ds.collect() catch { @@ -168,8 +167,6 @@ abstract class QueryTest extends PlanTest { } } - checkJsonFormat(analyzedDF) - assertEmptyMissingInput(analyzedDF) QueryTest.checkAnswer(analyzedDF, expectedAnswer) match { @@ -228,139 +225,6 @@ abstract class QueryTest extends PlanTest { planWithCaching) } - private def checkJsonFormat(ds: Dataset[_]): Unit = { - // Get the analyzed plan and rewrite the PredicateSubqueries in order to make sure that - // RDD and Data resolution does not break. - val logicalPlan = ds.queryExecution.analyzed - - // bypass some cases that we can't handle currently. - logicalPlan.transform { - case _: ObjectConsumer => return - case _: ObjectProducer => return - case _: AppendColumns => return - case _: TypedFilter => return - case _: LogicalRelation => return - case p if p.getClass.getSimpleName == "MetastoreRelation" => return - case _: MemoryPlan => return - case p: InMemoryRelation => - p.child.transform { - case _: ObjectConsumerExec => return - case _: ObjectProducerExec => return - } - p - }.transformAllExpressions { - case _: ImperativeAggregate => return - case _: TypedAggregateExpression => return - case Literal(_, _: ObjectType) => return - case _: UserDefinedGenerator => return - } - - // bypass hive tests before we fix all corner cases in hive module. - if (this.getClass.getName.startsWith("org.apache.spark.sql.hive")) return - - val jsonString = try { - logicalPlan.toJSON - } catch { - case NonFatal(e) => - fail( - s""" - |Failed to parse logical plan to JSON: - |${logicalPlan.treeString} - """.stripMargin, e) - } - - // scala function is not serializable to JSON, use null to replace them so that we can compare - // the plans later. - val normalized1 = logicalPlan.transformAllExpressions { - case udf: ScalaUDF => udf.copy(function = null) - case gen: UserDefinedGenerator => gen.copy(function = null) - // After SPARK-17356: the JSON representation no longer has the Metadata. We need to remove - // the Metadata from the normalized plan so that we can compare this plan with the - // JSON-deserialzed plan. - case a @ Alias(child, name) if a.explicitMetadata.isDefined => - Alias(child, name)(a.exprId, a.qualifier, Some(Metadata.empty), a.isGenerated) - case a: AttributeReference if a.metadata != Metadata.empty => - AttributeReference(a.name, a.dataType, a.nullable, Metadata.empty)(a.exprId, a.qualifier, - a.isGenerated) - } - - // RDDs/data are not serializable to JSON, so we need to collect LogicalPlans that contains - // these non-serializable stuff, and use these original ones to replace the null-placeholders - // in the logical plans parsed from JSON. - val logicalRDDs = new ArrayDeque[LogicalRDD]() - val localRelations = new ArrayDeque[LocalRelation]() - val inMemoryRelations = new ArrayDeque[InMemoryRelation]() - def collectData: (LogicalPlan => Unit) = { - case l: LogicalRDD => - logicalRDDs.offer(l) - case l: LocalRelation => - localRelations.offer(l) - case i: InMemoryRelation => - inMemoryRelations.offer(i) - case p => - p.expressions.foreach { - _.foreach { - case s: SubqueryExpression => - s.plan.foreach(collectData) - case _ => - } - } - } - logicalPlan.foreach(collectData) - - - val jsonBackPlan = try { - TreeNode.fromJSON[LogicalPlan](jsonString, spark.sparkContext) - } catch { - case NonFatal(e) => - fail( - s""" - |Failed to rebuild the logical plan from JSON: - |${logicalPlan.treeString} - | - |${logicalPlan.prettyJson} - """.stripMargin, e) - } - - def renormalize: PartialFunction[LogicalPlan, LogicalPlan] = { - case l: LogicalRDD => - val origin = logicalRDDs.pop() - LogicalRDD(l.output, origin.rdd)(spark) - case l: LocalRelation => - val origin = localRelations.pop() - l.copy(data = origin.data) - case l: InMemoryRelation => - val origin = inMemoryRelations.pop() - InMemoryRelation( - l.output, - l.useCompression, - l.batchSize, - l.storageLevel, - origin.child, - l.tableName)( - origin.cachedColumnBuffers, - origin.batchStats) - case p => - p.transformExpressions { - case s: SubqueryExpression => - s.withNewPlan(s.plan.transformDown(renormalize)) - } - } - val normalized2 = jsonBackPlan.transformDown(renormalize) - - assert(logicalRDDs.isEmpty) - assert(localRelations.isEmpty) - assert(inMemoryRelations.isEmpty) - - if (normalized1 != normalized2) { - fail( - s""" - |== FAIL: the logical plan parsed from json does not match the original one === - |${sideBySide(logicalPlan.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) - } - } - /** * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans. */ From ef7fa839601805d3aeeb9f002855ac2c49f1ec49 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 16 Sep 2016 11:24:26 -0700 Subject: [PATCH 60/83] [SPARK-17558] Bump Hadoop 2.7 version from 2.7.2 to 2.7.3 ## What changes were proposed in this pull request? This patch bumps the Hadoop version in hadoop-2.7 profile from 2.7.2 to 2.7.3, which was recently released and contained a number of bug fixes. ## How was this patch tested? The change should be covered by existing tests. Author: Reynold Xin Closes #15115 from rxin/SPARK-17558. --- dev/deps/spark-deps-hadoop-2.7 | 30 +++++++++++++++--------------- pom.xml | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index d464c97ed1d67..63566125373dd 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -59,21 +59,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.7.2.jar -hadoop-auth-2.7.2.jar -hadoop-client-2.7.2.jar -hadoop-common-2.7.2.jar -hadoop-hdfs-2.7.2.jar -hadoop-mapreduce-client-app-2.7.2.jar -hadoop-mapreduce-client-common-2.7.2.jar -hadoop-mapreduce-client-core-2.7.2.jar -hadoop-mapreduce-client-jobclient-2.7.2.jar -hadoop-mapreduce-client-shuffle-2.7.2.jar -hadoop-yarn-api-2.7.2.jar -hadoop-yarn-client-2.7.2.jar -hadoop-yarn-common-2.7.2.jar -hadoop-yarn-server-common-2.7.2.jar -hadoop-yarn-server-web-proxy-2.7.2.jar +hadoop-annotations-2.7.3.jar +hadoop-auth-2.7.3.jar +hadoop-client-2.7.3.jar +hadoop-common-2.7.3.jar +hadoop-hdfs-2.7.3.jar +hadoop-mapreduce-client-app-2.7.3.jar +hadoop-mapreduce-client-common-2.7.3.jar +hadoop-mapreduce-client-core-2.7.3.jar +hadoop-mapreduce-client-jobclient-2.7.3.jar +hadoop-mapreduce-client-shuffle-2.7.3.jar +hadoop-yarn-api-2.7.3.jar +hadoop-yarn-client-2.7.3.jar +hadoop-yarn-common-2.7.3.jar +hadoop-yarn-server-common-2.7.3.jar +hadoop-yarn-server-web-proxy-2.7.3.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar diff --git a/pom.xml b/pom.xml index ef83c184d0237..b5141736011b9 100644 --- a/pom.xml +++ b/pom.xml @@ -2524,7 +2524,7 @@ hadoop-2.7 - 2.7.2 + 2.7.3 0.9.3 3.4.6 2.6.0 From ccff86daa99aaf5e5928c56a017268f937f9424d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 16 Sep 2016 13:43:05 -0700 Subject: [PATCH 61/83] [SPARK-17561][DOCS] DataFrameWriter documentation formatting problems ## What changes were proposed in this pull request? Fix `
      /
    • ` problems in SQL scaladoc. ## How was this patch tested? Scaladoc build and manual verification of generated HTML. Author: Sean Owen Closes #15117 from srowen/SPARK-17561. --- .../apache/spark/sql/DataFrameReader.scala | 32 ++++++++-------- .../apache/spark/sql/DataFrameWriter.scala | 12 ++++++ .../sql/streaming/DataStreamReader.scala | 38 ++++++++++++------- 3 files changed, 53 insertions(+), 29 deletions(-) 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 93bf74d06b71d..d29d90ce40453 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 @@ -269,14 +269,15 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts - * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • @@ -395,13 +396,14 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
    • `maxMalformedLogPerPartition` (default `10`): sets the maximum number of malformed rows * Spark will log for each partition. Malformed records beyond this number will be ignored.
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • - `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • - `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • - `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    * @since 2.0.0 */ 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 c05c7a6551600..e137f076a0cab 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 @@ -397,7 +397,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * your external database systems. * * You can set the following JDBC-specific option(s) for storing JDBC: + *
      *
    • `truncate` (default `false`): use `TRUNCATE TABLE` instead of `DROP TABLE`.
    • + *
    * * In case of failures, users should turn off `truncate` option to use `DROP TABLE` again. Also, * due to the different behavior of `TRUNCATE TABLE` among DBMS, it's not always safe to use this. @@ -486,6 +488,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following JSON-specific option(s) for writing JSON files: + *
      *
    • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
    • @@ -495,6 +498,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 1.4.0 */ @@ -510,10 +514,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following Parquet-specific option(s) for writing Parquet files: + *
      *
    • `compression` (default is the value specified in `spark.sql.parquet.compression.codec`): * compression codec to use when saving to file. This can be one of the known case-insensitive * shorten names(none, `snappy`, `gzip`, and `lzo`). This will override * `spark.sql.parquet.compression.codec`.
    • + *
    * * @since 1.4.0 */ @@ -529,9 +535,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following ORC-specific option(s) for writing ORC files: + *
      *
    • `compression` (default `snappy`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names(`none`, `snappy`, `zlib`, and `lzo`). * This will override `orc.compress`.
    • + *
    * * @since 1.5.0 * @note Currently, this method can only be used after enabling Hive support @@ -553,9 +561,11 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following option(s) for writing text files: + *
      *
    • `compression` (default `null`): compression codec to use when saving to file. This can be * one of the known case-insensitive shorten names (`none`, `bzip2`, `gzip`, `lz4`, * `snappy` and `deflate`).
    • + *
    * * @since 1.6.0 */ @@ -571,6 +581,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * }}} * * You can set the following CSV-specific option(s) for writing CSV files: + *
      *
    • `sep` (default `,`): sets the single character as a separator for each * field and value.
    • *
    • `quote` (default `"`): sets the single character used for escaping quoted values where @@ -593,6 +604,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 2.0.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 3ad1125229c97..c25f71af7362a 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 @@ -161,6 +161,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * schema in advance, use the version that specifies the schema to avoid the extra scan. * * You can set the following JSON-specific options to deal with non-standard JSON files: + *
      *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
    • *
    • `primitivesAsString` (default `false`): infers all primitive values as a string type
    • @@ -175,14 +176,15 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `allowBackslashEscapingAnyCharacter` (default `false`): allows accepting quoting of all * character using backslash quoting mechanism
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts the - * malformed string into a new field configured by `columnNameOfCorruptRecord`. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • - *
      + * during parsing. + *
        + *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record, and puts + * the malformed string into a new field configured by `columnNameOfCorruptRecord`. When + * a schema is set by user, it sets `null` for extra fields.
      • + *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • + *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + *
      + * *
    • `columnNameOfCorruptRecord` (default is the value specified in * `spark.sql.columnNameOfCorruptRecord`): allows renaming the new field having malformed string * created by `PERMISSIVE` mode. This overrides `spark.sql.columnNameOfCorruptRecord`.
    • @@ -192,6 +194,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSZZ`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + *
    * * @since 2.0.0 */ @@ -207,6 +210,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * specify the schema explicitly using [[schema]]. * * You can set the following CSV-specific options to deal with CSV files: + *
      *
    • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
    • *
    • `sep` (default `,`): sets the single character as a separator for each @@ -245,12 +249,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
    • `maxCharsPerColumn` (default `1000000`): defines the maximum number of characters allowed * for any given value being read.
    • *
    • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing.
    • - *
        - *
      • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When - * a schema is set by user, it sets `null` for extra fields.
      • - *
      • `DROPMALFORMED` : ignores the whole corrupted records.
      • - *
      • `FAILFAST` : throws an exception when it meets corrupted records.
      • + * during parsing. + *
          + *
        • `PERMISSIVE` : sets other fields to `null` when it meets a corrupted record. When + * a schema is set by user, it sets `null` for extra fields.
        • + *
        • `DROPMALFORMED` : ignores the whole corrupted records.
        • + *
        • `FAILFAST` : throws an exception when it meets corrupted records.
        • + *
        + * *
      * * @since 2.0.0 @@ -263,12 +269,14 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * Loads a Parquet file stream, returning the result as a [[DataFrame]]. * * You can set the following Parquet-specific option(s) for reading Parquet files: + *
        *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
      • *
      • `mergeSchema` (default is the value specified in `spark.sql.parquet.mergeSchema`): sets * whether we should merge schemas collected from all * Parquet part-files. This will override * `spark.sql.parquet.mergeSchema`.
      • + *
      * * @since 2.0.0 */ @@ -292,8 +300,10 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * }}} * * You can set the following text-specific options to deal with text files: + *
        *
      • `maxFilesPerTrigger` (default: no max limit): sets the maximum number of new files to be * considered in every trigger.
      • + *
      * * @since 2.0.0 */ From 60c287f5d3f16b649af7708a189189feedb6e479 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 16 Sep 2016 14:02:56 -0700 Subject: [PATCH 62/83] [SPARK-17549][SQL] Only collect table size stat in driver for cached relation. The existing code caches all stats for all columns for each partition in the driver; for a large relation, this causes extreme memory usage, which leads to gc hell and application failures. It seems that only the size in bytes of the data is actually used in the driver, so instead just colllect that. In executors, the full stats are still kept, but that's not a big problem; we expect the data to be distributed and thus not really incur in too much memory pressure in each individual executor. There are also potential improvements on the executor side, since the data being stored currently is very wasteful (e.g. storing boxed types vs. primitive types for stats). But that's a separate issue. On a mildly related change, I'm also adding code to catch exceptions in the code generator since Janino was breaking with the test data I tried this patch on. Tested with unit tests and by doing a count a very wide table (20k columns) with many partitions. Author: Marcelo Vanzin Closes #15112 from vanzin/SPARK-17549. --- .../expressions/codegen/CodeGenerator.scala | 18 +++++++++----- .../execution/columnar/InMemoryRelation.scala | 24 +++++-------------- .../columnar/InMemoryColumnarQuerySuite.scala | 14 +++++++++++ 3 files changed, 32 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index f982c222af5f0..33b9b804fc601 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -23,6 +23,7 @@ import java.util.{Map => JavaMap} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.util.control.NonFatal import com.google.common.cache.{CacheBuilder, CacheLoader} import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, SimpleCompiler} @@ -910,14 +911,19 @@ object CodeGenerator extends Logging { codeAttrField.setAccessible(true) classes.foreach { case (_, classBytes) => CodegenMetrics.METRIC_GENERATED_CLASS_BYTECODE_SIZE.update(classBytes.length) - val cf = new ClassFile(new ByteArrayInputStream(classBytes)) - cf.methodInfos.asScala.foreach { method => - method.getAttributes().foreach { a => - if (a.getClass.getName == codeAttr.getName) { - CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( - codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + try { + val cf = new ClassFile(new ByteArrayInputStream(classBytes)) + cf.methodInfos.asScala.foreach { method => + method.getAttributes().foreach { a => + if (a.getClass.getName == codeAttr.getName) { + CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE.update( + codeAttrField.get(a).asInstanceOf[Array[Byte]].length) + } } } + } catch { + case NonFatal(e) => + logWarning("Error calculating stats of compiled class.", e) } } } 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 479934a7afc75..56bd5c1891e8d 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 @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.columnar -import scala.collection.JavaConverters._ - import org.apache.commons.lang3.StringUtils import org.apache.spark.network.util.JavaUtils @@ -31,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.Statistics import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.CollectionAccumulator +import org.apache.spark.util.LongAccumulator object InMemoryRelation { @@ -63,8 +61,7 @@ case class InMemoryRelation( @transient child: SparkPlan, tableName: Option[String])( @transient var _cachedColumnBuffers: RDD[CachedBatch] = null, - val batchStats: CollectionAccumulator[InternalRow] = - child.sqlContext.sparkContext.collectionAccumulator[InternalRow]) + val batchStats: LongAccumulator = child.sqlContext.sparkContext.longAccumulator) extends logical.LeafNode with MultiInstanceRelation { override protected def innerChildren: Seq[QueryPlan[_]] = Seq(child) @@ -74,21 +71,12 @@ case class InMemoryRelation( @transient val partitionStatistics = new PartitionStatistics(output) override lazy val statistics: Statistics = { - if (batchStats.value.isEmpty) { + if (batchStats.value == 0L) { // Underlying columnar RDD hasn't been materialized, no useful statistics information // available, return the default statistics. Statistics(sizeInBytes = child.sqlContext.conf.defaultSizeInBytes) } else { - // Underlying columnar RDD has been materialized, required information has also been - // collected via the `batchStats` accumulator. - val sizeOfRow: Expression = - BindReferences.bindReference( - output.map(a => partitionStatistics.forAttribute(a).sizeInBytes).reduce(Add), - partitionStatistics.schema) - - val sizeInBytes = - batchStats.value.asScala.map(row => sizeOfRow.eval(row).asInstanceOf[Long]).sum - Statistics(sizeInBytes = sizeInBytes) + Statistics(sizeInBytes = batchStats.value.longValue) } } @@ -139,10 +127,10 @@ case class InMemoryRelation( rowCount += 1 } + batchStats.add(totalSize) + val stats = InternalRow.fromSeq(columnBuilders.map(_.columnStats.collectedStatistics) .flatMap(_.values)) - - batchStats.add(stats) CachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index 937839644ad5f..0daa29b666f62 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -232,4 +232,18 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { val columnTypes2 = List.fill(length2)(IntegerType) val columnarIterator2 = GenerateColumnAccessor.generate(columnTypes2) } + + test("SPARK-17549: cached table size should be correctly calculated") { + val data = spark.sparkContext.parallelize(1 to 10, 5).toDF() + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None) + + // Materialize the data. + val expectedAnswer = data.collect() + checkAnswer(cached, expectedAnswer) + + // Check that the right size was calculated. + assert(cached.batchStats.value === expectedAnswer.size * INT.defaultSize) + } + } From 19f894e986967dabb8d473db0ab53c2b29e83ded Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Sat, 17 Sep 2016 12:28:42 +0100 Subject: [PATCH 63/83] Correct fetchsize property name in docs ## What changes were proposed in this pull request? Replace `fetchSize` with `fetchsize` in the docs. ## How was this patch tested? I manually tested `fetchSize` and `fetchsize`. The latter has an effect. See also [`JdbcUtils.scala#L38`](https://github.com/apache/spark/blob/v2.0.0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala#L38) for the definition of the property. Author: Daniel Darabos Closes #14975 from darabos/patch-3. --- docs/sql-programming-guide.md | 2 +- .../test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 28cc88c322b7e..4ac5fae566abe 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1053,7 +1053,7 @@ the Data Sources API. The following options are supported: - fetchSize + fetchsize The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). 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 2d8ee338a9804..10f15ca280689 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 @@ -289,7 +289,7 @@ class JDBCSuite extends SparkFunSuite assert(names(2).equals("mary")) } - test("SELECT first field when fetchSize is two") { + test("SELECT first field when fetchsize is two") { val names = sql("SELECT NAME FROM fetchtwo").collect().map(x => x.getString(0)).sortWith(_ < _) assert(names.size === 3) assert(names(0).equals("fred")) @@ -305,7 +305,7 @@ class JDBCSuite extends SparkFunSuite assert(ids(2) === 3) } - test("SELECT second field when fetchSize is two") { + test("SELECT second field when fetchsize is two") { val ids = sql("SELECT THEID FROM fetchtwo").collect().map(x => x.getInt(0)).sortWith(_ < _) assert(ids.size === 3) assert(ids(0) === 1) @@ -352,7 +352,7 @@ class JDBCSuite extends SparkFunSuite urlWithUserAndPass, "TEST.PEOPLE", new Properties()).collect().length === 3) } - test("Basic API with illegal FetchSize") { + test("Basic API with illegal fetchsize") { val properties = new Properties() properties.setProperty(JdbcUtils.JDBC_BATCH_FETCH_SIZE, "-1") val e = intercept[SparkException] { From ae3b76b3ab9a810d94771ddb83a3d86c61d561b8 Mon Sep 17 00:00:00 2001 From: Xin Ren Date: Sat, 17 Sep 2016 12:30:25 +0100 Subject: [PATCH 64/83] [SPARK-17567][DOCS] Use valid url to Spark RDD paper https://issues.apache.org/jira/browse/SPARK-17567 ## What changes were proposed in this pull request? Documentation (http://spark.apache.org/docs/latest/api/scala/#org.apache.spark.rdd.RDD) contains broken link to Spark paper (http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf). I found it elsewhere (https://www.usenix.org/system/files/conference/nsdi12/nsdi12-final138.pdf) and I hope it is the same one. It should be uploaded to and linked from some Apache controlled storage, so it won't break again. ## How was this patch tested? Tested manually on local laptop. Author: Xin Ren Closes #15121 from keypointt/SPARK-17567. --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 10b5f8291a03a..6dc334ceb52ea 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -70,7 +70,7 @@ import org.apache.spark.util.random.{BernoulliCellSampler, BernoulliSampler, Poi * All of the scheduling and execution in Spark is done based on these methods, allowing each RDD * to implement its own way of computing itself. Indeed, users can implement custom RDDs (e.g. for * reading data from a new storage system) by overriding these functions. Please refer to the - * [[http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details + * [[http://people.csail.mit.edu/matei/papers/2012/nsdi_spark.pdf Spark paper]] for more details * on RDD internals. */ abstract class RDD[T: ClassTag]( From b7d1923335a5567f9856da8205e8e9d3392d7644 Mon Sep 17 00:00:00 2001 From: William Benton Date: Sat, 17 Sep 2016 12:49:58 +0100 Subject: [PATCH 65/83] [SPARK-17548][MLLIB] Word2VecModel.findSynonyms no longer spuriously rejects the best match when invoked with a vector ## What changes were proposed in this pull request? This pull request changes the behavior of `Word2VecModel.findSynonyms` so that it will not spuriously reject the best match when invoked with a vector that does not correspond to a word in the model's vocabulary. Instead of blindly discarding the best match, the changed implementation discards a match that corresponds to the query word (in cases where `findSynonyms` is invoked with a word) or that has an identical angle to the query vector. ## How was this patch tested? I added a test to `Word2VecSuite` to ensure that the word with the most similar vector from a supplied vector would not be spuriously rejected. Author: William Benton Closes #15105 from willb/fix/findSynonyms. --- .../apache/spark/ml/feature/Word2Vec.scala | 20 +++++----- .../api/python/Word2VecModelWrapper.scala | 22 +++++++++-- .../apache/spark/mllib/feature/Word2Vec.scala | 37 ++++++++++++++----- .../spark/mllib/feature/Word2VecSuite.scala | 16 ++++++++ python/pyspark/mllib/feature.py | 12 ++++-- 5 files changed, 83 insertions(+), 24 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index c2b434c3d5cb1..14c05123c62ed 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -221,24 +221,26 @@ class Word2VecModel private[ml] ( } /** - * Find "num" number of words closest in similarity to the given word. - * Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word. + * Find "num" number of words closest in similarity to the given word, not + * including the word itself. Returns a dataframe with the words and the + * cosine similarities between the synonyms and the given word. */ @Since("1.5.0") def findSynonyms(word: String, num: Int): DataFrame = { - findSynonyms(wordVectors.transform(word), num) + val spark = SparkSession.builder().getOrCreate() + spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") } /** - * Find "num" number of words closest to similarity to the given vector representation - * of the word. Returns a dataframe with the words and the cosine similarities between the - * synonyms and the given word vector. + * Find "num" number of words whose vector representation most similar to the supplied vector. + * If the supplied vector is the vector representation of a word in the model's vocabulary, + * that word will be in the results. Returns a dataframe with the words and the cosine + * similarities between the synonyms and the given word vector. */ @Since("2.0.0") - def findSynonyms(word: Vector, num: Int): DataFrame = { + def findSynonyms(vec: Vector, num: Int): DataFrame = { val spark = SparkSession.builder().getOrCreate() - spark.createDataFrame(wordVectors.findSynonyms(word, num)).toDF("word", "similarity") + spark.createDataFrame(wordVectors.findSynonyms(vec, num)).toDF("word", "similarity") } /** @group setParam */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala index 4b4ed2291d139..5cbfbff3e4a62 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/Word2VecModelWrapper.scala @@ -43,18 +43,34 @@ private[python] class Word2VecModelWrapper(model: Word2VecModel) { rdd.rdd.map(model.transform) } + /** + * Finds synonyms of a word; do not include the word itself in results. + * @param word a word + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ def findSynonyms(word: String, num: Int): JList[Object] = { - val vec = transform(word) - findSynonyms(vec, num) + prepareResult(model.findSynonyms(word, num)) } + /** + * Finds words similar to the the vector representation of a word without + * filtering results. + * @param vector a vector + * @param num number of synonyms to find + * @return a list consisting of a list of words and a vector of cosine similarities + */ def findSynonyms(vector: Vector, num: Int): JList[Object] = { - val result = model.findSynonyms(vector, num) + prepareResult(model.findSynonyms(vector, num)) + } + + private def prepareResult(result: Array[(String, Double)]) = { val similarity = Vectors.dense(result.map(_._2)) val words = result.map(_._1) List(words, similarity).map(_.asInstanceOf[Object]).asJava } + def getVectors: JMap[String, JList[Float]] = { model.getVectors.map { case (k, v) => (k, v.toList.asJava) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index 908198740b501..42ca9665e5843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -518,7 +518,7 @@ class Word2VecModel private[spark] ( } /** - * Find synonyms of a word + * Find synonyms of a word; do not include the word itself in results. * @param word a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) @@ -526,17 +526,34 @@ class Word2VecModel private[spark] ( @Since("1.1.0") def findSynonyms(word: String, num: Int): Array[(String, Double)] = { val vector = transform(word) - findSynonyms(vector, num) + findSynonyms(vector, num, Some(word)) } /** - * Find synonyms of the vector representation of a word + * Find synonyms of the vector representation of a word, possibly + * including any words in the model vocabulary whose vector respresentation + * is the supplied vector. * @param vector vector representation of a word * @param num number of synonyms to find * @return array of (word, cosineSimilarity) */ @Since("1.1.0") def findSynonyms(vector: Vector, num: Int): Array[(String, Double)] = { + findSynonyms(vector, num, None) + } + + /** + * Find synonyms of the vector representation of a word, rejecting + * words identical to the value of wordOpt, if one is supplied. + * @param vector vector representation of a word + * @param num number of synonyms to find + * @param wordOpt optionally, a word to reject from the results list + * @return array of (word, cosineSimilarity) + */ + private def findSynonyms( + vector: Vector, + num: Int, + wordOpt: Option[String]): Array[(String, Double)] = { require(num > 0, "Number of similar words should > 0") // TODO: optimize top-k val fVector = vector.toArray.map(_.toFloat) @@ -563,12 +580,14 @@ class Word2VecModel private[spark] ( ind += 1 } - wordList.zip(cosVec) - .toSeq - .sortBy(-_._2) - .take(num + 1) - .tail - .toArray + val scored = wordList.zip(cosVec).toSeq.sortBy(-_._2) + + val filtered = wordOpt match { + case Some(w) => scored.take(num + 1).filter(tup => w != tup._1) + case None => scored + } + + filtered.take(num).toArray } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index 22de4c4ac40e6..f4fa216b8eba0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils @@ -68,6 +69,21 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { assert(syms(1)._1 == "japan") } + test("findSynonyms doesn't reject similar word vectors when called with a vector") { + val num = 2 + val word2VecMap = Map( + ("china", Array(0.50f, 0.50f, 0.50f, 0.50f)), + ("japan", Array(0.40f, 0.50f, 0.50f, 0.50f)), + ("taiwan", Array(0.60f, 0.50f, 0.50f, 0.50f)), + ("korea", Array(0.45f, 0.60f, 0.60f, 0.60f)) + ) + val model = new Word2VecModel(word2VecMap) + val syms = model.findSynonyms(Vectors.dense(Array(0.52, 0.5, 0.5, 0.5)), num) + assert(syms.length == num) + assert(syms(0)._1 == "china") + assert(syms(1)._1 == "taiwan") + } + test("model load / save") { val word2VecMap = Map( diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index b32d0c70ec6a7..5d99644fca254 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -544,8 +544,7 @@ def load(cls, sc, path): @ignore_unicode_prefix class Word2Vec(object): - """ - Word2Vec creates vector representation of words in a text corpus. + """Word2Vec creates vector representation of words in a text corpus. The algorithm first constructs a vocabulary from the corpus and then learns vector representation of words in the vocabulary. The vector representation can be used as features in @@ -567,13 +566,19 @@ class Word2Vec(object): >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) + Querying for synonyms of a word will not return that word: + >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] [u'b', u'c'] + + But querying for synonyms of a vector may return the word whose + representation is that vector: + >>> vec = model.transform("a") >>> syms = model.findSynonyms(vec, 2) >>> [s[0] for s in syms] - [u'b', u'c'] + [u'a', u'b'] >>> import os, tempfile >>> path = tempfile.mkdtemp() @@ -591,6 +596,7 @@ class Word2Vec(object): ... pass .. versionadded:: 1.2.0 + """ def __init__(self): """ From 06684bb0b2a9e2487bfe28f380f25f6ca746b7e6 Mon Sep 17 00:00:00 2001 From: David Navas Date: Sat, 17 Sep 2016 16:22:23 +0100 Subject: [PATCH 66/83] [SPARK-17529][CORE] Implement BitSet.clearUntil and use it during merge joins ## What changes were proposed in this pull request? Add a clearUntil() method on BitSet (adapted from the pre-existing setUntil() method). Use this method to clear the subset of the BitSet which needs to be used during merge joins. ## How was this patch tested? dev/run-tests, as well as performance tests on skewed data as described in jira. I expect there to be a small local performance hit using BitSet.clearUntil rather than BitSet.clear for normally shaped (unskewed) joins (additional read on the last long). This is expected to be de-minimis and was not specifically tested. Author: David Navas Closes #15084 from davidnavas/bitSet. --- .../apache/spark/util/collection/BitSet.scala | 28 ++++++++++------ .../spark/util/collection/BitSetSuite.scala | 32 +++++++++++++++++++ .../execution/joins/SortMergeJoinExec.scala | 4 +-- 3 files changed, 52 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index 7ab67fc3a2de9..e63e0e3e1f68f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -17,6 +17,8 @@ package org.apache.spark.util.collection +import java.util.Arrays + /** * A simple, fixed-size bit set implementation. This implementation is fast because it avoids * safety/bound checking. @@ -35,21 +37,14 @@ class BitSet(numBits: Int) extends Serializable { /** * Clear all set bits. */ - def clear(): Unit = { - var i = 0 - while (i < numWords) { - words(i) = 0L - i += 1 - } - } + def clear(): Unit = Arrays.fill(words, 0) /** * Set all the bits up to a given index */ - def setUntil(bitIndex: Int) { + def setUntil(bitIndex: Int): Unit = { val wordIndex = bitIndex >> 6 // divide by 64 - var i = 0 - while(i < wordIndex) { words(i) = -1; i += 1 } + Arrays.fill(words, 0, wordIndex, -1) if(wordIndex < words.length) { // Set the remaining bits (note that the mask could still be zero) val mask = ~(-1L << (bitIndex & 0x3f)) @@ -57,6 +52,19 @@ class BitSet(numBits: Int) extends Serializable { } } + /** + * Clear all the bits up to a given index + */ + def clearUntil(bitIndex: Int): Unit = { + val wordIndex = bitIndex >> 6 // divide by 64 + Arrays.fill(words, 0, wordIndex, 0) + if(wordIndex < words.length) { + // Clear the remaining bits + val mask = -1L << (bitIndex & 0x3f) + words(wordIndex) &= mask + } + } + /** * Compute the bit-wise AND of the two sets returning the * result. diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index 69dbfa9cd7141..0169c9926e68f 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -152,4 +152,36 @@ class BitSetSuite extends SparkFunSuite { assert(bitsetDiff.nextSetBit(85) === 85) assert(bitsetDiff.nextSetBit(86) === -1) } + + test( "[gs]etUntil" ) { + val bitSet = new BitSet(100) + + bitSet.setUntil(bitSet.capacity) + + (0 until bitSet.capacity).foreach { i => + assert(bitSet.get(i)) + } + + bitSet.clearUntil(bitSet.capacity) + + (0 until bitSet.capacity).foreach { i => + assert(!bitSet.get(i)) + } + + val setUntil = bitSet.capacity / 2 + bitSet.setUntil(setUntil) + + val clearUntil = setUntil / 2 + bitSet.clearUntil(clearUntil) + + (0 until clearUntil).foreach { i => + assert(!bitSet.get(i)) + } + (clearUntil until setUntil).foreach { i => + assert(bitSet.get(i)) + } + (setUntil until bitSet.capacity).foreach { i => + assert(!bitSet.get(i)) + } + } } 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 b46af2a99a1e0..81b3e1d224ab6 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 @@ -954,12 +954,12 @@ private class SortMergeFullOuterJoinScanner( } if (leftMatches.size <= leftMatched.capacity) { - leftMatched.clear() + leftMatched.clearUntil(leftMatches.size) } else { leftMatched = new BitSet(leftMatches.size) } if (rightMatches.size <= rightMatched.capacity) { - rightMatched.clear() + rightMatched.clearUntil(rightMatches.size) } else { rightMatched = new BitSet(rightMatches.size) } From 19132d53741c83ea5ce648909479cd175a1d3033 Mon Sep 17 00:00:00 2001 From: sandy Date: Sat, 17 Sep 2016 16:25:03 +0100 Subject: [PATCH 67/83] [SPARK-17575][DOCS] Remove extra table tags in configuration document ## What changes were proposed in this pull request? Remove extra table tags in configurations document. ## How was this patch tested? Run all test cases and generate document. Before with extra tag its look like below ![config-wrong1](https://cloud.githubusercontent.com/assets/8075390/18608239/c602bb60-7d01-11e6-875e-f38558997dd3.png) ![config-wrong2](https://cloud.githubusercontent.com/assets/8075390/18608241/cf3b672c-7d01-11e6-935e-1e73f9e6e578.png) After removing tags its looks like below ![config](https://cloud.githubusercontent.com/assets/8075390/18608245/e156eb8e-7d01-11e6-98aa-3be68d4d1961.png) ![config2](https://cloud.githubusercontent.com/assets/8075390/18608247/e84eecd4-7d01-11e6-9738-a3f7ff8fe834.png) Author: sandy Closes #15130 from phalodi/SPARK-17575. --- docs/configuration.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8aea74505e28b..b50565367a98b 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -123,6 +123,7 @@ of the most common options to set are: Number of cores to use for the driver process, only in cluster mode. + spark.driver.maxResultSize 1g @@ -217,7 +218,7 @@ Apart from these, the following properties are also available, and may be useful
      Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-class-path command line option or in - your default properties file. + your default properties file. @@ -244,7 +245,7 @@ Apart from these, the following properties are also available, and may be useful
      Note: In client mode, this config must not be set through the SparkConf directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-library-path command line option or in - your default properties file. + your default properties file. From defe9aadcf106f5109b4a399fae84665c9cbbae3 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sat, 17 Sep 2016 16:52:30 +0100 Subject: [PATCH 68/83] [SPARK-17480][SQL][FOLLOWUP] Fix more instances which calls List.length/size which is O(n) ## What changes were proposed in this pull request? This PR fixes all the instances which was fixed in the previous PR. To make sure, I manually debugged and also checked the Scala source. `length` in [LinearSeqOptimized.scala#L49-L57](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/LinearSeqOptimized.scala#L49-L57) is O(n). Also, `size` calls `length` via [SeqLike.scala#L106](https://github.com/scala/scala/blob/2.11.x/src/library/scala/collection/SeqLike.scala#L106). For debugging, I have created these as below: ```scala ArrayBuffer(1, 2, 3) Array(1, 2, 3) List(1, 2, 3) Seq(1, 2, 3) ``` and then called `size` and `length` for each to debug. ## How was this patch tested? I ran the bash as below on Mac ```bash find . -name *.scala -type f -exec grep -il "while (.*\\.length)" {} \; | grep "src/main" find . -name *.scala -type f -exec grep -il "while (.*\\.size)" {} \; | grep "src/main" ``` and then checked each. Author: hyukjinkwon Closes #15093 from HyukjinKwon/SPARK-17480-followup. --- .../sql/catalyst/analysis/Analyzer.scala | 28 ++++++------------- .../expressions/conditionalExpressions.scala | 3 +- .../sql/catalyst/expressions/ordering.scala | 3 +- .../sql/catalyst/util/QuantileSummaries.scala | 10 +++---- .../datasources/jdbc/JdbcUtils.scala | 2 +- .../spark/sql/hive/HiveInspectors.scala | 6 ++-- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/hiveUDFs.scala | 3 +- .../spark/sql/hive/orc/OrcFileFormat.scala | 6 ++-- 9 files changed, 31 insertions(+), 33 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 5210f42c557b6..cc62d5e7c8826 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 @@ -1663,27 +1663,17 @@ class Analyzer( } }.toSeq - // Third, for every Window Spec, we add a Window operator and set currentChild as the - // child of it. - var currentChild = child - var i = 0 - while (i < groupedWindowExpressions.size) { - val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) - // Set currentChild to the newly created Window operator. - currentChild = - Window( - windowExpressions, - partitionSpec, - orderSpec, - currentChild) - - // Move to next Window Spec. - i += 1 - } + // Third, we aggregate them by adding each Window operator for each Window Spec and then + // setting this to the child of the next Window operator. + val windowOps = + groupedWindowExpressions.foldLeft(child) { + case (last, ((partitionSpec, orderSpec), windowExpressions)) => + Window(windowExpressions, partitionSpec, orderSpec, last) + } - // Finally, we create a Project to output currentChild's output + // Finally, we create a Project to output windowOps's output // newExpressionsWithWindowFunctions. - Project(currentChild.output ++ newExpressionsWithWindowFunctions, currentChild) + Project(windowOps.output ++ newExpressionsWithWindowFunctions, windowOps) } // end of addWindow // We have to use transformDown at here to make sure the rule of diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 1dd70bcfcfe87..71d4e9a3c9471 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -125,7 +125,8 @@ abstract class CaseWhenBase( override def eval(input: InternalRow): Any = { var i = 0 - while (i < branches.size) { + val size = branches.size + while (i < size) { if (java.lang.Boolean.TRUE.equals(branches(i)._1.eval(input))) { return branches(i)._2.eval(input) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala index 79d2052c38a27..e24a3de3cfdbe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ordering.scala @@ -31,7 +31,8 @@ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 - while (i < ordering.size) { + val size = ordering.size + while (i < size) { val order = ordering(i) val left = order.child.eval(a) val right = order.child.eval(b) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index fd62bd511fac0..27928c493d5ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -91,10 +91,10 @@ class QuantileSummaries( var sampleIdx = 0 // The index of the sample currently being inserted. var opsIdx: Int = 0 - while(opsIdx < sorted.length) { + while (opsIdx < sorted.length) { val currentSample = sorted(opsIdx) // Add all the samples before the next observation. - while(sampleIdx < sampled.size && sampled(sampleIdx).value <= currentSample) { + while (sampleIdx < sampled.length && sampled(sampleIdx).value <= currentSample) { newSamples += sampled(sampleIdx) sampleIdx += 1 } @@ -102,7 +102,7 @@ class QuantileSummaries( // If it is the first one to insert, of if it is the last one currentCount += 1 val delta = - if (newSamples.isEmpty || (sampleIdx == sampled.size && opsIdx == sorted.length - 1)) { + if (newSamples.isEmpty || (sampleIdx == sampled.length && opsIdx == sorted.length - 1)) { 0 } else { math.floor(2 * relativeError * currentCount).toInt @@ -114,7 +114,7 @@ class QuantileSummaries( } // Add all the remaining existing samples - while(sampleIdx < sampled.size) { + while (sampleIdx < sampled.length) { newSamples += sampled(sampleIdx) sampleIdx += 1 } @@ -195,7 +195,7 @@ class QuantileSummaries( // Minimum rank at current sample var minRank = 0 var i = 1 - while (i < sampled.size - 1) { + while (i < sampled.length - 1) { val curSample = sampled(i) minRank += curSample.g val maxRank = minRank + curSample.delta 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 b09fd511a9074..3db1d1f109fb7 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 @@ -369,7 +369,7 @@ object JdbcUtils extends Logging { val bytes = rs.getBytes(pos + 1) var ans = 0L var j = 0 - while (j < bytes.size) { + while (j < bytes.length) { ans = 256 * ans + (255 & bytes(j)) j = j + 1 } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 4e74452f6cd12..e4b963efeaf18 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -703,7 +703,8 @@ private[hive] trait HiveInspectors { // 1. create the pojo (most likely) object val result = x.create() var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { // 2. set the property for the pojo val tpe = structType(i).dataType x.setStructFieldData( @@ -720,7 +721,8 @@ private[hive] trait HiveInspectors { val row = a.asInstanceOf[InternalRow] val result = new java.util.ArrayList[AnyRef](fieldRefs.size) var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { val tpe = structType(i).dataType result.add(wrap(row.get(i, tpe), fieldRefs.get(i).getFieldObjectInspector, tpe)) i += 1 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index b4808fdbed9c9..ec7e53efc87f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -427,7 +427,8 @@ private[hive] object HadoopTableReader extends HiveInspectors with Logging { iterator.map { value => val raw = converter.convert(rawDeser.deserialize(value)) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = soi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 9347aeb8e09a8..962dd5a52ebc0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -153,7 +153,8 @@ private[hive] case class HiveGenericUDF( returnInspector // Make sure initialized. var i = 0 - while (i < children.length) { + val length = children.length + while (i < length) { val idx = i deferredObjects(i).asInstanceOf[DeferredObjectAdapter] .set(() => children(idx).eval(input)) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index 286197b50e229..03b508e11aa76 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -190,7 +190,8 @@ private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) row: InternalRow): Unit = { val fieldRefs = oi.getAllStructFieldRefs var i = 0 - while (i < fieldRefs.size) { + val size = fieldRefs.size + while (i < size) { oi.setStructFieldData( struct, @@ -289,7 +290,8 @@ private[orc] object OrcRelation extends HiveInspectors { iterator.map { value => val raw = deserializer.deserialize(value) var i = 0 - while (i < fieldRefs.length) { + val length = fieldRefs.length + while (i < length) { val fieldValue = oi.getStructFieldData(raw, fieldRefs(i)) if (fieldValue == null) { mutableRow.setNullAt(fieldOrdinals(i)) From 21035a66b3ada119efb0934d67ede8095682a4af Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 17 Sep 2016 11:46:15 -0700 Subject: [PATCH 69/83] [SPARK-17491] Close serialization stream to fix wrong answer bug in putIteratorAsBytes() ## What changes were proposed in this pull request? `MemoryStore.putIteratorAsBytes()` may silently lose values when used with `KryoSerializer` because it does not properly close the serialization stream before attempting to deserialize the already-serialized values, which may cause values buffered in Kryo's internal buffers to not be read. This is the root cause behind a user-reported "wrong answer" bug in PySpark caching reported by bennoleslie on the Spark user mailing list in a thread titled "pyspark persist MEMORY_ONLY vs MEMORY_AND_DISK". Due to Spark 2.0's automatic use of KryoSerializer for "safe" types (such as byte arrays, primitives, etc.) this misuse of serializers manifested itself as silent data corruption rather than a StreamCorrupted error (which you might get from JavaSerializer). The minimal fix, implemented here, is to close the serialization stream before attempting to deserialize written values. In addition, this patch adds several additional assertions / precondition checks to prevent misuse of `PartiallySerializedBlock` and `ChunkedByteBufferOutputStream`. ## How was this patch tested? The original bug was masked by an invalid assert in the memory store test cases: the old assert compared two results record-by-record with `zip` but didn't first check that the lengths of the two collections were equal, causing missing records to go unnoticed. The updated test case reproduced this bug. In addition, I added a new `PartiallySerializedBlockSuite` to unit test that component. Author: Josh Rosen Closes #15043 from JoshRosen/partially-serialized-block-values-iterator-bugfix. --- .../org/apache/spark/scheduler/Task.scala | 1 + .../spark/storage/memory/MemoryStore.scala | 89 ++++++-- .../spark/util/ByteBufferOutputStream.scala | 27 ++- .../io/ChunkedByteBufferOutputStream.scala | 12 +- .../spark/storage/MemoryStoreSuite.scala | 34 ++- .../PartiallySerializedBlockSuite.scala | 215 ++++++++++++++++++ .../PartiallyUnrolledIteratorSuite.scala | 2 +- .../ChunkedByteBufferOutputStreamSuite.scala | 8 + 8 files changed, 344 insertions(+), 44 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index 35c4dafe9c19c..1ed36bf0692f8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -230,6 +230,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task) Utils.writeByteBuffer(taskBytes, out) + out.close() out.toByteBuffer } diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index ec1b0f7149271..205d469f48144 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -33,7 +33,7 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage.{BlockId, BlockInfoManager, StorageLevel} import org.apache.spark.unsafe.Platform -import org.apache.spark.util.{CompletionIterator, SizeEstimator, Utils} +import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -277,6 +277,7 @@ private[spark] class MemoryStore( "released too much unroll memory") Left(new PartiallyUnrolledIterator( this, + MemoryMode.ON_HEAP, unrollMemoryUsedByThisBlock, unrolled = arrayValues.toIterator, rest = Iterator.empty)) @@ -285,7 +286,11 @@ private[spark] class MemoryStore( // We ran out of space while unrolling the values for this block logUnrollFailureMessage(blockId, vector.estimateSize()) Left(new PartiallyUnrolledIterator( - this, unrollMemoryUsedByThisBlock, unrolled = vector.iterator, rest = values)) + this, + MemoryMode.ON_HEAP, + unrollMemoryUsedByThisBlock, + unrolled = vector.iterator, + rest = values)) } } @@ -394,7 +399,7 @@ private[spark] class MemoryStore( redirectableStream, unrollMemoryUsedByThisBlock, memoryMode, - bbos.toChunkedByteBuffer, + bbos, values, classTag)) } @@ -655,6 +660,7 @@ private[spark] class MemoryStore( * The result of a failed [[MemoryStore.putIteratorAsValues()]] call. * * @param memoryStore the memoryStore, used for freeing memory. + * @param memoryMode the memory mode (on- or off-heap). * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. * @param unrolled an iterator for the partially-unrolled values. * @param rest the rest of the original iterator passed to @@ -662,13 +668,14 @@ private[spark] class MemoryStore( */ private[storage] class PartiallyUnrolledIterator[T]( memoryStore: MemoryStore, + memoryMode: MemoryMode, unrollMemory: Long, private[this] var unrolled: Iterator[T], rest: Iterator[T]) extends Iterator[T] { private def releaseUnrollMemory(): Unit = { - memoryStore.releaseUnrollMemoryForThisTask(MemoryMode.ON_HEAP, unrollMemory) + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) // SPARK-17503: Garbage collects the unrolling memory before the life end of // PartiallyUnrolledIterator. unrolled = null @@ -706,7 +713,7 @@ private[storage] class PartiallyUnrolledIterator[T]( /** * A wrapper which allows an open [[OutputStream]] to be redirected to a different sink. */ -private class RedirectableOutputStream extends OutputStream { +private[storage] class RedirectableOutputStream extends OutputStream { private[this] var os: OutputStream = _ def setOutputStream(s: OutputStream): Unit = { os = s } override def write(b: Int): Unit = os.write(b) @@ -726,7 +733,8 @@ private class RedirectableOutputStream extends OutputStream { * @param redirectableOutputStream an OutputStream which can be redirected to a different sink. * @param unrollMemory the amount of unroll memory used by the values in `unrolled`. * @param memoryMode whether the unroll memory is on- or off-heap - * @param unrolled a byte buffer containing the partially-serialized values. + * @param bbos byte buffer output stream containing the partially-serialized values. + * [[redirectableOutputStream]] initially points to this output stream. * @param rest the rest of the original iterator passed to * [[MemoryStore.putIteratorAsValues()]]. * @param classTag the [[ClassTag]] for the block. @@ -735,14 +743,19 @@ private[storage] class PartiallySerializedBlock[T]( memoryStore: MemoryStore, serializerManager: SerializerManager, blockId: BlockId, - serializationStream: SerializationStream, - redirectableOutputStream: RedirectableOutputStream, - unrollMemory: Long, + private val serializationStream: SerializationStream, + private val redirectableOutputStream: RedirectableOutputStream, + val unrollMemory: Long, memoryMode: MemoryMode, - unrolled: ChunkedByteBuffer, + bbos: ChunkedByteBufferOutputStream, rest: Iterator[T], classTag: ClassTag[T]) { + private lazy val unrolledBuffer: ChunkedByteBuffer = { + bbos.close() + bbos.toChunkedByteBuffer + } + // If the task does not fully consume `valuesIterator` or otherwise fails to consume or dispose of // this PartiallySerializedBlock then we risk leaking of direct buffers, so we use a task // completion listener here in order to ensure that `unrolled.dispose()` is called at least once. @@ -751,7 +764,23 @@ private[storage] class PartiallySerializedBlock[T]( taskContext.addTaskCompletionListener { _ => // When a task completes, its unroll memory will automatically be freed. Thus we do not call // releaseUnrollMemoryForThisTask() here because we want to avoid double-freeing. - unrolled.dispose() + unrolledBuffer.dispose() + } + } + + // Exposed for testing + private[storage] def getUnrolledChunkedByteBuffer: ChunkedByteBuffer = unrolledBuffer + + private[this] var discarded = false + private[this] var consumed = false + + private def verifyNotConsumedAndNotDiscarded(): Unit = { + if (consumed) { + throw new IllegalStateException( + "Can only call one of finishWritingToStream() or valuesIterator() and can only call once.") + } + if (discarded) { + throw new IllegalStateException("Cannot call methods on a discarded PartiallySerializedBlock") } } @@ -759,15 +788,18 @@ private[storage] class PartiallySerializedBlock[T]( * Called to dispose of this block and free its memory. */ def discard(): Unit = { - try { - // We want to close the output stream in order to free any resources associated with the - // serializer itself (such as Kryo's internal buffers). close() might cause data to be - // written, so redirect the output stream to discard that data. - redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) - serializationStream.close() - } finally { - unrolled.dispose() - memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + if (!discarded) { + try { + // We want to close the output stream in order to free any resources associated with the + // serializer itself (such as Kryo's internal buffers). close() might cause data to be + // written, so redirect the output stream to discard that data. + redirectableOutputStream.setOutputStream(ByteStreams.nullOutputStream()) + serializationStream.close() + } finally { + discarded = true + unrolledBuffer.dispose() + memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) + } } } @@ -776,8 +808,10 @@ private[storage] class PartiallySerializedBlock[T]( * and then serializing the values from the original input iterator. */ def finishWritingToStream(os: OutputStream): Unit = { + verifyNotConsumedAndNotDiscarded() + consumed = true // `unrolled`'s underlying buffers will be freed once this input stream is fully read: - ByteStreams.copy(unrolled.toInputStream(dispose = true), os) + ByteStreams.copy(unrolledBuffer.toInputStream(dispose = true), os) memoryStore.releaseUnrollMemoryForThisTask(memoryMode, unrollMemory) redirectableOutputStream.setOutputStream(os) while (rest.hasNext) { @@ -794,13 +828,22 @@ private[storage] class PartiallySerializedBlock[T]( * `close()` on it to free its resources. */ def valuesIterator: PartiallyUnrolledIterator[T] = { + verifyNotConsumedAndNotDiscarded() + consumed = true + // Close the serialization stream so that the serializer's internal buffers are freed and any + // "end-of-stream" markers can be written out so that `unrolled` is a valid serialized stream. + serializationStream.close() // `unrolled`'s underlying buffers will be freed once this input stream is fully read: val unrolledIter = serializerManager.dataDeserializeStream( - blockId, unrolled.toInputStream(dispose = true))(classTag) + blockId, unrolledBuffer.toInputStream(dispose = true))(classTag) + // The unroll memory will be freed once `unrolledIter` is fully consumed in + // PartiallyUnrolledIterator. If the iterator is not consumed by the end of the task then any + // extra unroll memory will automatically be freed by a `finally` block in `Task`. new PartiallyUnrolledIterator( memoryStore, + memoryMode, unrollMemory, - unrolled = CompletionIterator[T, Iterator[T]](unrolledIter, discard()), + unrolled = unrolledIter, rest = rest) } } diff --git a/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala index 09e7579ae9606..9077b86f9ba1d 100644 --- a/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/ByteBufferOutputStream.scala @@ -29,7 +29,32 @@ private[spark] class ByteBufferOutputStream(capacity: Int) extends ByteArrayOutp def getCount(): Int = count + private[this] var closed: Boolean = false + + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b) + } + + override def write(b: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ByteBufferOutputStream") + super.write(b, off, len) + } + + override def reset(): Unit = { + require(!closed, "cannot reset a closed ByteBufferOutputStream") + super.reset() + } + + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + def toByteBuffer: ByteBuffer = { - return ByteBuffer.wrap(buf, 0, count) + require(closed, "can only call toByteBuffer() after ByteBufferOutputStream has been closed") + ByteBuffer.wrap(buf, 0, count) } } diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala index 67b50d1e70437..a625b3289538a 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStream.scala @@ -49,10 +49,19 @@ private[spark] class ChunkedByteBufferOutputStream( */ private[this] var position = chunkSize private[this] var _size = 0 + private[this] var closed: Boolean = false def size: Long = _size + override def close(): Unit = { + if (!closed) { + super.close() + closed = true + } + } + override def write(b: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") allocateNewChunkIfNeeded() chunks(lastChunkIndex).put(b.toByte) position += 1 @@ -60,6 +69,7 @@ private[spark] class ChunkedByteBufferOutputStream( } override def write(bytes: Array[Byte], off: Int, len: Int): Unit = { + require(!closed, "cannot write to a closed ChunkedByteBufferOutputStream") var written = 0 while (written < len) { allocateNewChunkIfNeeded() @@ -73,7 +83,6 @@ private[spark] class ChunkedByteBufferOutputStream( @inline private def allocateNewChunkIfNeeded(): Unit = { - require(!toChunkedByteBufferWasCalled, "cannot write after toChunkedByteBuffer() is called") if (position == chunkSize) { chunks += allocator(chunkSize) lastChunkIndex += 1 @@ -82,6 +91,7 @@ private[spark] class ChunkedByteBufferOutputStream( } def toChunkedByteBuffer: ChunkedByteBuffer = { + require(closed, "cannot call toChunkedByteBuffer() unless close() has been called") require(!toChunkedByteBufferWasCalled, "toChunkedByteBuffer() can only be called once") toChunkedByteBufferWasCalled = true if (lastChunkIndex == -1) { 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 c11de826677e0..9929ea033a99f 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -79,6 +79,13 @@ class MemoryStoreSuite (memoryStore, blockInfoManager) } + private def assertSameContents[T](expected: Seq[T], actual: Seq[T], hint: String): Unit = { + assert(actual.length === expected.length, s"wrong number of values returned in $hint") + expected.iterator.zip(actual.iterator).foreach { case (e, a) => + assert(e === a, s"$hint did not return original values!") + } + } + test("reserve/release unroll memory") { val (memoryStore, _) = makeMemoryStore(12000) assert(memoryStore.currentUnrollMemory === 0) @@ -137,9 +144,7 @@ class MemoryStoreSuite var putResult = putIteratorAsValues("unroll", smallList.iterator, ClassTag.Any) assert(putResult.isRight) assert(memoryStore.currentUnrollMemoryForThisTask === 0) - smallList.iterator.zip(memoryStore.getValues("unroll").get).foreach { case (e, a) => - assert(e === a, "getValues() did not return original values!") - } + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") blockInfoManager.lockForWriting("unroll") assert(memoryStore.remove("unroll")) blockInfoManager.removeBlock("unroll") @@ -152,9 +157,7 @@ class MemoryStoreSuite assert(memoryStore.currentUnrollMemoryForThisTask === 0) assert(memoryStore.contains("someBlock2")) assert(!memoryStore.contains("someBlock1")) - smallList.iterator.zip(memoryStore.getValues("unroll").get).foreach { case (e, a) => - assert(e === a, "getValues() did not return original values!") - } + assertSameContents(smallList, memoryStore.getValues("unroll").get.toSeq, "getValues") blockInfoManager.lockForWriting("unroll") assert(memoryStore.remove("unroll")) blockInfoManager.removeBlock("unroll") @@ -167,9 +170,7 @@ class MemoryStoreSuite assert(memoryStore.currentUnrollMemoryForThisTask > 0) // we returned an iterator assert(!memoryStore.contains("someBlock2")) assert(putResult.isLeft) - bigList.iterator.zip(putResult.left.get).foreach { case (e, a) => - assert(e === a, "putIterator() did not return original values!") - } + assertSameContents(bigList, putResult.left.get.toSeq, "putIterator") // The unroll memory was freed once the iterator returned by putIterator() was fully traversed. assert(memoryStore.currentUnrollMemoryForThisTask === 0) } @@ -316,9 +317,8 @@ class MemoryStoreSuite assert(res.isLeft) assert(memoryStore.currentUnrollMemoryForThisTask > 0) val valuesReturnedFromFailedPut = res.left.get.valuesIterator.toSeq // force materialization - valuesReturnedFromFailedPut.zip(bigList).foreach { case (e, a) => - assert(e === a, "PartiallySerializedBlock.valuesIterator() did not return original values!") - } + assertSameContents( + bigList, valuesReturnedFromFailedPut, "PartiallySerializedBlock.valuesIterator()") // The unroll memory was freed once the iterator was fully traversed. assert(memoryStore.currentUnrollMemoryForThisTask === 0) } @@ -340,12 +340,10 @@ class MemoryStoreSuite res.left.get.finishWritingToStream(bos) // The unroll memory was freed once the block was fully written. assert(memoryStore.currentUnrollMemoryForThisTask === 0) - val deserializationStream = serializerManager.dataDeserializeStream[Any]( - "b1", new ByteBufferInputStream(bos.toByteBuffer))(ClassTag.Any) - deserializationStream.zip(bigList.iterator).foreach { case (e, a) => - assert(e === a, - "PartiallySerializedBlock.finishWritingtoStream() did not write original values!") - } + val deserializedValues = serializerManager.dataDeserializeStream[Any]( + "b1", new ByteBufferInputStream(bos.toByteBuffer))(ClassTag.Any).toSeq + assertSameContents( + bigList, deserializedValues, "PartiallySerializedBlock.finishWritingToStream()") } test("multiple unrolls by the same thread") { diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala new file mode 100644 index 0000000000000..ec4f2637fadd0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala @@ -0,0 +1,215 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.storage + +import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import org.mockito.Mockito +import org.mockito.Mockito.atLeastOnce +import org.mockito.invocation.InvocationOnMock +import org.mockito.stubbing.Answer +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} + +import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} +import org.apache.spark.memory.MemoryMode +import org.apache.spark.serializer.{JavaSerializer, SerializationStream, SerializerManager} +import org.apache.spark.storage.memory.{MemoryStore, PartiallySerializedBlock, RedirectableOutputStream} +import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream} +import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} + +class PartiallySerializedBlockSuite + extends SparkFunSuite + with BeforeAndAfterEach + with PrivateMethodTester { + + private val blockId = new TestBlockId("test") + private val conf = new SparkConf() + private val memoryStore = Mockito.mock(classOf[MemoryStore], Mockito.RETURNS_SMART_NULLS) + private val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) + + private val getSerializationStream = PrivateMethod[SerializationStream]('serializationStream) + private val getRedirectableOutputStream = + PrivateMethod[RedirectableOutputStream]('redirectableOutputStream) + + override protected def beforeEach(): Unit = { + super.beforeEach() + Mockito.reset(memoryStore) + } + + private def partiallyUnroll[T: ClassTag]( + iter: Iterator[T], + numItemsToBuffer: Int): PartiallySerializedBlock[T] = { + + val bbos: ChunkedByteBufferOutputStream = { + val spy = Mockito.spy(new ChunkedByteBufferOutputStream(128, ByteBuffer.allocate)) + Mockito.doAnswer(new Answer[ChunkedByteBuffer] { + override def answer(invocationOnMock: InvocationOnMock): ChunkedByteBuffer = { + Mockito.spy(invocationOnMock.callRealMethod().asInstanceOf[ChunkedByteBuffer]) + } + }).when(spy).toChunkedByteBuffer + spy + } + + val serializer = serializerManager.getSerializer(implicitly[ClassTag[T]]).newInstance() + val redirectableOutputStream = Mockito.spy(new RedirectableOutputStream) + redirectableOutputStream.setOutputStream(bbos) + val serializationStream = Mockito.spy(serializer.serializeStream(redirectableOutputStream)) + + (1 to numItemsToBuffer).foreach { _ => + assert(iter.hasNext) + serializationStream.writeObject[T](iter.next()) + } + + val unrollMemory = bbos.size + new PartiallySerializedBlock[T]( + memoryStore, + serializerManager, + blockId, + serializationStream = serializationStream, + redirectableOutputStream, + unrollMemory = unrollMemory, + memoryMode = MemoryMode.ON_HEAP, + bbos, + rest = iter, + classTag = implicitly[ClassTag[T]]) + } + + test("valuesIterator() and finishWritingToStream() cannot be called after discard() is called") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(null) + } + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("discard() can be called more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.discard() + partiallySerializedBlock.discard() + } + + test("cannot call valuesIterator() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("cannot call finishWritingToStream() more than once") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call finishWritingToStream() after valuesIterator()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.valuesIterator + intercept[IllegalStateException] { + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + } + } + + test("cannot call valuesIterator() after finishWritingToStream()") { + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + partiallySerializedBlock.finishWritingToStream(new ByteBufferOutputStream()) + intercept[IllegalStateException] { + partiallySerializedBlock.valuesIterator + } + } + + test("buffers are deallocated in a TaskCompletionListener") { + try { + TaskContext.setTaskContext(TaskContext.empty()) + val partiallySerializedBlock = partiallyUnroll((1 to 10).iterator, 2) + TaskContext.get().asInstanceOf[TaskContextImpl].markTaskCompleted() + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer).dispose() + Mockito.verifyNoMoreInteractions(memoryStore) + } finally { + TaskContext.unset() + } + } + + private def testUnroll[T: ClassTag]( + testCaseName: String, + items: Seq[T], + numItemsToBuffer: Int): Unit = { + + test(s"$testCaseName with discard() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + partiallySerializedBlock.discard() + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + } + + test(s"$testCaseName with finishWritingToStream() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val bbos = Mockito.spy(new ByteBufferOutputStream()) + partiallySerializedBlock.finishWritingToStream(bbos) + + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + Mockito.verify(bbos).close() + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + + val serializer = serializerManager.getSerializer(implicitly[ClassTag[T]]).newInstance() + val deserialized = + serializer.deserializeStream(new ByteBufferInputStream(bbos.toByteBuffer)).asIterator.toSeq + assert(deserialized === items) + } + + test(s"$testCaseName with valuesIterator() and numBuffered = $numItemsToBuffer") { + val partiallySerializedBlock = partiallyUnroll(items.iterator, numItemsToBuffer) + val valuesIterator = partiallySerializedBlock.valuesIterator + Mockito.verify(partiallySerializedBlock.invokePrivate(getSerializationStream())).close() + Mockito.verify(partiallySerializedBlock.invokePrivate(getRedirectableOutputStream())).close() + + val deserializedItems = valuesIterator.toArray.toSeq + Mockito.verify(memoryStore).releaseUnrollMemoryForThisTask( + MemoryMode.ON_HEAP, partiallySerializedBlock.unrollMemory) + Mockito.verifyNoMoreInteractions(memoryStore) + Mockito.verify(partiallySerializedBlock.getUnrolledChunkedByteBuffer, atLeastOnce).dispose() + assert(deserializedItems === items) + } + } + + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 50) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 0) + testUnroll("basic numbers", 1 to 1000, numItemsToBuffer = 1000) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 50) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 0) + testUnroll("case classes", (1 to 1000).map(x => MyCaseClass(x.toString)), numItemsToBuffer = 1000) + testUnroll("empty iterator", Seq.empty[String], numItemsToBuffer = 0) +} + +private case class MyCaseClass(str: String) 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 02c2331dc3946..4253cc8ca4cd1 100644 --- a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala @@ -33,7 +33,7 @@ class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { val rest = (unrollSize until restSize + unrollSize).iterator val memoryStore = mock[MemoryStore] - val joinIterator = new PartiallyUnrolledIterator(memoryStore, unrollSize, unroll, rest) + val joinIterator = new PartiallyUnrolledIterator(memoryStore, ON_HEAP, unrollSize, unroll, rest) // Firstly iterate over unrolling memory iterator (0 until unrollSize).foreach { value => diff --git a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala index 226622075a6cc..86961745673c6 100644 --- a/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/io/ChunkedByteBufferOutputStreamSuite.scala @@ -28,12 +28,14 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { test("empty output") { val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) + o.close() assert(o.toChunkedByteBuffer.size === 0) } test("write a single byte") { val o = new ChunkedByteBufferOutputStream(1024, ByteBuffer.allocate) o.write(10) + o.close() val chunkedByteBuffer = o.toChunkedByteBuffer assert(chunkedByteBuffer.getChunks().length === 1) assert(chunkedByteBuffer.getChunks().head.array().toSeq === Seq(10.toByte)) @@ -43,6 +45,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](9)) o.write(99) + o.close() val chunkedByteBuffer = o.toChunkedByteBuffer assert(chunkedByteBuffer.getChunks().length === 1) assert(chunkedByteBuffer.getChunks().head.array()(9) === 99.toByte) @@ -52,6 +55,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(new Array[Byte](10)) o.write(99) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 2) assert(arrays(1).length === 1) @@ -63,6 +67,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) @@ -74,6 +79,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 1) assert(arrays.head.length === ref.length) @@ -85,6 +91,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) @@ -101,6 +108,7 @@ class ChunkedByteBufferOutputStreamSuite extends SparkFunSuite { Random.nextBytes(ref) val o = new ChunkedByteBufferOutputStream(10, ByteBuffer.allocate) o.write(ref) + o.close() val arrays = o.toChunkedByteBuffer.getChunks().map(_.array()) assert(arrays.length === 3) assert(arrays(0).length === 10) From d6cbf8d54cf3df2bfa67b42265f07fce47b6c1e7 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sun, 18 Sep 2016 15:37:15 +0800 Subject: [PATCH 70/83] [SPARK-17518][SQL] Block Users to Specify the Internal Data Source Provider Hive ### What changes were proposed in this pull request? In Spark 2.1, we introduced a new internal provider `hive` for telling Hive serde tables from data source tables. This PR is to block users to specify this in `DataFrameWriter` and SQL APIs. ### How was this patch tested? Added a test case Author: gatorsmile Closes #15073 from gatorsmile/formatHive. --- .../apache/spark/sql/DataFrameWriter.scala | 3 ++ .../spark/sql/execution/SparkSqlParser.scala | 5 +- .../spark/sql/internal/CatalogImpl.scala | 2 +- .../spark/sql/internal/CatalogSuite.scala | 7 +++ .../spark/sql/hive/HiveStrategies.scala | 2 +- .../sql/hive/MetastoreDataSourcesSuite.scala | 51 +++++++++++++++++++ 6 files changed, 67 insertions(+), 3 deletions(-) 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 e137f076a0cab..64d3422cb4b54 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 @@ -357,6 +357,9 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveAsTable(tableIdent: TableIdentifier): Unit = { + if (source.toLowerCase == "hive") { + throw new AnalysisException("Cannot create hive serde table with saveAsTable API") + } val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) 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 7ba1a9ff223de..5359cedc80974 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 @@ -22,7 +22,7 @@ import scala.collection.JavaConverters._ import org.antlr.v4.runtime.{ParserRuleContext, Token} import org.antlr.v4.runtime.tree.TerminalNode -import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.parser._ @@ -316,6 +316,9 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder { } val options = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty) val provider = ctx.tableProvider.qualifiedName.getText + if (provider.toLowerCase == "hive") { + throw new AnalysisException("Cannot create hive serde table with CREATE TABLE USING") + } val schema = Option(ctx.colTypeList()).map(createStructType) val partitionColumnNames = Option(ctx.partitionColumnNames) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 1f87f0e73a3ba..78ad710a6262e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -258,7 +258,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { source: String, schema: StructType, options: Map[String, String]): DataFrame = { - if (source == "hive") { + if (source.toLowerCase == "hive") { throw new AnalysisException("Cannot create hive serde table with createExternalTable API.") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index b221eed7b2426..549fd63f7462e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -322,6 +322,13 @@ class CatalogSuite assert(e2.message == "Cannot create a file-based external data source table without path") } + test("createExternalTable should fail if provider is hive") { + val e = intercept[AnalysisException] { + spark.catalog.createExternalTable("tbl", "HiVe", Map.empty[String, String]) + } + assert(e.message.contains("Cannot create hive serde table with createExternalTable API")) + } + // TODO: add tests for the rest of them } 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 fb11c849edd94..9d2930948d6ba 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 @@ -61,7 +61,7 @@ private[hive] trait HiveStrategies { // `ErrorIfExists` mode, and `DataFrameWriter.saveAsTable` doesn't support hive serde // tables yet. if (mode == SaveMode.Append || mode == SaveMode.Overwrite) { - throw new AnalysisException("" + + throw new AnalysisException( "CTAS for hive serde tables does not support append or overwrite semantics.") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 3466733d7fdcd..0f331bae930f4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.client.HiveClient @@ -1151,6 +1152,56 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } + test("save API - format hive") { + withTempDir { dir => + val path = dir.getCanonicalPath + val e = intercept[ClassNotFoundException] { + spark.range(10).write.format("hive").mode(SaveMode.Ignore).save(path) + }.getMessage + assert(e.contains("Failed to find data source: hive")) + } + } + + test("saveAsTable API - format hive") { + val tableName = "tab1" + withTable(tableName) { + val e = intercept[AnalysisException] { + spark.range(10).write.format("hive").mode(SaveMode.Overwrite).saveAsTable(tableName) + }.getMessage + assert(e.contains("Cannot create hive serde table with saveAsTable API")) + } + } + + test("create a data source table using hive") { + val tableName = "tab1" + withTable (tableName) { + val e = intercept[AnalysisException] { + sql( + s""" + |CREATE TABLE $tableName + |(col1 int) + |USING hive + """.stripMargin) + }.getMessage + assert(e.contains("Cannot create hive serde table with CREATE TABLE USING")) + } + } + + test("create a temp view using hive") { + val tableName = "tab1" + withTable (tableName) { + val e = intercept[ClassNotFoundException] { + sql( + s""" + |CREATE TEMPORARY VIEW $tableName + |(col1 int) + |USING hive + """.stripMargin) + }.getMessage + assert(e.contains("Failed to find data source: hive")) + } + } + test("saveAsTable - source and target are the same table") { val tableName = "tab1" withTable(tableName) { From 3357bc76df10e975b94c6dd96d1df6bb30daef5e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 18 Sep 2016 21:15:35 +0800 Subject: [PATCH 71/83] [SPARK-17541][SQL] fix some DDL bugs about table management when same-name temp view exists ## What changes were proposed in this pull request? In `SessionCatalog`, we have several operations(`tableExists`, `dropTable`, `loopupRelation`, etc) that handle both temp views and metastore tables/views. This brings some bugs to DDL commands that want to handle temp view only or metastore table/view only. These bugs are: 1. `CREATE TABLE USING` will fail if a same-name temp view exists 2. `Catalog.dropTempView`will un-cache and drop metastore table if a same-name table exists 3. `saveAsTable` will fail or have unexpected behaviour if a same-name temp view exists. These bug fixes are pulled out from https://github.com/apache/spark/pull/14962 and targets both master and 2.0 branch ## How was this patch tested? new regression tests Author: Wenchen Fan Closes #15099 from cloud-fan/fix-view. --- .../sql/catalyst/catalog/SessionCatalog.scala | 32 +++++--- .../catalog/SessionCatalogSuite.scala | 24 +++--- .../apache/spark/sql/DataFrameWriter.scala | 9 ++- .../command/createDataSourceTables.scala | 22 ++++-- .../spark/sql/internal/CatalogImpl.scala | 8 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 11 +++ .../spark/sql/internal/CatalogSuite.scala | 11 +++ .../sql/test/DataFrameReaderWriterSuite.scala | 76 +++++++++++++++++++ .../sql/hive/MetastoreDataSourcesSuite.scala | 13 ++-- .../sql/sources/HadoopFsRelationTest.scala | 10 +-- 10 files changed, 170 insertions(+), 46 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 9fb5db573b70f..574c3d7eeeec9 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 @@ -325,9 +325,9 @@ class SessionCatalog( new Path(new Path(dbLocation), formatTableName(tableIdent.table)).toString } - // ------------------------------------------------------------- - // | Methods that interact with temporary and metastore tables | - // ------------------------------------------------------------- + // ---------------------------------------------- + // | Methods that interact with temp views only | + // ---------------------------------------------- /** * Create a temporary table. @@ -343,6 +343,24 @@ class SessionCatalog( tempTables.put(table, tableDefinition) } + /** + * Return a temporary view exactly as it was stored. + */ + def getTempView(name: String): Option[LogicalPlan] = synchronized { + tempTables.get(formatTableName(name)) + } + + /** + * Drop a temporary view. + */ + def dropTempView(name: String): Unit = synchronized { + tempTables.remove(formatTableName(name)) + } + + // ------------------------------------------------------------- + // | Methods that interact with temporary and metastore tables | + // ------------------------------------------------------------- + /** * Rename a table. * @@ -492,14 +510,6 @@ class SessionCatalog( tempTables.clear() } - /** - * Return a temporary table exactly as it was stored. - * For testing only. - */ - private[catalog] def getTempTable(name: String): Option[LogicalPlan] = synchronized { - tempTables.get(formatTableName(name)) - } - // ---------------------------------------------------------------------------- // Partitions // ---------------------------------------------------------------------------- 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 012df629bbdef..84b77ad250b5c 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 @@ -201,16 +201,16 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable2 = Range(1, 20, 2, 10) catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) catalog.createTempView("tbl2", tempTable2, overrideIfExists = false) - assert(catalog.getTempTable("tbl1") == Option(tempTable1)) - assert(catalog.getTempTable("tbl2") == Option(tempTable2)) - assert(catalog.getTempTable("tbl3").isEmpty) + assert(catalog.getTempView("tbl1") == Option(tempTable1)) + assert(catalog.getTempView("tbl2") == Option(tempTable2)) + assert(catalog.getTempView("tbl3").isEmpty) // Temporary table already exists intercept[TempTableAlreadyExistsException] { catalog.createTempView("tbl1", tempTable1, overrideIfExists = false) } // Temporary table already exists but we override it catalog.createTempView("tbl1", tempTable2, overrideIfExists = true) - assert(catalog.getTempTable("tbl1") == Option(tempTable2)) + assert(catalog.getTempView("tbl1") == Option(tempTable2)) } test("drop table") { @@ -251,11 +251,11 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") - assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be dropped first sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) - assert(sessionCatalog.getTempTable("tbl1") == None) + assert(sessionCatalog.getTempView("tbl1") == None) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If temp table does not exist, the table in the current database should be dropped sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false) @@ -265,7 +265,7 @@ class SessionCatalogSuite extends SparkFunSuite { sessionCatalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false) sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false, purge = false) - assert(sessionCatalog.getTempTable("tbl1") == Some(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Some(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl2")) } @@ -303,17 +303,17 @@ class SessionCatalogSuite extends SparkFunSuite { val tempTable = Range(1, 10, 2, 10) sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false) sessionCatalog.setCurrentDatabase("db2") - assert(sessionCatalog.getTempTable("tbl1") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl1") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is not specified, temp table should be renamed first sessionCatalog.renameTable(TableIdentifier("tbl1"), "tbl3") - assert(sessionCatalog.getTempTable("tbl1").isEmpty) - assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl1").isEmpty) + assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2")) // If database is specified, temp tables are never renamed sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), "tbl4") - assert(sessionCatalog.getTempTable("tbl3") == Option(tempTable)) - assert(sessionCatalog.getTempTable("tbl4").isEmpty) + assert(sessionCatalog.getTempView("tbl3") == Option(tempTable)) + assert(sessionCatalog.getTempView("tbl4").isEmpty) assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4")) } 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 64d3422cb4b54..9e343b5d24986 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 @@ -361,7 +361,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { throw new AnalysisException("Cannot create hive serde table with saveAsTable API") } - val tableExists = df.sparkSession.sessionState.catalog.tableExists(tableIdent) + val sessionState = df.sparkSession.sessionState + val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = tableIdent.copy(database = Some(db)) + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + val tableExists = sessionState.catalog.tableExists(tableIdentWithDB) (tableExists, mode) match { case (true, SaveMode.Ignore) => @@ -387,7 +392,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { bucketSpec = getBucketSpec ) val cmd = CreateTable(tableDesc, mode, Some(df.logicalPlan)) - df.sparkSession.sessionState.executePlan(cmd).toRdd + sessionState.executePlan(cmd).toRdd } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index b1830e6cf3ea8..d8e20b09c1add 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -47,11 +47,15 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo assert(table.provider.isDefined) val sessionState = sparkSession.sessionState - if (sessionState.catalog.tableExists(table.identifier)) { + val db = table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = table.identifier.copy(database = Some(db)) + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + if (sessionState.catalog.tableExists(tableIdentWithDB)) { if (ignoreIfExists) { return Seq.empty[Row] } else { - throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.") + throw new AnalysisException(s"Table ${tableIdentWithDB.unquotedString} already exists.") } } @@ -128,9 +132,11 @@ case class CreateDataSourceTableAsSelectCommand( assert(table.provider.isDefined) assert(table.schema.isEmpty) - val tableName = table.identifier.unquotedString val provider = table.provider.get val sessionState = sparkSession.sessionState + val db = table.identifier.database.getOrElse(sessionState.catalog.getCurrentDatabase) + val tableIdentWithDB = table.identifier.copy(database = Some(db)) + val tableName = tableIdentWithDB.unquotedString val optionsWithPath = if (table.tableType == CatalogTableType.MANAGED) { table.storage.properties + ("path" -> sessionState.catalog.defaultTablePath(table.identifier)) @@ -140,7 +146,9 @@ case class CreateDataSourceTableAsSelectCommand( var createMetastoreTable = false var existingSchema = Option.empty[StructType] - if (sparkSession.sessionState.catalog.tableExists(table.identifier)) { + // Pass a table identifier with database part, so that `tableExists` won't check temp views + // unexpectedly. + if (sparkSession.sessionState.catalog.tableExists(tableIdentWithDB)) { // Check if we need to throw an exception or just return. mode match { case SaveMode.ErrorIfExists => @@ -165,7 +173,7 @@ case class CreateDataSourceTableAsSelectCommand( // inserting into (i.e. using the same compression). EliminateSubqueryAliases( - sessionState.catalog.lookupRelation(table.identifier)) match { + sessionState.catalog.lookupRelation(tableIdentWithDB)) match { case l @ LogicalRelation(_: InsertableRelation | _: HadoopFsRelation, _, _) => // check if the file formats match l.relation match { @@ -188,7 +196,7 @@ case class CreateDataSourceTableAsSelectCommand( throw new AnalysisException(s"Saving data in ${o.toString} is not supported.") } case SaveMode.Overwrite => - sparkSession.sql(s"DROP TABLE IF EXISTS $tableName") + sessionState.catalog.dropTable(tableIdentWithDB, ignoreIfNotExists = true, purge = false) // Need to create the table again. createMetastoreTable = true } @@ -230,7 +238,7 @@ case class CreateDataSourceTableAsSelectCommand( } // Refresh the cache of the table in the catalog. - sessionState.catalog.refreshTable(table.identifier) + sessionState.catalog.refreshTable(tableIdentWithDB) Seq.empty[Row] } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 78ad710a6262e..3fa62985624f0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalog.{Catalog, Column, Database, Function, Table} import org.apache.spark.sql.catalyst.{DefinedByConstructorParams, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, SessionCatalog} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, SubqueryAlias} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.types.StructType @@ -284,8 +284,10 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * @since 2.0.0 */ override def dropTempView(viewName: String): Unit = { - sparkSession.sharedState.cacheManager.uncacheQuery(sparkSession.table(viewName)) - sessionCatalog.dropTable(TableIdentifier(viewName), ignoreIfNotExists = true, purge = false) + sparkSession.sessionState.catalog.getTempView(viewName).foreach { tempView => + sparkSession.sharedState.cacheManager.uncacheQuery(Dataset.ofRows(sparkSession, tempView)) + sessionCatalog.dropTempView(viewName) + } } /** 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 3cc3b319f5a57..0ee8c959eeb4d 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 @@ -2667,4 +2667,15 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { }.limit(1).queryExecution.toRdd.count() assert(numRecordsRead.value === 10) } + + test("CREATE TABLE USING should not fail if a same-name temp view exists") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + sql("CREATE TABLE same_name(i int) USING json") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + assert(spark.table("default.same_name").collect().isEmpty) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala index 549fd63f7462e..3dc67ffafb048 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala @@ -329,6 +329,17 @@ class CatalogSuite assert(e.message.contains("Cannot create hive serde table with createExternalTable API")) } + test("dropTempView should not un-cache and drop metastore table if a same-name table exists") { + withTable("same_name") { + spark.range(10).write.saveAsTable("same_name") + sql("CACHE TABLE same_name") + assert(spark.catalog.isCached("default.same_name")) + spark.catalog.dropTempView("same_name") + assert(spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + assert(spark.catalog.isCached("default.same_name")) + } + } + // TODO: add tests for the rest of them } 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 63b0e4588e4a6..7368dad62859b 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 @@ -22,6 +22,7 @@ import java.io.File import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} import org.apache.spark.util.Utils @@ -464,4 +465,79 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkAnswer(df, spark.createDataset(expectedResult).toDF()) assert(df.schema === expectedSchema) } + + test("saveAsTable with mode Append should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Append should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Append).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode ErrorIfExists should not fail if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.ErrorIfExists).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not drop the temp view if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + assert(spark.sessionState.catalog.getTempView("same_name").isDefined) + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } + + test("saveAsTable with mode Overwrite should not fail if the table already exists " + + "and a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + sql("CREATE TABLE same_name(id LONG) USING parquet") + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Overwrite).saveAsTable("same_name") + checkAnswer(spark.table("same_name"), spark.range(10).toDF()) + checkAnswer(spark.table("default.same_name"), spark.range(20).toDF()) + } + } + } + + test("saveAsTable with mode Ignore should create the table if the table not exists " + + "but a same-name temp view exist") { + withTable("same_name") { + withTempView("same_name") { + spark.range(10).createTempView("same_name") + spark.range(20).write.mode(SaveMode.Ignore).saveAsTable("same_name") + assert( + spark.sessionState.catalog.tableExists(TableIdentifier("same_name", Some("default")))) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 0f331bae930f4..7143adf02b0e6 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -339,7 +339,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv }.getMessage assert( - message.contains("Table ctasJsonTable already exists."), + message.contains("Table default.ctasJsonTable already exists."), "We should complain that ctasJsonTable already exists") // The following statement should be fine if it has IF NOT EXISTS. @@ -515,7 +515,7 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv assert( intercept[AnalysisException] { sparkSession.catalog.createExternalTable("createdJsonTable", jsonFilePath.toString) - }.getMessage.contains("Table createdJsonTable already exists."), + }.getMessage.contains("Table default.createdJsonTable already exists."), "We should complain that createdJsonTable already exists") } @@ -907,7 +907,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv val e = intercept[AnalysisException] { createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendOrcToParquet") } - assert(e.getMessage.contains("The file format of the existing table appendOrcToParquet " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendOrcToParquet " + "is `org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat`. " + "It doesn't match the specified format `orc`")) } @@ -918,7 +919,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv createDF(10, 19).write.mode(SaveMode.Append).format("parquet") .saveAsTable("appendParquetToJson") } - assert(e.getMessage.contains("The file format of the existing table appendParquetToJson " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendParquetToJson " + "is `org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + "It doesn't match the specified format `parquet`")) } @@ -929,7 +931,8 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv createDF(10, 19).write.mode(SaveMode.Append).format("text") .saveAsTable("appendTextToJson") } - assert(e.getMessage.contains("The file format of the existing table appendTextToJson is " + + assert(e.getMessage.contains( + "The file format of the existing table default.appendTextToJson is " + "`org.apache.spark.sql.execution.datasources.json.JsonFileFormat`. " + "It doesn't match the specified format `text`")) } 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 27bb9676e9abf..22f13a494cd4c 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 @@ -337,9 +337,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - ErrorIfExists") { - Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - - withTempView("t") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") intercept[AnalysisException] { testDF.write.format(dataSourceName).mode(SaveMode.ErrorIfExists).saveAsTable("t") } @@ -347,9 +346,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes } test("saveAsTable()/load() - non-partitioned table - Ignore") { - Seq.empty[(Int, String)].toDF().createOrReplaceTempView("t") - - withTempView("t") { + withTable("t") { + sql("CREATE TABLE t(i INT) USING parquet") testDF.write.format(dataSourceName).mode(SaveMode.Ignore).saveAsTable("t") assert(spark.table("t").collect().isEmpty) } From 7d24523c17d7c3712af67104458a6450ad6a08a8 Mon Sep 17 00:00:00 2001 From: jiangxingbo Date: Sun, 18 Sep 2016 16:04:37 +0100 Subject: [PATCH 72/83] [SPARK-17506][SQL] Improve the check double values equality rule. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? In `ExpressionEvalHelper`, we check the equality between two double values by comparing whether the expected value is within the range [target - tolerance, target + tolerance], but this can cause a negative false when the compared numerics are very large. Before: ``` val1 = 1.6358558070241E306 val2 = 1.6358558070240974E306 ExpressionEvalHelper.compareResults(val1, val2) false ``` In fact, `val1` and `val2` are but with different precisions, we should tolerant this case by comparing with percentage range, eg.,expected is within range [target - target * tolerance_percentage, target + target * tolerance_percentage]. After: ``` val1 = 1.6358558070241E306 val2 = 1.6358558070240974E306 ExpressionEvalHelper.compareResults(val1, val2) true ``` ## How was this patch tested? Exsiting testcases. Author: jiangxingbo Closes #15059 from jiangxb1987/deq. --- .../ArithmeticExpressionSuite.scala | 8 ++--- .../expressions/ExpressionEvalHelper.scala | 29 +++++++++++++++++-- 2 files changed, 30 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 687387507e214..5c9824289b3cb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -170,11 +170,9 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Remainder(positiveLongLit, positiveLongLit), 0L) checkEvaluation(Remainder(negativeLongLit, negativeLongLit), 0L) - // TODO: the following lines would fail the test due to inconsistency result of interpret - // and codegen for remainder between giant values, seems like a numeric stability issue - // DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => - // checkConsistencyBetweenInterpretedAndCodegen(Remainder, tpe, tpe) - // } + DataTypeTestUtils.numericTypeWithoutDecimal.foreach { tpe => + checkConsistencyBetweenInterpretedAndCodegen(Remainder, tpe, tpe) + } } test("Abs") { 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 668543a28bd30..f0c149c02b9aa 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.scalacheck.Gen import org.scalactic.TripleEqualsSupport.Spread +import org.scalatest.exceptions.TestFailedException import org.scalatest.prop.GeneratorDrivenPropertyChecks import org.apache.spark.SparkFunSuite @@ -289,13 +290,37 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks { (result, expected) match { case (result: Array[Byte], expected: Array[Byte]) => java.util.Arrays.equals(result, expected) - case (result: Double, expected: Spread[Double @unchecked]) => - expected.asInstanceOf[Spread[Double]].isWithin(result) case (result: Double, expected: Double) if result.isNaN && expected.isNaN => true + case (result: Double, expected: Double) => + relativeErrorComparison(result, expected) case (result: Float, expected: Float) if result.isNaN && expected.isNaN => true case _ => result == expected } } + + /** + * Private helper function for comparing two values using relative tolerance. + * Note that if x or y is extremely close to zero, i.e., smaller than Double.MinPositiveValue, + * the relative tolerance is meaningless, so the exception will be raised to warn users. + * + * TODO: this duplicates functions in spark.ml.util.TestingUtils.relTol and + * spark.mllib.util.TestingUtils.relTol, they could be moved to common utils sub module for the + * whole spark project which does not depend on other modules. See more detail in discussion: + * https://github.com/apache/spark/pull/15059#issuecomment-246940444 + */ + private def relativeErrorComparison(x: Double, y: Double, eps: Double = 1E-8): Boolean = { + val absX = math.abs(x) + val absY = math.abs(y) + val diff = math.abs(x - y) + if (x == y) { + true + } else if (absX < Double.MinPositiveValue || absY < Double.MinPositiveValue) { + throw new TestFailedException( + s"$x or $y is extremely close to zero, so the relative tolerance is meaningless.", 0) + } else { + diff < eps * math.min(absX, absY) + } + } } From e460dcdd624df9cc604d820d113c70010e3fddc8 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 18 Sep 2016 16:22:31 +0100 Subject: [PATCH 73/83] [SPARK-17546][DEPLOY] start-* scripts should use hostname -f ## What changes were proposed in this pull request? Call `hostname -f` to get fully qualified host name ## How was this patch tested? Jenkins tests of course, but also verified output of command on OS X and Linux Author: Sean Owen Closes #15129 from srowen/SPARK-17546. --- sbin/start-master.sh | 2 +- sbin/start-mesos-dispatcher.sh | 2 +- sbin/start-slaves.sh | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 981cb15bc0006..d970fcc45e2c1 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -48,7 +48,7 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST=`hostname` + SPARK_MASTER_HOST=`hostname -f` fi if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then diff --git a/sbin/start-mesos-dispatcher.sh b/sbin/start-mesos-dispatcher.sh index 06a966d1c20b4..ef65fb9539146 100755 --- a/sbin/start-mesos-dispatcher.sh +++ b/sbin/start-mesos-dispatcher.sh @@ -34,7 +34,7 @@ if [ "$SPARK_MESOS_DISPATCHER_PORT" = "" ]; then fi if [ "$SPARK_MESOS_DISPATCHER_HOST" = "" ]; then - SPARK_MESOS_DISPATCHER_HOST=`hostname` + SPARK_MESOS_DISPATCHER_HOST=`hostname -f` fi if [ "$SPARK_MESOS_DISPATCHER_NUM" = "" ]; then diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 0fa1605489704..7d8871251f81b 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -32,7 +32,7 @@ if [ "$SPARK_MASTER_PORT" = "" ]; then fi if [ "$SPARK_MASTER_HOST" = "" ]; then - SPARK_MASTER_HOST="`hostname`" + SPARK_MASTER_HOST="`hostname -f`" fi # Launch the slaves From 032335eb4809221deee8e780f34749ac0e5b1482 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 18 Sep 2016 19:18:49 +0100 Subject: [PATCH 74/83] [SPARK-17586][BUILD] Do not call static member via instance reference ## What changes were proposed in this pull request? This PR fixes a warning message as below: ``` [WARNING] .../UnsafeInMemorySorter.java:284: warning: [static] static method should be qualified by type name, TaskMemoryManager, instead of by an expression [WARNING] currentPageNumber = memoryManager.decodePageNumber(recordPointer) ``` by referencing the static member via class not instance reference. ## How was this patch tested? Existing tests should cover this - Jenkins tests. Author: hyukjinkwon Closes #15141 from HyukjinKwon/SPARK-17586. --- .../spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index be382955c0d42..3b1ece4373f49 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -281,7 +281,7 @@ public boolean hasNext() { public void loadNext() { // This pointer points to a 4-byte record length, followed by the record's bytes final long recordPointer = array.get(offset + position); - currentPageNumber = memoryManager.decodePageNumber(recordPointer); + currentPageNumber = TaskMemoryManager.decodePageNumber(recordPointer); baseObject = memoryManager.getPage(recordPointer); baseOffset = memoryManager.getOffsetInPage(recordPointer) + 4; // Skip over record length recordLength = Platform.getInt(baseObject, baseOffset - 4); From 5930e0beb966ffb0a706e6ce03b325d0c0602157 Mon Sep 17 00:00:00 2001 From: Liwei Lin Date: Sun, 18 Sep 2016 19:25:58 +0100 Subject: [PATCH 75/83] [SPARK-16462][SPARK-16460][SPARK-15144][SQL] Make CSV cast null values properly ## Problem CSV in Spark 2.0.0: - does not read null values back correctly for certain data types such as `Boolean`, `TimestampType`, `DateType` -- this is a regression comparing to 1.6; - does not read empty values (specified by `options.nullValue`) as `null`s for `StringType` -- this is compatible with 1.6 but leads to problems like SPARK-16903. ## What changes were proposed in this pull request? This patch makes changes to read all empty values back as `null`s. ## How was this patch tested? New test cases. Author: Liwei Lin Closes #14118 from lw-lin/csv-cast-null. --- python/pyspark/sql/readwriter.py | 3 +- python/pyspark/sql/streaming.py | 3 +- .../apache/spark/sql/DataFrameReader.scala | 3 +- .../datasources/csv/CSVInferSchema.scala | 108 ++++++++---------- .../sql/streaming/DataStreamReader.scala | 3 +- .../execution/datasources/csv/CSVSuite.scala | 2 +- .../datasources/csv/CSVTypeCastSuite.scala | 54 +++++---- 7 files changed, 93 insertions(+), 83 deletions(-) diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3d79e0ccccee4..a6860efa89b9e 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -329,7 +329,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non being read should be skipped. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. + the default value, empty string. Since 2.0.1, this ``nullValue`` param + applies to all supported types including the string type. :param nanValue: sets the string representation of a non-number value. If None is set, it uses the default value, ``NaN``. :param positiveInf: sets the string representation of a positive infinity value. If None diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 67375f6b5f942..01364517edd0e 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -497,7 +497,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non being read should be skipped. If None is set, it uses the default value, ``false``. :param nullValue: sets the string representation of a null value. If None is set, it uses - the default value, empty string. + the default value, empty string. Since 2.0.1, this ``nullValue`` param + applies to all supported types including the string type. :param nanValue: sets the string representation of a non-number value. If None is set, it uses the default value, ``NaN``. :param positiveInf: sets the string representation of a positive infinity value. If None 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 d29d90ce40453..30f39c70fe0bf 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 @@ -376,7 +376,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * from values being read should be skipped. *
    • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
    • - *
    • `nullValue` (default empty string): sets the string representation of a null value.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value. Since + * 2.0.1, this applies to all supported types including the string type.
    • *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
    • diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala index 1ca6eff1b8c2e..3ab775c909238 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala @@ -232,66 +232,58 @@ private[csv] object CSVTypeCast { nullable: Boolean = true, options: CSVOptions = CSVOptions()): Any = { - castType match { - case _: ByteType => if (datum == options.nullValue && nullable) null else datum.toByte - case _: ShortType => if (datum == options.nullValue && nullable) null else datum.toShort - case _: IntegerType => if (datum == options.nullValue && nullable) null else datum.toInt - case _: LongType => if (datum == options.nullValue && nullable) null else datum.toLong - case _: FloatType => - if (datum == options.nullValue && nullable) { - null - } else if (datum == options.nanValue) { - Float.NaN - } else if (datum == options.negativeInf) { - Float.NegativeInfinity - } else if (datum == options.positiveInf) { - Float.PositiveInfinity - } else { - Try(datum.toFloat) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) - } - case _: DoubleType => - if (datum == options.nullValue && nullable) { - null - } else if (datum == options.nanValue) { - Double.NaN - } else if (datum == options.negativeInf) { - Double.NegativeInfinity - } else if (datum == options.positiveInf) { - Double.PositiveInfinity - } else { - Try(datum.toDouble) - .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) - } - case _: BooleanType => datum.toBoolean - case dt: DecimalType => - if (datum == options.nullValue && nullable) { - null - } else { - val value = new BigDecimal(datum.replaceAll(",", "")) - Decimal(value, dt.precision, dt.scale) - } - case _: TimestampType => - // 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 + if (nullable && datum == options.nullValue) { + null + } else { + castType match { + case _: ByteType => datum.toByte + case _: ShortType => datum.toShort + case _: IntegerType => datum.toInt + case _: LongType => datum.toLong + case _: FloatType => + datum match { + case options.nanValue => Float.NaN + case options.negativeInf => Float.NegativeInfinity + case options.positiveInf => Float.PositiveInfinity + case _ => + Try(datum.toFloat) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).floatValue()) } - case _: DateType => - // 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) + case _: DoubleType => + datum match { + case options.nanValue => Double.NaN + case options.negativeInf => Double.NegativeInfinity + case options.positiveInf => Double.PositiveInfinity + case _ => + Try(datum.toDouble) + .getOrElse(NumberFormat.getInstance(Locale.getDefault).parse(datum).doubleValue()) } - case _: StringType => UTF8String.fromString(datum) - case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options) - case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") + case _: BooleanType => datum.toBoolean + case dt: DecimalType => + val value = new BigDecimal(datum.replaceAll(",", "")) + Decimal(value, dt.precision, dt.scale) + case _: TimestampType => + // 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 + } + case _: DateType => + // 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) + } + case _: StringType => UTF8String.fromString(datum) + case udt: UserDefinedType[_] => castTo(datum, udt.sqlType, nullable, options) + case _ => throw new RuntimeException(s"Unsupported type: ${castType.typeName}") + } } } 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 c25f71af7362a..9d174051bc923 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 @@ -232,7 +232,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * from values being read should be skipped. *
    • `ignoreTrailingWhiteSpace` (default `false`): defines whether or not trailing * whitespaces from values being read should be skipped.
    • - *
    • `nullValue` (default empty string): sets the string representation of a null value.
    • + *
    • `nullValue` (default empty string): sets the string representation of a null value. Since + * 2.0.1, this applies to all supported types including the string type.
    • *
    • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
    • *
    • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
    • 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 1930862118e9b..29aac9def6924 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 @@ -554,7 +554,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils { verifyCars(cars, withHeader = true, checkValues = false) val results = cars.collect() - assert(results(0).toSeq === Array(2012, "Tesla", "S", "null", "null")) + assert(results(0).toSeq === Array(2012, "Tesla", "S", null, null)) assert(results(2).toSeq === Array(null, "Chevy", "Volt", null, null)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala index 3ce643e667ce4..dae92f626c225 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVTypeCastSuite.scala @@ -68,16 +68,46 @@ class CSVTypeCastSuite extends SparkFunSuite { } test("Nullable types are handled") { - assert(CSVTypeCast.castTo("", IntegerType, nullable = true, CSVOptions()) == null) + assertNull( + CSVTypeCast.castTo("-", ByteType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", ShortType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", IntegerType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", LongType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", FloatType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DoubleType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", BooleanType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DecimalType.DoubleDecimal, true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", TimestampType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", DateType, nullable = true, CSVOptions("nullValue", "-"))) + assertNull( + CSVTypeCast.castTo("-", StringType, nullable = true, CSVOptions("nullValue", "-"))) } - test("String type should always return the same as the input") { + test("String type should also respect `nullValue`") { + assertNull( + CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions())) assert( - CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions()) == + CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions()) == UTF8String.fromString("")) + assert( - CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions()) == + CSVTypeCast.castTo("", StringType, nullable = true, CSVOptions("nullValue", "null")) == + UTF8String.fromString("")) + assert( + CSVTypeCast.castTo("", StringType, nullable = false, CSVOptions("nullValue", "null")) == UTF8String.fromString("")) + + assertNull( + CSVTypeCast.castTo(null, StringType, nullable = true, CSVOptions("nullValue", "null"))) } test("Throws exception for empty string with non null type") { @@ -170,20 +200,4 @@ class CSVTypeCastSuite extends SparkFunSuite { assert(doubleVal2 == Double.PositiveInfinity) } - test("Type-specific null values are used for casting") { - assertNull( - CSVTypeCast.castTo("-", ByteType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", ShortType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", IntegerType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", LongType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", FloatType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", DoubleType, nullable = true, CSVOptions("nullValue", "-"))) - assertNull( - CSVTypeCast.castTo("-", DecimalType.DoubleDecimal, true, CSVOptions("nullValue", "-"))) - } } From 98dfd17b2e9b936cb807ad238abb79287c4344e1 Mon Sep 17 00:00:00 2001 From: petermaxlee Date: Sun, 18 Sep 2016 15:22:01 -0700 Subject: [PATCH 76/83] [SPARK-17571][SQL] AssertOnQuery.condition should always return Boolean value ## What changes were proposed in this pull request? AssertOnQuery has two apply constructor: one that accepts a closure that returns boolean, and another that accepts a closure that returns Unit. This is actually very confusing because developers could mistakenly think that AssertOnQuery always require a boolean return type and verifies the return result, when indeed the value of the last statement is ignored in one of the constructors. This pull request makes the two constructor consistent and always require boolean value. It will overall make the test suites more robust against developer errors. As an evidence for the confusing behavior, this change also identified a bug with an existing test case due to file system time granularity. This pull request fixes that test case as well. ## How was this patch tested? This is a test only change. Author: petermaxlee Closes #15127 from petermaxlee/SPARK-17571. --- .../apache/spark/sql/streaming/FileStreamSourceSuite.scala | 7 +++++-- .../scala/org/apache/spark/sql/streaming/StreamTest.scala | 4 ++-- .../spark/sql/streaming/StreamingQueryListenerSuite.scala | 3 +++ 3 files changed, 10 insertions(+), 4 deletions(-) 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 886f7be59db93..a02a36c00499c 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 @@ -354,7 +354,9 @@ class FileStreamSourceSuite extends FileStreamSourceTest { CheckAnswer("a", "b"), // SLeeps longer than 5ms (maxFileAge) - AssertOnQuery { _ => Thread.sleep(10); true }, + // Unfortunately since a lot of file system does not have modification time granularity + // finer grained than 1 sec, we need to use 1 sec here. + AssertOnQuery { _ => Thread.sleep(1000); true }, AddTextFileData("c\nd", src, tmp), CheckAnswer("a", "b", "c", "d"), @@ -363,7 +365,8 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val source = streamExecution.logicalPlan.collect { case e: StreamingExecutionRelation => e.source.asInstanceOf[FileStreamSource] }.head - source.seenFiles.size == 1 + assert(source.seenFiles.size == 1) + true } ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index af2b58116b2aa..6c5b170d9c7c3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -188,8 +188,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with Timeouts { new AssertOnQuery(condition, message) } - def apply(message: String)(condition: StreamExecution => Unit): AssertOnQuery = { - new AssertOnQuery(s => { condition(s); true }, message) + def apply(message: String)(condition: StreamExecution => Boolean): AssertOnQuery = { + new AssertOnQuery(condition, message) } } 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 77602e8167fa3..831543a47420a 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 @@ -66,6 +66,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // No progress events or termination events assert(listener.progressStatuses.isEmpty) assert(listener.terminationStatus === null) + true }, AddDataMemory(input, Seq(1, 2, 3)), CheckAnswer(1, 2, 3), @@ -84,6 +85,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { // No termination events assert(listener.terminationStatus === null) } + true }, StopStream, AssertOnQuery("Incorrect query status in onQueryTerminated") { query => @@ -97,6 +99,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { assert(listener.terminationException === None) } listener.checkAsyncErrors() + true } ) } From 3b18214a4662d964d3740da284ac21fbfbb9b844 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 19 Sep 2016 09:38:25 +0100 Subject: [PATCH 77/83] [SPARK-17297][DOCS] Clarify window/slide duration as absolute time, not relative to a calendar ## What changes were proposed in this pull request? Clarify that slide and window duration are absolute, and not relative to a calendar. ## How was this patch tested? Doc build (no functional change) Author: Sean Owen Closes #15142 from srowen/SPARK-17297. --- R/pkg/R/functions.R | 8 ++++++-- .../scala/org/apache/spark/sql/functions.scala | 15 +++++++++++---- 2 files changed, 17 insertions(+), 6 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index ceedbe76711b1..4d94b4cd05d44 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2713,11 +2713,15 @@ setMethod("from_unixtime", signature(x = "Column"), #' @param x a time Column. Must be of TimestampType. #' @param windowDuration a string specifying the width of the window, e.g. '1 second', #' '1 day 12 hours', '2 minutes'. Valid interval strings are 'week', -#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. +#' 'day', 'hour', 'minute', 'second', 'millisecond', 'microsecond'. Note that +#' the duration is a fixed length of time, and does not vary over time +#' according to a calendar. For example, '1 day' always means 86,400,000 +#' milliseconds, not a calendar day. #' @param slideDuration a string specifying the sliding interval of the window. Same format as #' \code{windowDuration}. A new window will be generated every #' \code{slideDuration}. Must be less than or equal to -#' the \code{windowDuration}. +#' the \code{windowDuration}. This duration is likewise absolute, and does not +#' vary according to a calendar. #' @param startTime the offset with respect to 1970-01-01 00:00:00 UTC with which to start #' window intervals. For example, in order to have hourly tumbling windows #' that start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide 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 18e736ab69861..960c87f60e624 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 @@ -2606,12 +2606,15 @@ object functions { * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for - * valid duration identifiers. + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration - * identifiers. + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. * @param startTime The offset with respect to 1970-01-01 00:00:00 UTC with which to start * window intervals. For example, in order to have hourly tumbling windows that * start 15 minutes past the hour, e.g. 12:15-13:15, 13:15-14:15... provide @@ -2660,11 +2663,15 @@ object functions { * The time column must be of TimestampType. * @param windowDuration A string specifying the width of the window, e.g. `10 minutes`, * `1 second`. Check [[org.apache.spark.unsafe.types.CalendarInterval]] for - * valid duration identifiers. + * valid duration identifiers. Note that the duration is a fixed length of + * time, and does not vary over time according to a calendar. For example, + * `1 day` always means 86,400,000 milliseconds, not a calendar day. * @param slideDuration A string specifying the sliding interval of the window, e.g. `1 minute`. * A new window will be generated every `slideDuration`. Must be less than * or equal to the `windowDuration`. Check - * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration. + * [[org.apache.spark.unsafe.types.CalendarInterval]] for valid duration + * identifiers. This duration is likewise absolute, and does not vary + * according to a calendar. * * @group datetime_funcs * @since 2.0.0 From 2b7ebffde99081e74c5b0a7762dc433be36bbdd6 Mon Sep 17 00:00:00 2001 From: sureshthalamati Date: Mon, 19 Sep 2016 09:56:16 -0700 Subject: [PATCH 78/83] [SPARK-17473][SQL] fixing docker integration tests error due to different versions of jars. ## What changes were proposed in this pull request? Docker tests are using older version of jersey jars (1.19), which was used in older releases of spark. In 2.0 releases Spark was upgraded to use 2.x verison of Jersey. After upgrade to new versions, docker tests are failing with AbstractMethodError. Now that spark is upgraded to 2.x jersey version, using of shaded docker jars may not be required any more. Removed the exclusions/overrides of jersey related classes from pom file, and changed the docker-client to use regular jar instead of shaded one. ## How was this patch tested? Tested using existing docker-integration-tests Author: sureshthalamati Closes #15114 from sureshthalamati/docker_testfix-spark-17473. --- external/docker-integration-tests/pom.xml | 68 ----------------------- pom.xml | 1 - 2 files changed, 69 deletions(-) diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 7417199e7693d..57d553b75b872 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -49,38 +49,7 @@ com.spotify docker-client - shaded test - - - - com.fasterxml.jackson.jaxrs - jackson-jaxrs-json-provider - - - com.fasterxml.jackson.datatype - jackson-datatype-guava - - - com.fasterxml.jackson.core - jackson-databind - - - org.glassfish.jersey.core - jersey-client - - - org.glassfish.jersey.connectors - jersey-apache-connector - - - org.glassfish.jersey.media - jersey-media-json-jackson - - org.apache.httpcomponents @@ -152,43 +121,6 @@ test - - - com.sun.jersey - jersey-server - 1.19 - test - - - com.sun.jersey - jersey-core - 1.19 - test - - - com.sun.jersey - jersey-servlet - 1.19 - test - - - com.sun.jersey - jersey-json - 1.19 - test - - - stax - stax-api - - - - -