From 8460b09054c9aa488df4fd1e7461a75b7a646e4b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 25 Apr 2017 16:28:26 -0700 Subject: [PATCH 001/151] Preparing development version 2.1.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/java8-tests/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- launcher/pom.xml | 2 +- mesos/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- yarn/pom.xml | 2 +- 39 files changed, 40 insertions(+), 40 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1ceda7ba024c0..2d461ca68920b 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.1.1 +Version: 2.1.2 Title: R Frontend for Apache Spark Description: The SparkR package provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index cc290c03c9dfa..6e092ef8928b6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index ccf4b27b34a69..77a4b64e8da9d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 98a23249cc192..1a2d85a2ead6b 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index dc1ad144dee6f..7a57e8964f6fc 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 250b69699332e..ff2d5c52730bf 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 0697ed625b261..b9bf0342eb600 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index cedae5fc279cc..f8a0e577777ee 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 28c4f95afe198..bad3655452fb4 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 75f48a59ab152..e21d011c4f83f 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.1.1 -SPARK_VERSION_SHORT: 2.1.1 +SPARK_VERSION: 2.1.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.1.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.7" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 72ee896f76238..8fa731fb340a2 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ac407dd48beb0..2cf0b41ee3544 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 92992e2f7081c..6ea318bf4af6e 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 7e0423a44b141..de3d17e9b9c05 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e1b86cec49c43..9361fdac11c5b 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/java8-tests/pom.xml b/external/java8-tests/pom.xml index 8b0583a861e43..f73e4f0aabc29 100644 --- a/external/java8-tests/pom.xml +++ b/external/java8-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 1ca601e765a75..66a679661f1d3 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 7ae63a5fa5654..c84c0408f483a 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 7a84764798244..961b80df50c5a 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 9bf41c5cfc2af..e56ed102ac89a 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 940112f641b06..e260e434f8dd3 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index e3305e91591b4..72e14f58e38f9 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 7610fad9f29e0..182f963cdd03c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 2fd4fd53d1aac..d6ba472a1fc99 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index ac6692194a79b..87e34b8a4b00e 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 3917251515d37..db4b15b10499e 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/mesos/pom.xml b/mesos/pom.xml index 6d84d45f3be8f..262316a193cac 100644 --- a/mesos/pom.xml +++ b/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 01a4b86121ebe..dae5b86d5fcb8 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 44f189cb8c063..be87ad2d1994b 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a985cf011de4b..a66156c9050a2 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 96b5e44bb320a..4447e3d9c7616 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.1.1" +__version__ = "2.1.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 12142c89db7f9..2cefaa191afdc 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 53d961d70038b..4b4a8eb3815e1 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c11710f4dfd6e..a03a9593e8520 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index f7ea320c74ae7..1abc0a253098c 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index fb61f1495df01..b62f800277cee 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index ddad02f2bffed..644fc50bf507b 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 193c0c5881715..11b58afdcac7a 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index 1933a0ebccf5d..e21df4ec1dc53 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.1.1 + 2.1.2-SNAPSHOT ../pom.xml From 6696ad0e8ce196a27e2908108f6e7eb7661affc4 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Wed, 26 Apr 2017 11:39:10 +0800 Subject: [PATCH 002/151] [SPARK-20439][SQL][BACKPORT-2.1] Fix Catalog API listTables and getTable when failed to fetch table metadata ### What changes were proposed in this pull request? This PR is to backport https://github.com/apache/spark/pull/17730 to Spark 2.1 --- -- `spark.catalog.listTables` and `spark.catalog.getTable` does not work if we are unable to retrieve table metadata due to any reason (e.g., table serde class is not accessible or the table type is not accepted by Spark SQL). After this PR, the APIs still return the corresponding Table without the description and tableType) ### How was this patch tested? Added a test case Author: Xiao Li Closes #17760 from gatorsmile/backport-17730. --- .../spark/sql/internal/CatalogImpl.scala | 28 +++++++++++++++---- .../sql/hive/execution/HiveDDLSuite.scala | 8 ++++++ 2 files changed, 31 insertions(+), 5 deletions(-) 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 9d0b2141d453f..c3c3513090730 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.internal import scala.collection.JavaConverters._ import scala.reflect.runtime.universe.TypeTag +import scala.util.control.NonFatal import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ @@ -99,14 +100,27 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { CatalogImpl.makeDataset(tables, sparkSession) } + /** + * Returns a Table for the given table/view or temporary view. + * + * Note that this function requires the table already exists in the Catalog. + * + * If the table metadata retrieval failed due to any reason (e.g., table serde class + * is not accessible or the table type is not accepted by Spark SQL), this function + * still returns the corresponding Table without the description and tableType) + */ private def makeTable(tableIdent: TableIdentifier): Table = { - val metadata = sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent) + val metadata = try { + Some(sessionCatalog.getTempViewOrPermanentTableMetadata(tableIdent)) + } catch { + case NonFatal(_) => None + } val isTemp = sessionCatalog.isTemporaryTable(tableIdent) new Table( name = tableIdent.table, - database = metadata.identifier.database.orNull, - description = metadata.comment.orNull, - tableType = if (isTemp) "TEMPORARY" else metadata.tableType.name, + database = metadata.map(_.identifier.database).getOrElse(tableIdent.database).orNull, + description = metadata.map(_.comment.orNull).orNull, + tableType = if (isTemp) "TEMPORARY" else metadata.map(_.tableType.name).orNull, isTemporary = isTemp) } @@ -197,7 +211,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { * `AnalysisException` when no `Table` can be found. */ override def getTable(dbName: String, tableName: String): Table = { - makeTable(TableIdentifier(tableName, Option(dbName))) + if (tableExists(dbName, tableName)) { + makeTable(TableIdentifier(tableName, Option(dbName))) + } else { + throw new AnalysisException(s"Table or view '$tableName' not found in database '$dbName'") + } } /** 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 3b9437da372c2..037ab470f3032 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 @@ -1060,6 +1060,14 @@ class HiveDDLSuite s"CREATE INDEX $indexName ON TABLE $tabName (a) AS 'COMPACT' WITH DEFERRED REBUILD") val indexTabName = spark.sessionState.catalog.listTables("default", s"*$indexName*").head.table + + // Even if index tables exist, listTables and getTable APIs should still work + checkAnswer( + spark.catalog.listTables().toDF(), + Row(indexTabName, "default", null, null, false) :: + Row(tabName, "default", null, "MANAGED", false) :: Nil) + assert(spark.catalog.getTable("default", indexTabName).name === indexTabName) + intercept[TableAlreadyExistsException] { sql(s"CREATE TABLE $indexTabName(b int)") } From 5131b0a963699b6910949ae2361fa740dafdb678 Mon Sep 17 00:00:00 2001 From: Bill Chambers Date: Fri, 28 Apr 2017 10:18:31 -0700 Subject: [PATCH 003/151] [SPARK-20496][SS] Bug in KafkaWriter Looks at Unanalyzed Plans ## What changes were proposed in this pull request? We didn't enforce analyzed plans in Spark 2.1 when writing out to Kafka. ## How was this patch tested? New unit test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Bill Chambers Closes #17804 from anabranch/SPARK-20496-2. (cherry picked from commit 733b81b835f952ab96723c749461d6afc0c71974) Signed-off-by: Burak Yavuz --- .../apache/spark/sql/kafka010/KafkaWriter.scala | 4 ++-- .../spark/sql/kafka010/KafkaSinkSuite.scala | 16 ++++++++++++++++ 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index a637d52c933a3..61936e32fd837 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -47,7 +47,7 @@ private[kafka010] object KafkaWriter extends Logging { queryExecution: QueryExecution, kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { - val schema = queryExecution.logical.output + val schema = queryExecution.analyzed.output schema.find(_.name == TOPIC_ATTRIBUTE_NAME).getOrElse( if (topic == None) { throw new AnalysisException(s"topic option required when no " + @@ -84,7 +84,7 @@ private[kafka010] object KafkaWriter extends Logging { queryExecution: QueryExecution, kafkaParameters: ju.Map[String, Object], topic: Option[String] = None): Unit = { - val schema = queryExecution.logical.output + val schema = queryExecution.analyzed.output validateQuery(queryExecution, kafkaParameters, topic) SQLExecution.withNewExecutionId(sparkSession, queryExecution) { queryExecution.toRdd.foreachPartition { iter => diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala index 490535623cb36..1e7f4f2c144b9 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSinkSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.functions._ import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{BinaryType, DataType} @@ -107,6 +108,21 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext { s"save mode overwrite not allowed for kafka")) } + test("SPARK-20496: batch - enforce analyzed plans") { + val inputEvents = + spark.range(1, 1000) + .select(to_json(struct("*")) as 'value) + + val topic = newTopic() + testUtils.createTopic(topic) + // used to throw UnresolvedException + inputEvents.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + } + test("streaming - write to kafka with topic field") { val input = MemoryStream[String] val topic = newTopic() From 868b4a1aa954d03d9ba29b9d7743eeefeece682c Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 1 May 2017 10:25:29 -0700 Subject: [PATCH 004/151] [SPARK-20517][UI] Fix broken history UI download link The download link in history server UI is concatenated with: ``` Download ``` Here `num` field represents number of attempts, this is not equal to REST APIs. In the REST API, if attempt id is not existed the URL should be `api/v1/applications//logs`, otherwise the URL should be `api/v1/applications///logs`. Using `` to represent `` will lead to the issue of "no such app". Manual verification. CC ajbozarth can you please review this change, since you add this feature before? Thanks! Author: jerryshao Closes #17795 from jerryshao/SPARK-20517. (cherry picked from commit ab30590f448d05fc1864c54a59b6815bdeef8fc7) Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/ui/static/historypage-template.html | 2 +- .../main/resources/org/apache/spark/ui/static/historypage.js | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html index 42e2d9abdeb5e..6ba3b092dc658 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage-template.html @@ -77,7 +77,7 @@ {{duration}} {{sparkUser}} {{lastUpdated}} - Download + Download {{/attempts}} {{/applications}} diff --git a/core/src/main/resources/org/apache/spark/ui/static/historypage.js b/core/src/main/resources/org/apache/spark/ui/static/historypage.js index 8fd91865b0429..d095a2c635cf1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/historypage.js +++ b/core/src/main/resources/org/apache/spark/ui/static/historypage.js @@ -114,6 +114,9 @@ $(document).ready(function() { attempt["startTime"] = formatDate(attempt["startTime"]); attempt["endTime"] = formatDate(attempt["endTime"]); attempt["lastUpdated"] = formatDate(attempt["lastUpdated"]); + attempt["log"] = uiRoot + "/api/v1/applications/" + id + "/" + + (attempt.hasOwnProperty("attemptId") ? attempt["attemptId"] + "/" : "") + "logs"; + var app_clone = {"id" : id, "name" : name, "num" : num, "attempts" : [attempt]}; array.push(app_clone); } From 5915588a92e6da955cd42621c3701031547ebf9e Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 1 May 2017 14:48:02 -0700 Subject: [PATCH 005/151] [SPARK-20540][CORE] Fix unstable executor requests. There are two problems fixed in this commit. First, the ExecutorAllocationManager sets a timeout to avoid requesting executors too often. However, the timeout is always updated based on its value and a timeout, not the current time. If the call is delayed by locking for more than the ongoing scheduler timeout, the manager will request more executors on every run. This seems to be the main cause of SPARK-20540. The second problem is that the total number of requested executors is not tracked by the CoarseGrainedSchedulerBackend. Instead, it calculates the value based on the current status of 3 variables: the number of known executors, the number of executors that have been killed, and the number of pending executors. But, the number of pending executors is never less than 0, even though there may be more known than requested. When executors are killed and not replaced, this can cause the request sent to YARN to be incorrect because there were too many executors due to the scheduler's state being slightly out of date. This is fixed by tracking the currently requested size explicitly. ## How was this patch tested? Existing tests. Author: Ryan Blue Closes #17813 from rdblue/SPARK-20540-fix-dynamic-allocation. (cherry picked from commit 2b2dd08e975dd7fbf261436aa877f1d7497ed31f) Signed-off-by: Marcelo Vanzin --- .../spark/ExecutorAllocationManager.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 32 ++++++++++++++++--- .../StandaloneDynamicAllocationSuite.scala | 6 ++-- 3 files changed, 33 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 1366251d0618f..f054a78d57883 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -331,7 +331,7 @@ private[spark] class ExecutorAllocationManager( val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") - addTime += sustainedSchedulerBacklogTimeoutS * 1000 + addTime = now + (sustainedSchedulerBacklogTimeoutS * 1000) delta } else { 0 diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 3452487e72e88..6fa239e4fee6b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -69,6 +69,10 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // `CoarseGrainedSchedulerBackend.this`. private val executorDataMap = new HashMap[String, ExecutorData] + // Number of executors requested by the cluster manager, [[ExecutorAllocationManager]] + @GuardedBy("CoarseGrainedSchedulerBackend.this") + private var requestedTotalExecutors = 0 + // Number of executors requested from the cluster manager that have not registered yet @GuardedBy("CoarseGrainedSchedulerBackend.this") private var numPendingExecutors = 0 @@ -390,6 +394,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp * */ protected def reset(): Unit = { val executors = synchronized { + requestedTotalExecutors = 0 numPendingExecutors = 0 executorsPendingToRemove.clear() Set() ++ executorDataMap.keys @@ -463,12 +468,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp logInfo(s"Requesting $numAdditionalExecutors additional executor(s) from the cluster manager") val response = synchronized { + requestedTotalExecutors += numAdditionalExecutors numPendingExecutors += numAdditionalExecutors logDebug(s"Number of pending executors is now $numPendingExecutors") + if (requestedTotalExecutors != + (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { + logDebug( + s"""requestExecutors($numAdditionalExecutors): Executor request doesn't match: + |requestedTotalExecutors = $requestedTotalExecutors + |numExistingExecutors = $numExistingExecutors + |numPendingExecutors = $numPendingExecutors + |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + } // Account for executors pending to be added or removed - doRequestTotalExecutors( - numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) + doRequestTotalExecutors(requestedTotalExecutors) } defaultAskTimeout.awaitResult(response) @@ -500,6 +514,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } val response = synchronized { + this.requestedTotalExecutors = numExecutors this.localityAwareTasks = localityAwareTasks this.hostToLocalTaskCount = hostToLocalTaskCount @@ -575,8 +590,17 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // take into account executors that are pending to be added or removed. val adjustTotalExecutors = if (!replace) { - doRequestTotalExecutors( - numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size) + requestedTotalExecutors = math.max(requestedTotalExecutors - executorsToKill.size, 0) + if (requestedTotalExecutors != + (numExistingExecutors + numPendingExecutors - executorsPendingToRemove.size)) { + logDebug( + s"""killExecutors($executorIds, $replace, $force): Executor counts do not match: + |requestedTotalExecutors = $requestedTotalExecutors + |numExistingExecutors = $numExistingExecutors + |numPendingExecutors = $numPendingExecutors + |executorsPendingToRemove = ${executorsPendingToRemove.size}""".stripMargin) + } + doRequestTotalExecutors(requestedTotalExecutors) } else { numPendingExecutors += knownExecutors.size Future.successful(true) diff --git a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala index 05dad7a4b86ad..281484c6170ea 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -354,12 +354,13 @@ class StandaloneDynamicAllocationSuite test("kill the same executor twice (SPARK-9795)") { sc = new SparkContext(appConf) val appId = sc.applicationId + sc.requestExecutors(2) eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() assert(apps.size === 1) assert(apps.head.id === appId) assert(apps.head.executors.size === 2) - assert(apps.head.getExecutorLimit === Int.MaxValue) + assert(apps.head.getExecutorLimit === 2) } // sync executors between the Master and the driver, needed because // the driver refuses to kill executors it does not know about @@ -378,12 +379,13 @@ class StandaloneDynamicAllocationSuite test("the pending replacement executors should not be lost (SPARK-10515)") { sc = new SparkContext(appConf) val appId = sc.applicationId + sc.requestExecutors(2) eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() assert(apps.size === 1) assert(apps.head.id === appId) assert(apps.head.executors.size === 2) - assert(apps.head.getExecutorLimit === Int.MaxValue) + assert(apps.head.getExecutorLimit === 2) } // sync executors between the Master and the driver, needed because // the driver refuses to kill executors it does not know about From d10b0f65459a66c15418ed79285f41d947198164 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 3 May 2017 10:08:46 +0800 Subject: [PATCH 006/151] [SPARK-20558][CORE] clear InheritableThreadLocal variables in SparkContext when stopping it ## What changes were proposed in this pull request? To better understand this problem, let's take a look at an example first: ``` object Main { def main(args: Array[String]): Unit = { var t = new Test new Thread(new Runnable { override def run() = {} }).start() println("first thread finished") t.a = null t = new Test new Thread(new Runnable { override def run() = {} }).start() } } class Test { var a = new InheritableThreadLocal[String] { override protected def childValue(parent: String): String = { println("parent value is: " + parent) parent } } a.set("hello") } ``` The result is: ``` parent value is: hello first thread finished parent value is: hello parent value is: hello ``` Once an `InheritableThreadLocal` has been set value, child threads will inherit its value as long as it has not been GCed, so setting the variable which holds the `InheritableThreadLocal` to `null` doesn't work as we expected. In `SparkContext`, we have an `InheritableThreadLocal` for local properties, we should clear it when stopping `SparkContext`, or all the future child threads will still inherit it and copy the properties and waste memory. This is the root cause of https://issues.apache.org/jira/browse/SPARK-20548 , which creates/stops `SparkContext` many times and finally have a lot of `InheritableThreadLocal` alive, and cause OOM when starting new threads in the internal thread pools. ## How was this patch tested? N/A Author: Wenchen Fan Closes #17833 from cloud-fan/core. (cherry picked from commit b946f3160eb7953fb30edf1f097ea87be75b33e7) Signed-off-by: Wenchen Fan --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 6e24656e4635d..cf03ae5456358 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1850,6 +1850,9 @@ class SparkContext(config: SparkConf) extends Logging { } SparkEnv.set(null) } + // Clear this `InheritableThreadLocal`, or it will still be inherited in child threads even this + // `SparkContext` is stopped. + localProperties.remove() // Unset YARN mode system env variable, to allow switching between cluster types. System.clearProperty("SPARK_YARN_MODE") SparkContext.clearActiveContext() From 179f5370e68aa3c1f035f8ac400129c3935e96f8 Mon Sep 17 00:00:00 2001 From: jyu00 Date: Fri, 5 May 2017 11:36:51 +0100 Subject: [PATCH 007/151] [SPARK-20546][DEPLOY] spark-class gets syntax error in posix mode ## What changes were proposed in this pull request? Updated spark-class to turn off posix mode so the process substitution doesn't cause a syntax error. ## How was this patch tested? Existing unit tests, manual spark-shell testing with posix mode on Author: jyu00 Closes #17852 from jyu00/master. (cherry picked from commit 5773ab121d5d7cbefeef17ff4ac6f8af36cc1251) Signed-off-by: Sean Owen --- bin/spark-class | 2 ++ 1 file changed, 2 insertions(+) diff --git a/bin/spark-class b/bin/spark-class index 77ea40cc37946..65d3b9612909a 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -72,6 +72,8 @@ build_command() { printf "%d\0" $? } +# Turn off posix mode since it does not allow process substitution +set +o posix CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") From 2a7f5dae5906f01973d2744ab7b59b54e42f302b Mon Sep 17 00:00:00 2001 From: Jarrett Meyer Date: Fri, 5 May 2017 08:30:42 -0700 Subject: [PATCH 008/151] [SPARK-20613] Remove excess quotes in Windows executable ## What changes were proposed in this pull request? Quotes are already added to the RUNNER variable on line 54. There is no need to put quotes on line 67. If you do, you will get an error when launching Spark. '""C:\Program' is not recognized as an internal or external command, operable program or batch file. ## How was this patch tested? Tested manually on Windows 10. Author: Jarrett Meyer Closes #17861 from jarrettmeyer/fix-windows-cmd. (cherry picked from commit b9ad2d1916af5091c8585d06ccad8219e437e2bc) Signed-off-by: Felix Cheung --- bin/spark-class2.cmd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 9faa7d65f83e4..f6157f42843e8 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -51,7 +51,7 @@ if not "x%SPARK_PREPEND_CLASSES%"=="x" ( rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" ( - set RUNNER="%JAVA_HOME%\bin\java" + set RUNNER=%JAVA_HOME%\bin\java ) else ( where /q "%RUNNER%" if ERRORLEVEL 1 ( From 704b249b6a3ea956086d6c6ef50da18e8228eeb4 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 5 May 2017 11:08:26 -0700 Subject: [PATCH 009/151] [SPARK-20603][SS][TEST] Set default number of topic partitions to 1 to reduce the load ## What changes were proposed in this pull request? I checked the logs of https://amplab.cs.berkeley.edu/jenkins/job/spark-branch-2.2-test-maven-hadoop-2.7/47/ and found it took several seconds to create Kafka internal topic `__consumer_offsets`. As Kafka creates this topic lazily, the topic creation happens in the first test `deserialization of initial offset with Spark 2.1.0` and causes it timeout. This PR changes `offsets.topic.num.partitions` from the default value 50 to 1 to make creating `__consumer_offsets` (50 partitions -> 1 partition) much faster. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #17863 from zsxwing/fix-kafka-flaky-test. (cherry picked from commit bd5788287957d8610a6d19c273b75bd4cdd2d166) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala index c2cbd86260bc5..4345f882130a5 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaTestUtils.scala @@ -281,6 +281,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L props.put("log.flush.interval.messages", "1") props.put("replica.socket.timeout.ms", "1500") props.put("delete.topic.enable", "true") + props.put("offsets.topic.num.partitions", "1") props.putAll(withBrokerProps.asJava) props } From a1112c615b05d615048159c9d324aa10a4391d4e Mon Sep 17 00:00:00 2001 From: Juliusz Sompolski Date: Fri, 5 May 2017 15:31:06 -0700 Subject: [PATCH 010/151] [SPARK-20616] RuleExecutor logDebug of batch results should show diff to start of batch ## What changes were proposed in this pull request? Due to a likely typo, the logDebug msg printing the diff of query plans shows a diff to the initial plan, not diff to the start of batch. ## How was this patch tested? Now the debug message prints the diff between start and end of batch. Author: Juliusz Sompolski Closes #17875 from juliuszsompolski/SPARK-20616. (cherry picked from commit 5d75b14bf0f4c1f0813287efaabf49797908ed55) Signed-off-by: Reynold Xin --- .../org/apache/spark/sql/catalyst/rules/RuleExecutor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index 6fc828f63f152..85b368c862630 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -122,7 +122,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { logDebug( s""" |=== Result of Batch ${batch.name} === - |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")} + |${sideBySide(batchStartPlan.treeString, curPlan.treeString).mkString("\n")} """.stripMargin) } else { logTrace(s"Batch ${batch.name} has no effect.") From f7a91a17e8e20965b3e634e611690a96f72cec6b Mon Sep 17 00:00:00 2001 From: Jon McLean Date: Tue, 9 May 2017 09:47:50 +0100 Subject: [PATCH 011/151] [SPARK-20615][ML][TEST] SparseVector.argmax throws IndexOutOfBoundsException ## What changes were proposed in this pull request? Added a check for for the number of defined values. Previously the argmax function assumed that at least one value was defined if the vector size was greater than zero. ## How was this patch tested? Tests were added to the existing VectorsSuite to cover this case. Author: Jon McLean Closes #17877 from jonmclean/vectorArgmaxIndexBug. (cherry picked from commit be53a78352ae7c70d8a07d0df24574b3e3129b4a) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/ml/linalg/Vectors.scala | 2 ++ .../scala/org/apache/spark/ml/linalg/VectorsSuite.scala | 7 +++++++ .../main/scala/org/apache/spark/mllib/linalg/Vectors.scala | 2 ++ .../scala/org/apache/spark/mllib/linalg/VectorsSuite.scala | 7 +++++++ 4 files changed, 18 insertions(+) diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala index 22e4ec693b1f7..7bc2cb1701f98 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Vectors.scala @@ -657,6 +657,8 @@ class SparseVector @Since("2.0.0") ( override def argmax: Int = { if (size == 0) { -1 + } else if (numActives == 0) { + 0 } else { // Find the max active entry. var maxIdx = indices(0) diff --git a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala index ea22c2787fb3c..bd71656337699 100644 --- a/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala +++ b/mllib-local/src/test/scala/org/apache/spark/ml/linalg/VectorsSuite.scala @@ -125,6 +125,13 @@ class VectorsSuite extends SparkMLFunSuite { val vec8 = Vectors.sparse(5, Array(1, 2), Array(0.0, -1.0)) assert(vec8.argmax === 0) + + // Check for case when sparse vector is non-empty but the values are empty + val vec9 = Vectors.sparse(100, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec9.argmax === 0) + + val vec10 = Vectors.sparse(1, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec10.argmax === 0) } test("vector equals") { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 63ea9d3264b0f..52828492619f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -846,6 +846,8 @@ class SparseVector @Since("1.0.0") ( override def argmax: Int = { if (size == 0) { -1 + } else if (numActives == 0) { + 0 } else { // Find the max active entry. var maxIdx = indices(0) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 71a3ceac1b947..6172cffee861c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -122,6 +122,13 @@ class VectorsSuite extends SparkFunSuite with Logging { val vec8 = Vectors.sparse(5, Array(1, 2), Array(0.0, -1.0)) assert(vec8.argmax === 0) + + // Check for case when sparse vector is non-empty but the values are empty + val vec9 = Vectors.sparse(100, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec9.argmax === 0) + + val vec10 = Vectors.sparse(1, Array.empty[Int], Array.empty[Double]).asInstanceOf[SparseVector] + assert(vec10.argmax === 0) } test("vector equals") { From 12c937ede309d6886ce1ceadff2691f31dcdd6d3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 9 May 2017 11:25:29 -0700 Subject: [PATCH 012/151] [SPARK-20627][PYSPARK] Drop the hadoop distirbution name from the Python version ## What changes were proposed in this pull request? Drop the hadoop distirbution name from the Python version (PEP440 - https://www.python.org/dev/peps/pep-0440/). We've been using the local version string to disambiguate between different hadoop versions packaged with PySpark, but PEP0440 states that local versions should not be used when publishing up-stream. Since we no longer make PySpark pip packages for different hadoop versions, we can simply drop the hadoop information. If at a later point we need to start publishing different hadoop versions we can look at make different packages or similar. ## How was this patch tested? Ran `make-distribution` locally Author: Holden Karau Closes #17885 from holdenk/SPARK-20627-remove-pip-local-version-string. (cherry picked from commit 1b85bcd9298cf84dd746fe8e91ab0b0df69ef17e) Signed-off-by: Holden Karau --- dev/create-release/release-build.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index ab17f2fe7077b..c4ddc2186f5dc 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -163,9 +163,9 @@ if [[ "$1" == "package" ]]; then export ZINC_PORT=$ZINC_PORT echo "Creating distribution: $NAME ($FLAGS)" - # Write out the NAME and VERSION to PySpark version info we rewrite the - into a . and SNAPSHOT - # to dev0 to be closer to PEP440. We use the NAME as a "local version". - PYSPARK_VERSION=`echo "$SPARK_VERSION+$NAME" | sed -r "s/-/./" | sed -r "s/SNAPSHOT/dev0/"` + # Write out the VERSION to PySpark version info we rewrite the - into a . and SNAPSHOT + # to dev0 to be closer to PEP440. + PYSPARK_VERSION=`echo "$SPARK_VERSION" | sed -r "s/-/./" | sed -r "s/SNAPSHOT/dev0/"` echo "__version__='$PYSPARK_VERSION'" > python/pyspark/version.py # Get maven home set by MVN From 50f28dfe43410dadabbecc62e34fde8bacc8aee6 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 9 May 2017 19:45:00 -0700 Subject: [PATCH 013/151] [SPARK-17685][SQL] Make SortMergeJoinExec's currentVars is null when calling createJoinKey ## What changes were proposed in this pull request? The following SQL query cause `IndexOutOfBoundsException` issue when `LIMIT > 1310720`: ```sql CREATE TABLE tab1(int int, int2 int, str string); CREATE TABLE tab2(int int, int2 int, str string); INSERT INTO tab1 values(1,1,'str'); INSERT INTO tab1 values(2,2,'str'); INSERT INTO tab2 values(1,1,'str'); INSERT INTO tab2 values(2,3,'str'); SELECT count(*) FROM ( SELECT t1.int, t2.int2 FROM (SELECT * FROM tab1 LIMIT 1310721) t1 INNER JOIN (SELECT * FROM tab2 LIMIT 1310721) t2 ON (t1.int = t2.int AND t1.int2 = t2.int2) ) t; ``` This pull request fix this issue. ## How was this patch tested? unit tests Author: Yuming Wang Closes #17920 from wangyum/SPARK-17685. (cherry picked from commit 771abeb46f637592aba2e63db2ed05b6cabfd0be) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/joins/SortMergeJoinExec.scala | 1 + .../org/apache/spark/sql/DataFrameJoinSuite.scala | 10 ++++++++++ 2 files changed, 11 insertions(+) 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 a1f941644f807..89a9b38132732 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 @@ -342,6 +342,7 @@ case class SortMergeJoinExec( keys: Seq[Expression], input: Seq[Attribute]): Seq[ExprCode] = { ctx.INPUT_ROW = row + ctx.currentVars = null keys.map(BindReferences.bindReference(_, input).genCode(ctx)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index 541ffb58e727f..9383e8381f38a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -248,4 +248,14 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { val ab = a.join(b, Seq("a"), "fullouter") checkAnswer(ab.join(c, "a"), Row(3, null, 4, 1) :: Nil) } + + test("SPARK-17685: WholeStageCodegenExec throws IndexOutOfBoundsException") { + val df = Seq((1, 1, "1"), (2, 2, "3")).toDF("int", "int2", "str") + val df2 = Seq((1, 1, "1"), (2, 3, "5")).toDF("int", "int2", "str") + val limit = 1310721 + val innerJoin = df.limit(limit).join(df2.limit(limit), Seq("int", "int2"), "inner") + .agg(count($"int")) + checkAnswer(innerJoin, Row(1) :: Nil) + } + } From 8e097890a7b39cd8320ed2f2b98dc2b520a87cbf Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 May 2017 14:36:36 +0800 Subject: [PATCH 014/151] [SPARK-20686][SQL] PropagateEmptyRelation incorrectly handles aggregate without grouping The query ``` SELECT 1 FROM (SELECT COUNT(*) WHERE FALSE) t1 ``` should return a single row of output because the subquery is an aggregate without a group-by and thus should return a single row. However, Spark incorrectly returns zero rows. This is caused by SPARK-16208 / #13906, a patch which added an optimizer rule to propagate EmptyRelation through operators. The logic for handling aggregates is wrong: it checks whether aggregate expressions are non-empty for deciding whether the output should be empty, whereas it should be checking grouping expressions instead: An aggregate with non-empty grouping expression will return one output row per group. If the input to the grouped aggregate is empty then all groups will be empty and thus the output will be empty. It doesn't matter whether the aggregation output columns include aggregate expressions since that won't affect the number of output rows. If the grouping expressions are empty, however, then the aggregate will always produce a single output row and thus we cannot propagate the EmptyRelation. The current implementation is incorrect and also misses an optimization opportunity by not propagating EmptyRelation in the case where a grouped aggregate has aggregate expressions (in other words, `SELECT COUNT(*) from emptyRelation GROUP BY x` would _not_ be optimized to `EmptyRelation` in the old code, even though it safely could be). This patch resolves this issue by modifying `PropagateEmptyRelation` to consider only the presence/absence of grouping expressions, not the aggregate functions themselves, when deciding whether to propagate EmptyRelation. - Added end-to-end regression tests in `SQLQueryTest`'s `group-by.sql` file. - Updated unit tests in `PropagateEmptyRelationSuite`. Author: Josh Rosen Closes #17929 from JoshRosen/fix-PropagateEmptyRelation. (cherry picked from commit a90c5cd8226146a58362732171b92cb99a7bc4c7) Signed-off-by: Wenchen Fan --- .../optimizer/PropagateEmptyRelation.scala | 16 ++++++------ .../PropagateEmptyRelationSuite.scala | 8 +++--- .../resources/sql-tests/inputs/group-by.sql | 7 +++++ .../sql-tests/results/group-by.sql.out | 26 ++++++++++++++++++- 4 files changed, 44 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index 7400a01918c52..987cd7434b459 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -30,7 +29,7 @@ import org.apache.spark.sql.catalyst.rules._ * - Join with one or two empty children (including Intersect/Except). * 2. Unary-node Logical Plans * - Project/Filter/Sample/Join/Limit/Repartition with all empty children. - * - Aggregate with all empty children and without AggregateFunction expressions like COUNT. + * - Aggregate with all empty children and at least one grouping expression. * - Generate(Explode) with all empty children. Others like Hive UDTF may return results. */ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper { @@ -39,10 +38,6 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper { case _ => false } - private def containsAggregateExpression(e: Expression): Boolean = { - e.collectFirst { case _: AggregateFunction => () }.isDefined - } - private def empty(plan: LogicalPlan) = LocalRelation(plan.output, data = Seq.empty) def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -68,8 +63,13 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper { case _: LocalLimit => empty(p) case _: Repartition => empty(p) case _: RepartitionByExpression => empty(p) - // AggregateExpressions like COUNT(*) return their results like 0. - case Aggregate(_, ae, _) if !ae.exists(containsAggregateExpression) => empty(p) + // An aggregate with non-empty group expression will return one output row per group when the + // input to the aggregate is not empty. If the input to the aggregate is empty then all groups + // will be empty and thus the output will be empty. + // + // If the grouping expressions are empty, however, then the aggregate will always produce a + // single output row and thus we cannot propagate the EmptyRelation. + case Aggregate(ge, _, _) if ge.nonEmpty => empty(p) // Generators like Hive-style UDTF may return their records within `close`. case Generate(_: Explode, _, _, _, _, _) => empty(p) case _ => p diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index 908dde7a66988..2285be16938d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -142,7 +142,7 @@ class PropagateEmptyRelationSuite extends PlanTest { comparePlans(optimized, correctAnswer.analyze) } - test("propagate empty relation through Aggregate without aggregate function") { + test("propagate empty relation through Aggregate with grouping expressions") { val query = testRelation1 .where(false) .groupBy('a)('a, ('a + 1).as('x)) @@ -153,13 +153,13 @@ class PropagateEmptyRelationSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("don't propagate empty relation through Aggregate with aggregate function") { + test("don't propagate empty relation through Aggregate without grouping expressions") { val query = testRelation1 .where(false) - .groupBy('a)(count('a)) + .groupBy()() val optimized = Optimize.execute(query.analyze) - val correctAnswer = LocalRelation('a.int).groupBy('a)(count('a)).analyze + val correctAnswer = LocalRelation('a.int).groupBy()().analyze comparePlans(optimized, correctAnswer) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 4d0ed43153004..f4f5a043d4781 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -35,3 +35,10 @@ FROM testData; -- Aggregate with foldable input and multiple distinct groups. SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS c) GROUP BY a; + +-- Aggregate with empty input and non-empty GroupBy expressions. +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a; + +-- Aggregate with empty input and empty GroupBy expressions. +SELECT COUNT(1) FROM testData WHERE false; +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 4b87d5161fc0e..f64dd0007846a 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 18 -- !query 0 @@ -139,3 +139,27 @@ SELECT COUNT(DISTINCT b), COUNT(DISTINCT b, c) FROM (SELECT 1 AS a, 2 AS b, 3 AS struct -- !query 14 output 1 1 + + +-- !query 15 +SELECT a, COUNT(1) FROM testData WHERE false GROUP BY a +-- !query 15 schema +struct +-- !query 15 output + + + +-- !query 16 +SELECT COUNT(1) FROM testData WHERE false +-- !query 16 schema +struct +-- !query 16 output +0 + + +-- !query 17 +SELECT 1 FROM (SELECT COUNT(1) FROM testData WHERE false) t +-- !query 17 schema +struct<1:int> +-- !query 17 output +1 From 69786ea3a972af1b29a332dc11ac507ed4368cc6 Mon Sep 17 00:00:00 2001 From: zero323 Date: Wed, 10 May 2017 16:57:52 +0800 Subject: [PATCH 015/151] [SPARK-20631][PYTHON][ML] LogisticRegression._checkThresholdConsistency should use values not Params ## What changes were proposed in this pull request? - Replace `getParam` calls with `getOrDefault` calls. - Fix exception message to avoid unintended `TypeError`. - Add unit tests ## How was this patch tested? New unit tests. Author: zero323 Closes #17891 from zero323/SPARK-20631. (cherry picked from commit 804949c6bf00b8e26c39d48bbcc4d0470ee84e47) Signed-off-by: Yanbo Liang --- python/pyspark/ml/classification.py | 6 +++--- python/pyspark/ml/tests.py | 12 ++++++++++++ 2 files changed, 15 insertions(+), 3 deletions(-) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 570a414cc350c..2b47c402674ad 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -238,13 +238,13 @@ def getThresholds(self): def _checkThresholdConsistency(self): if self.isSet(self.threshold) and self.isSet(self.thresholds): - ts = self.getParam(self.thresholds) + ts = self.getOrDefault(self.thresholds) if len(ts) != 2: raise ValueError("Logistic Regression getThreshold only applies to" + " binary classification, but thresholds has length != 2." + - " thresholds: " + ",".join(ts)) + " thresholds: {0}".format(str(ts))) t = 1.0/(1.0 + ts[0]/ts[1]) - t2 = self.getParam(self.threshold) + t2 = self.getOrDefault(self.threshold) if abs(t2 - t) >= 1E-5: raise ValueError("Logistic Regression getThreshold found inconsistent values for" + " threshold (%g) and thresholds (equivalent to %g)" % (t2, t)) diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 70e0c6de4a7bd..7152036e385d2 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -808,6 +808,18 @@ def test_logistic_regression(self): except OSError: pass + def logistic_regression_check_thresholds(self): + self.assertIsInstance( + LogisticRegression(threshold=0.5, thresholds=[0.5, 0.5]), + LogisticRegressionModel + ) + + self.assertRaisesRegexp( + ValueError, + "Logistic Regression getThreshold found inconsistent.*$", + LogisticRegression, threshold=0.42, thresholds=[0.5, 0.5] + ) + def _compare_params(self, m1, m2, param): """ Compare 2 ML Params instances for the given param, and assert both have the same param value From bdc08ab644f8e0974681446d724600c5c2ac7a56 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 May 2017 19:30:00 +0800 Subject: [PATCH 016/151] [SPARK-20688][SQL] correctly check analysis for scalar sub-queries In `CheckAnalysis`, we should call `checkAnalysis` for `ScalarSubquery` at the beginning, as later we will call `plan.output` which is invalid if `plan` is not resolved. new regression test Author: Wenchen Fan Closes #17930 from cloud-fan/tmp. (cherry picked from commit 789bdbe3d0d9558043872161bdfa148ec021a849) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/analysis/CheckAnalysis.scala | 3 ++- .../scala/org/apache/spark/sql/SubquerySuite.scala | 10 +++++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 06bbd39ed11d6..e68bf81c6428b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -128,6 +128,8 @@ trait CheckAnalysis extends PredicateHelper { } case s @ ScalarSubquery(query, conditions, _) => + checkAnalysis(query) + // If no correlation, the output must be exactly one column if (conditions.isEmpty && query.output.size != 1) { failAnalysis( @@ -186,7 +188,6 @@ trait CheckAnalysis extends PredicateHelper { case fail => failAnalysis(s"Correlated scalar subqueries must be Aggregated: $fail") } } - checkAnalysis(query) s case s: SubqueryExpression => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index fb92f314088d1..b0b28337509fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -72,7 +72,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } } - test("rdd deserialization does not crash [SPARK-15791]") { + test("SPARK-15791: rdd deserialization does not crash") { sql("select (select 1 as b) as b").rdd.count() } @@ -839,4 +839,12 @@ class SubquerySuite extends QueryTest with SharedSQLContext { Row(0) :: Row(1) :: Nil) } } + + test("SPARK-20688: correctly check analysis for scalar sub-queries") { + withTempView("t") { + Seq(1 -> "a").toDF("i", "j").createTempView("t") + val e = intercept[AnalysisException](sql("SELECT (SELECT count(*) FROM t WHERE a = 1)")) + assert(e.message.contains("cannot resolve '`a`' given input columns: [i, j]")) + } + } } From 92a71a667dd3e13664015f2a9dd2a39e2c1514eb Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 10 May 2017 16:50:57 -0700 Subject: [PATCH 017/151] [SPARK-20685] Fix BatchPythonEvaluation bug in case of single UDF w/ repeated arg. ## What changes were proposed in this pull request? There's a latent corner-case bug in PySpark UDF evaluation where executing a `BatchPythonEvaluation` with a single multi-argument UDF where _at least one argument value is repeated_ will crash at execution with a confusing error. This problem was introduced in #12057: the code there has a fast path for handling a "batch UDF evaluation consisting of a single Python UDF", but that branch incorrectly assumes that a single UDF won't have repeated arguments and therefore skips the code for unpacking arguments from the input row (whose schema may not necessarily match the UDF inputs due to de-duplication of repeated arguments which occurred in the JVM before sending UDF inputs to Python). This fix here is simply to remove this special-casing: it turns out that the code in the "multiple UDFs" branch just so happens to work for the single-UDF case because Python treats `(x)` as equivalent to `x`, not as a single-argument tuple. ## How was this patch tested? New regression test in `pyspark.python.sql.tests` module (tested and confirmed that it fails before my fix). Author: Josh Rosen Closes #17927 from JoshRosen/SPARK-20685. (cherry picked from commit 8ddbc431d8b21d5ee57d3d209a4f25e301f15283) Signed-off-by: Xiao Li --- python/pyspark/sql/tests.py | 6 ++++++ python/pyspark/worker.py | 29 +++++++++++++---------------- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 22b1ffc900751..c3cc7426b177b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -324,6 +324,12 @@ def test_chained_udf(self): [row] = self.spark.sql("SELECT double(double(1) + 1)").collect() self.assertEqual(row[0], 6) + def test_single_udf_with_repeated_argument(self): + # regression test for SPARK-20685 + self.spark.catalog.registerFunction("add", lambda x, y: x + y, IntegerType()) + row = self.spark.sql("SELECT add(1, 1)").first() + self.assertEqual(tuple(row), (2, )) + def test_multiple_udfs(self): self.spark.catalog.registerFunction("double", lambda x: x * 2, IntegerType()) [row] = self.spark.sql("SELECT double(1), double(2)").collect() diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 09182829538fc..5eeaac70b80ed 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -86,22 +86,19 @@ def read_single_udf(pickleSer, infile): def read_udfs(pickleSer, infile): num_udfs = read_int(infile) - if num_udfs == 1: - # fast path for single UDF - _, udf = read_single_udf(pickleSer, infile) - mapper = lambda a: udf(*a) - else: - udfs = {} - call_udf = [] - for i in range(num_udfs): - arg_offsets, udf = read_single_udf(pickleSer, infile) - udfs['f%d' % i] = udf - args = ["a[%d]" % o for o in arg_offsets] - call_udf.append("f%d(%s)" % (i, ", ".join(args))) - # Create function like this: - # lambda a: (f0(a0), f1(a1, a2), f2(a3)) - mapper_str = "lambda a: (%s)" % (", ".join(call_udf)) - mapper = eval(mapper_str, udfs) + udfs = {} + call_udf = [] + for i in range(num_udfs): + arg_offsets, udf = read_single_udf(pickleSer, infile) + udfs['f%d' % i] = udf + args = ["a[%d]" % o for o in arg_offsets] + call_udf.append("f%d(%s)" % (i, ", ".join(args))) + # Create function like this: + # lambda a: (f0(a0), f1(a1, a2), f2(a3)) + # In the special case of a single UDF this will return a single result rather + # than a tuple of results; this is the format that the JVM side expects. + mapper_str = "lambda a: (%s)" % (", ".join(call_udf)) + mapper = eval(mapper_str, udfs) func = lambda _, it: map(mapper, it) ser = BatchedSerializer(PickleSerializer(), 100) From 6e89d574058bc2b96b14a691a07580be67f63707 Mon Sep 17 00:00:00 2001 From: liuxian Date: Fri, 12 May 2017 11:38:50 +0800 Subject: [PATCH 018/151] [SPARK-20665][SQL] Bround" and "Round" function return NULL spark-sql>select bround(12.3, 2); spark-sql>NULL For this case, the expected result is 12.3, but it is null. So ,when the second parameter is bigger than "decimal.scala", the result is not we expected. "round" function has the same problem. This PR can solve the problem for both of them. unit test cases in MathExpressionsSuite and MathFunctionsSuite Author: liuxian Closes #17906 from 10110346/wip_lx_0509. (cherry picked from commit 2b36eb696f6c738e1328582630755aaac4293460) Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/mathExpressions.scala | 12 ++++++------ .../catalyst/expressions/MathExpressionsSuite.scala | 7 +++---- .../org/apache/spark/sql/MathFunctionsSuite.scala | 13 +++++++++++++ 3 files changed, 22 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala index 65273a77b1054..54b8457403758 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathExpressions.scala @@ -1021,10 +1021,10 @@ abstract class RoundBase(child: Expression, scale: Expression, // not overriding since _scale is a constant int at runtime def nullSafeEval(input1: Any): Any = { - child.dataType match { - case _: DecimalType => + dataType match { + case DecimalType.Fixed(_, s) => val decimal = input1.asInstanceOf[Decimal] - if (decimal.changePrecision(decimal.precision, _scale, mode)) decimal else null + if (decimal.changePrecision(decimal.precision, s, mode)) decimal else null case ByteType => BigDecimal(input1.asInstanceOf[Byte]).setScale(_scale, mode).toByte case ShortType => @@ -1053,10 +1053,10 @@ abstract class RoundBase(child: Expression, scale: Expression, override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val ce = child.genCode(ctx) - val evaluationCode = child.dataType match { - case _: DecimalType => + val evaluationCode = dataType match { + case DecimalType.Fixed(_, s) => s""" - if (${ce.value}.changePrecision(${ce.value}.precision(), ${_scale}, + if (${ce.value}.changePrecision(${ce.value}.precision(), ${s}, java.math.BigDecimal.${modeStr})) { ${ev.value} = ${ce.value}; } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 6b5bfac94645c..1555dd1cf58d4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -546,15 +546,14 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val bdResults: Seq[BigDecimal] = Seq(BigDecimal(3.0), BigDecimal(3.1), BigDecimal(3.14), BigDecimal(3.142), BigDecimal(3.1416), BigDecimal(3.14159), BigDecimal(3.141593), BigDecimal(3.1415927)) - // round_scale > current_scale would result in precision increase - // and not allowed by o.a.s.s.types.Decimal.changePrecision, therefore null + (0 to 7).foreach { i => checkEvaluation(Round(bdPi, i), bdResults(i), EmptyRow) checkEvaluation(BRound(bdPi, i), bdResults(i), EmptyRow) } (8 to 10).foreach { scale => - checkEvaluation(Round(bdPi, scale), null, EmptyRow) - checkEvaluation(BRound(bdPi, scale), null, EmptyRow) + checkEvaluation(Round(bdPi, scale), bdPi, EmptyRow) + checkEvaluation(BRound(bdPi, scale), bdPi, EmptyRow) } DataTypeTestUtils.numericTypes.foreach { dataType => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala index 37443d0342980..0284f8311e180 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathFunctionsSuite.scala @@ -231,6 +231,19 @@ class MathFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(BigDecimal("0E3"), BigDecimal("0E2"), BigDecimal("0E1"), BigDecimal(3), BigDecimal("3.1"), BigDecimal("3.14"), BigDecimal("3.142"))) ) + + val bdPi: BigDecimal = BigDecimal(31415925L, 7) + checkAnswer( + sql(s"SELECT round($bdPi, 7), round($bdPi, 8), round($bdPi, 9), round($bdPi, 10), " + + s"round($bdPi, 100), round($bdPi, 6), round(null, 8)"), + Seq(Row(bdPi, bdPi, bdPi, bdPi, bdPi, BigDecimal("3.141593"), null)) + ) + + checkAnswer( + sql(s"SELECT bround($bdPi, 7), bround($bdPi, 8), bround($bdPi, 9), bround($bdPi, 10), " + + s"bround($bdPi, 100), bround($bdPi, 6), bround(null, 8)"), + Seq(Row(bdPi, bdPi, bdPi, bdPi, bdPi, BigDecimal("3.141592"), null)) + ) } test("exp") { From 95de4672ea18e8cc0e14ee574386d252948e003d Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Fri, 12 May 2017 20:38:36 +0800 Subject: [PATCH 019/151] [SPARK-17424] Fix unsound substitution bug in ScalaReflection. ## What changes were proposed in this pull request? This method gets a type's primary constructor and fills in type parameters with concrete types. For example, `MapPartitions[T, U] -> MapPartitions[Int, String]`. This Substitution fails when the actual type args are empty because they are still unknown. Instead, when there are no resolved types to subsitute, this returns the original args with unresolved type parameters. ## How was this patch tested? This doesn't affect substitutions where the type args are determined. This fixes our case where the actual type args are empty and our job runs successfully. Author: Ryan Blue Closes #15062 from rdblue/SPARK-17424-fix-unsound-reflect-substitution. (cherry picked from commit b23693390781a99ff9248ea07a22e68884ffc747) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/catalyst/ScalaReflection.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index fa1b900592b67..1a1daaeba4677 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -814,8 +814,16 @@ trait ScalaReflection { def getConstructorParameters(tpe: Type): Seq[(String, Type)] = { val formalTypeArgs = tpe.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = tpe - constructParams(tpe).map { p => - p.name.toString -> p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs) + val params = constructParams(tpe) + // if there are type variables to fill in, do the substitution (SomeClass[T] -> SomeClass[Int]) + if (actualTypeArgs.nonEmpty) { + params.map { p => + p.name.toString -> p.typeSignature.substituteTypes(formalTypeArgs, actualTypeArgs) + } + } else { + params.map { p => + p.name.toString -> p.typeSignature + } } } From 62969e9be01b55e1cf5ab2c9b273a9e08d6bc2c3 Mon Sep 17 00:00:00 2001 From: guoxiaolong Date: Mon, 15 May 2017 07:51:50 +0100 Subject: [PATCH 020/151] [SPARK-20705][WEB-UI] The sort function can not be used in the master page when you use Firefox or Google Chrome. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? When you open the master page, when you use Firefox or Google Chrom, the console of Firefox or Google Chrome is wrong. But The IE is no problem. e.g. ![error](https://cloud.githubusercontent.com/assets/26266482/25946143/74467a5c-367c-11e7-8f9f-d3585b1aea88.png) My Firefox version is 48.0.2. My Google Chrome version is 49.0.2623.75 m. ## How was this patch tested? manual tests Please review http://spark.apache.org/contributing.html before opening a pull request. Author: guoxiaolong Author: 郭小龙 10207633 Author: guoxiaolongzte Closes #17952 from guoxiaolongzte/SPARK-20705. (cherry picked from commit 99d5799927301e7dfceb9405e2829af3433f104b) Signed-off-by: Sean Owen --- .../main/resources/org/apache/spark/ui/static/sorttable.js | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js index ff241470f32df..9960d5c34d1fc 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/sorttable.js +++ b/core/src/main/resources/org/apache/spark/ui/static/sorttable.js @@ -207,8 +207,8 @@ sorttable = { hasInputs = (typeof node.getElementsByTagName == 'function') && node.getElementsByTagName('input').length; - - if (node.getAttribute("sorttable_customkey") != null) { + + if (node.nodeType == 1 && node.getAttribute("sorttable_customkey") != null) { return node.getAttribute("sorttable_customkey"); } else if (typeof node.textContent != 'undefined' && !hasInputs) { From 14b6a9d340a75e12ae49b3e8e885997aaffff79c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 15 May 2017 11:24:30 -0700 Subject: [PATCH 021/151] [SPARK-20735][SQL][TEST] Enable cross join in TPCDSQueryBenchmark ## What changes were proposed in this pull request? Since [SPARK-17298](https://issues.apache.org/jira/browse/SPARK-17298), some queries (q28, q61, q77, q88, q90) in the test suites fail with a message "_Use the CROSS JOIN syntax to allow cartesian products between these relations_". This benchmark is used as a reference model for Spark TPC-DS, so this PR aims to enable the correct configuration in `TPCDSQueryBenchmark.scala`. ## How was this patch tested? Manual. (Run TPCDSQueryBenchmark) Author: Dongjoon Hyun Closes #17977 from dongjoon-hyun/SPARK-20735. (cherry picked from commit bbd163d589e7503c5cb150d934e7565b18a908f2) Signed-off-by: Xiao Li --- .../spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala index 3988d9750b585..a23c87514d3a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/TPCDSQueryBenchmark.scala @@ -43,6 +43,7 @@ object TPCDSQueryBenchmark { .set("spark.driver.memory", "3g") .set("spark.executor.memory", "3g") .set("spark.sql.autoBroadcastJoinThreshold", (20 * 1024 * 1024).toString) + .set("spark.sql.crossJoin.enabled", "true") val spark = SparkSession.builder.config(conf).getOrCreate() From ba35c6bd320cda6fbb8ff92a6779986cdc464267 Mon Sep 17 00:00:00 2001 From: Andrew Ray Date: Wed, 17 May 2017 10:06:01 +0100 Subject: [PATCH 022/151] [SPARK-20769][DOC] Incorrect documentation for using Jupyter notebook ## What changes were proposed in this pull request? SPARK-13973 incorrectly removed the required PYSPARK_DRIVER_PYTHON_OPTS=notebook from documentation to use pyspark with Jupyter notebook. This patch corrects the documentation error. ## How was this patch tested? Tested invocation locally with ```bash PYSPARK_DRIVER_PYTHON=jupyter PYSPARK_DRIVER_PYTHON_OPTS=notebook ./bin/pyspark ``` Author: Andrew Ray Closes #18001 from aray/patch-1. (cherry picked from commit 1995417696a028f8a4fa7f706a77537c7182528d) Signed-off-by: Sean Owen --- docs/programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 353730c28f3c7..b9c7a95667d11 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -247,7 +247,7 @@ $ PYSPARK_DRIVER_PYTHON=ipython ./bin/pyspark To use the Jupyter notebook (previously known as the IPython notebook), {% highlight bash %} -$ PYSPARK_DRIVER_PYTHON=jupyter ./bin/pyspark +$ PYSPARK_DRIVER_PYTHON=jupyter PYSPARK_DRIVER_PYTHON_OPTS=notebook ./bin/pyspark {% endhighlight %} You can customize the `ipython` or `jupyter` commands by setting `PYSPARK_DRIVER_PYTHON_OPTS`. From e06d9367f619c1b66c8778af61725a069ba7eb71 Mon Sep 17 00:00:00 2001 From: liuzhaokun Date: Thu, 18 May 2017 17:44:40 +0100 Subject: [PATCH 023/151] [SPARK-20796] the location of start-master.sh in spark-standalone.md is wrong [https://issues.apache.org/jira/browse/SPARK-20796](https://issues.apache.org/jira/browse/SPARK-20796) the location of start-master.sh in spark-standalone.md should be "sbin/start-master.sh" rather than "bin/start-master.sh". Author: liuzhaokun Closes #18027 from liu-zhaokun/sbin. (cherry picked from commit 99452df44fb98c2721d427da4c97f549793615fe) Signed-off-by: Sean Owen --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 1c0b60f7b9346..084bd701fdcc4 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -83,7 +83,7 @@ Once you've set up this file, you can launch or stop your cluster with the follo - `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. - `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on. - `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. -- `sbin/stop-master.sh` - Stops the master that was started via the `bin/start-master.sh` script. +- `sbin/stop-master.sh` - Stops the master that was started via the `sbin/start-master.sh` script. - `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. - `sbin/stop-all.sh` - Stops both the master and the slaves as described above. From e326de485869f6c0708dba1df5f71faaee568f59 Mon Sep 17 00:00:00 2001 From: Ala Luszczak Date: Fri, 19 May 2017 13:18:48 +0200 Subject: [PATCH 024/151] [SPARK-20798] GenerateUnsafeProjection should check if a value is null before calling the getter ## What changes were proposed in this pull request? GenerateUnsafeProjection.writeStructToBuffer() did not honor the assumption that the caller must make sure that a value is not null before using the getter. This could lead to various errors. This change fixes that behavior. Example of code generated before: ```scala /* 059 */ final UTF8String fieldName = value.getUTF8String(0); /* 060 */ if (value.isNullAt(0)) { /* 061 */ rowWriter1.setNullAt(0); /* 062 */ } else { /* 063 */ rowWriter1.write(0, fieldName); /* 064 */ } ``` Example of code generated now: ```scala /* 060 */ boolean isNull1 = value.isNullAt(0); /* 061 */ UTF8String value1 = isNull1 ? null : value.getUTF8String(0); /* 062 */ if (isNull1) { /* 063 */ rowWriter1.setNullAt(0); /* 064 */ } else { /* 065 */ rowWriter1.write(0, value1); /* 066 */ } ``` ## How was this patch tested? Adds GenerateUnsafeProjectionSuite. Author: Ala Luszczak Closes #18030 from ala/fix-generate-unsafe-projection. (cherry picked from commit ce8edb8bf4db5f82bcfeb11efbdf5229b0d25dfa) Signed-off-by: Herman van Hovell --- .../codegen/GenerateUnsafeProjection.scala | 15 +++-- .../GenerateUnsafeProjectionSuite.scala | 61 +++++++++++++++++++ .../execution/vectorized/ColumnarBatch.java | 6 ++ 3 files changed, 78 insertions(+), 4 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 7e4c9089a2cb9..b358102d914bd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -50,10 +50,17 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro fieldTypes: Seq[DataType], bufferHolder: String): String = { val fieldEvals = fieldTypes.zipWithIndex.map { case (dt, i) => - val fieldName = ctx.freshName("fieldName") - val code = s"final ${ctx.javaType(dt)} $fieldName = ${ctx.getValue(input, dt, i.toString)};" - val isNull = s"$input.isNullAt($i)" - ExprCode(code, isNull, fieldName) + val javaType = ctx.javaType(dt) + val isNullVar = ctx.freshName("isNull") + val valueVar = ctx.freshName("value") + val defaultValue = ctx.defaultValue(dt) + val readValue = ctx.getValue(input, dt, i.toString) + val code = + s""" + boolean $isNullVar = $input.isNullAt($i); + $javaType $valueVar = $isNullVar ? $defaultValue : $readValue; + """ + ExprCode(code, isNullVar, valueVar) } s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala new file mode 100644 index 0000000000000..e9d21f8a8ebcd --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjectionSuite.scala @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BoundReference +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.types.{DataType, Decimal, StringType, StructType} +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} + +class GenerateUnsafeProjectionSuite extends SparkFunSuite { + test("Test unsafe projection string access pattern") { + val dataType = (new StructType).add("a", StringType) + val exprs = BoundReference(0, dataType, nullable = true) :: Nil + val projection = GenerateUnsafeProjection.generate(exprs) + val result = projection.apply(InternalRow(AlwaysNull)) + assert(!result.isNullAt(0)) + assert(result.getStruct(0, 1).isNullAt(0)) + } +} + +object AlwaysNull extends InternalRow { + override def numFields: Int = 1 + override def setNullAt(i: Int): Unit = {} + override def copy(): InternalRow = this + override def anyNull: Boolean = true + override def isNullAt(ordinal: Int): Boolean = true + override def update(i: Int, value: Any): Unit = notSupported + override def getBoolean(ordinal: Int): Boolean = notSupported + override def getByte(ordinal: Int): Byte = notSupported + override def getShort(ordinal: Int): Short = notSupported + override def getInt(ordinal: Int): Int = notSupported + override def getLong(ordinal: Int): Long = notSupported + override def getFloat(ordinal: Int): Float = notSupported + override def getDouble(ordinal: Int): Double = notSupported + override def getDecimal(ordinal: Int, precision: Int, scale: Int): Decimal = notSupported + override def getUTF8String(ordinal: Int): UTF8String = notSupported + override def getBinary(ordinal: Int): Array[Byte] = notSupported + override def getInterval(ordinal: Int): CalendarInterval = notSupported + override def getStruct(ordinal: Int, numFields: Int): InternalRow = notSupported + override def getArray(ordinal: Int): ArrayData = notSupported + override def getMap(ordinal: Int): MapData = notSupported + override def get(ordinal: Int, dataType: DataType): AnyRef = notSupported + private def notSupported: Nothing = throw new UnsupportedOperationException +} 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 a6ce4c2edc232..8b7b0e655b31d 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 @@ -198,21 +198,25 @@ public boolean anyNull() { @Override public Decimal getDecimal(int ordinal, int precision, int scale) { + if (columns[ordinal].isNullAt(rowId)) return null; return columns[ordinal].getDecimal(rowId, precision, scale); } @Override public UTF8String getUTF8String(int ordinal) { + if (columns[ordinal].isNullAt(rowId)) return null; return columns[ordinal].getUTF8String(rowId); } @Override public byte[] getBinary(int ordinal) { + if (columns[ordinal].isNullAt(rowId)) return null; return columns[ordinal].getBinary(rowId); } @Override public CalendarInterval getInterval(int ordinal) { + if (columns[ordinal].isNullAt(rowId)) return null; final int months = columns[ordinal].getChildColumn(0).getInt(rowId); final long microseconds = columns[ordinal].getChildColumn(1).getLong(rowId); return new CalendarInterval(months, microseconds); @@ -220,11 +224,13 @@ public CalendarInterval getInterval(int ordinal) { @Override public InternalRow getStruct(int ordinal, int numFields) { + if (columns[ordinal].isNullAt(rowId)) return null; return columns[ordinal].getStruct(rowId); } @Override public ArrayData getArray(int ordinal) { + if (columns[ordinal].isNullAt(rowId)) return null; return columns[ordinal].getArray(rowId); } From c53fe793dbe17a3598b0466ec644130cc251c30f Mon Sep 17 00:00:00 2001 From: liuzhaokun Date: Fri, 19 May 2017 15:26:39 +0100 Subject: [PATCH 025/151] [SPARK-20759] SCALA_VERSION in _config.yml should be consistent with pom.xml [https://issues.apache.org/jira/browse/SPARK-20759](https://issues.apache.org/jira/browse/SPARK-20759) SCALA_VERSION in _config.yml is 2.11.7, but 2.11.8 in pom.xml. So I think SCALA_VERSION in _config.yml should be consistent with pom.xml. Author: liuzhaokun Closes #17992 from liu-zhaokun/new. (cherry picked from commit dba2ca2c129b6d2597f1707e0315d4e238c40ed6) Signed-off-by: Sean Owen --- LICENSE | 10 +++++----- docs/_config.yml | 2 +- external/docker/spark-test/base/Dockerfile | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/LICENSE b/LICENSE index 7950dd6ceb6db..119ecbe15ab7a 100644 --- a/LICENSE +++ b/LICENSE @@ -249,11 +249,11 @@ The text of each license is also included at licenses/LICENSE-[project].txt. (Interpreter classes (all .scala files in repl/src/main/scala except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), and for SerializableMapWrapper in JavaUtils.scala) - (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.11.7 - http://www.scala-lang.org/) - (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.11.7 - http://www.scala-lang.org/) - (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.11.7 - http://www.scala-lang.org/) - (BSD-like) Scala Library (org.scala-lang:scala-library:2.11.7 - http://www.scala-lang.org/) - (BSD-like) Scalap (org.scala-lang:scalap:2.11.7 - http://www.scala-lang.org/) + (BSD-like) Scala Actors library (org.scala-lang:scala-actors:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-compiler:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scala Library (org.scala-lang:scala-library:2.11.8 - http://www.scala-lang.org/) + (BSD-like) Scalap (org.scala-lang:scalap:2.11.8 - http://www.scala-lang.org/) (BSD-style) scalacheck (org.scalacheck:scalacheck_2.11:1.10.0 - http://www.scalacheck.org) (BSD-style) spire (org.spire-math:spire_2.11:0.7.1 - http://spire-math.org) (BSD-style) spire-macros (org.spire-math:spire-macros_2.11:0.7.1 - http://spire-math.org) diff --git a/docs/_config.yml b/docs/_config.yml index e21d011c4f83f..ee38a4cc91998 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,7 +17,7 @@ include: SPARK_VERSION: 2.1.2-SNAPSHOT SPARK_VERSION_SHORT: 2.1.2 SCALA_BINARY_VERSION: "2.11" -SCALA_VERSION: "2.11.7" +SCALA_VERSION: "2.11.8" MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index 76f550f886ce4..5a95a9387c310 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -25,7 +25,7 @@ RUN apt-get update && \ apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.11.7 +ENV SCALA_VERSION 2.11.8 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark From e9804b3d4442e4c353af4a171e8f3c1915104bd0 Mon Sep 17 00:00:00 2001 From: liuzhaokun Date: Fri, 19 May 2017 20:47:30 +0100 Subject: [PATCH 026/151] [SPARK-20781] the location of Dockerfile in docker.properties.templat is wrong [https://issues.apache.org/jira/browse/SPARK-20781](https://issues.apache.org/jira/browse/SPARK-20781) the location of Dockerfile in docker.properties.template should be "../external/docker/spark-mesos/Dockerfile" Author: liuzhaokun Closes #18013 from liu-zhaokun/dockerfile_location. (cherry picked from commit 749418d285461958a0f22ed355edafd87f1ee913) Signed-off-by: Sean Owen --- conf/docker.properties.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/docker.properties.template b/conf/docker.properties.template index 55cb094b4af46..2ecb4f1464a4f 100644 --- a/conf/docker.properties.template +++ b/conf/docker.properties.template @@ -15,6 +15,6 @@ # limitations under the License. # -spark.mesos.executor.docker.image: +spark.mesos.executor.docker.image: spark.mesos.executor.docker.volumes: /usr/local/lib:/host/usr/local/lib:ro spark.mesos.executor.home: /opt/spark From c3a986b19616217b4b0d8e10e52fcc87fb5356ba Mon Sep 17 00:00:00 2001 From: Ignacio Bermudez Date: Mon, 22 May 2017 10:27:28 +0100 Subject: [PATCH 027/151] [SPARK-20687][MLLIB] mllib.Matrices.fromBreeze may crash when converting from Breeze sparse matrix ## What changes were proposed in this pull request? When two Breeze SparseMatrices are operated, the result matrix may contain provisional 0 values extra in rowIndices and data arrays. This causes an incoherence with the colPtrs data, but Breeze get away with this incoherence by keeping a counter of the valid data. In spark, when this matrices are converted to SparseMatrices, Sparks relies solely on rowIndices, data, and colPtrs, but these might be incorrect because of breeze internal hacks. Therefore, we need to slice both rowIndices and data, using their counter of active data This method is at least called by BlockMatrix when performing distributed block operations, causing exceptions on valid operations. See http://stackoverflow.com/questions/33528555/error-thrown-when-using-blockmatrix-add ## How was this patch tested? Added a test to MatricesSuite that verifies that the conversions are valid and that code doesn't crash. Originally the same code would crash on Spark. Bugfix for https://issues.apache.org/jira/browse/SPARK-20687 Author: Ignacio Bermudez Author: Ignacio Bermudez Corrales Closes #17940 from ghoto/bug-fix/SPARK-20687. (cherry picked from commit 06dda1d58f8670e996921e935d5f5402d664699e) Signed-off-by: Sean Owen --- .../apache/spark/mllib/linalg/Matrices.scala | 11 +++++++++- .../spark/mllib/linalg/MatricesSuite.scala | 20 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 6c39fe5d84865..2b2b5fe49ea32 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -992,7 +992,16 @@ object Matrices { new DenseMatrix(dm.rows, dm.cols, dm.data, dm.isTranspose) case sm: BSM[Double] => // There is no isTranspose flag for sparse matrices in Breeze - new SparseMatrix(sm.rows, sm.cols, sm.colPtrs, sm.rowIndices, sm.data) + val nsm = if (sm.rowIndices.length > sm.activeSize) { + // This sparse matrix has trailing zeros. + // Remove them by compacting the matrix. + val csm = sm.copy + csm.compact() + csm + } else { + sm + } + new SparseMatrix(nsm.rows, nsm.cols, nsm.colPtrs, nsm.rowIndices, nsm.data) case _ => throw new UnsupportedOperationException( s"Do not support conversion from type ${breeze.getClass.getName}.") diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 563756907d201..93c00d80974c3 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -513,6 +513,26 @@ class MatricesSuite extends SparkFunSuite { Matrices.fromBreeze(sum) } + test("Test FromBreeze when Breeze.CSCMatrix.rowIndices has trailing zeros. - SPARK-20687") { + // (2, 0, 0) + // (2, 0, 0) + val mat1Brz = Matrices.sparse(2, 3, Array(0, 2, 2, 2), Array(0, 1), Array(2, 2)).asBreeze + // (2, 1E-15, 1E-15) + // (2, 1E-15, 1E-15) + val mat2Brz = Matrices.sparse(2, 3, + Array(0, 2, 4, 6), + Array(0, 0, 0, 1, 1, 1), + Array(2, 1E-15, 1E-15, 2, 1E-15, 1E-15)).asBreeze + val t1Brz = mat1Brz - mat2Brz + val t2Brz = mat2Brz - mat1Brz + // The following operations raise exceptions on un-patch Matrices.fromBreeze + val t1 = Matrices.fromBreeze(t1Brz) + val t2 = Matrices.fromBreeze(t2Brz) + // t1 == t1Brz && t2 == t2Brz + assert((t1.asBreeze - t1Brz).iterator.map((x) => math.abs(x._2)).sum < 1E-15) + assert((t2.asBreeze - t2Brz).iterator.map((x) => math.abs(x._2)).sum < 1E-15) + } + test("row/col iterator") { val dm = new DenseMatrix(3, 2, Array(0, 1, 2, 3, 4, 0)) val sm = dm.toSparse From f5ef0762518892986aebcabbcc6c269273d2e1be Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Mon, 22 May 2017 10:10:41 -0700 Subject: [PATCH 028/151] [SPARK-20756][YARN] yarn-shuffle jar references unshaded guava and contains scala classes ## What changes were proposed in this pull request? This change ensures that all references to guava from within the yarn shuffle jar pointed to the shaded guava class already provided in the jar. Also, it explicitly excludes scala classes from being added to the jar. ## How was this patch tested? Ran unit tests on the module and they passed. javap now returns the expected result - reference to the shaded guava under `org/spark_project` (previously this was referring to `com.google...` ``` javap -cp common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar -c org/apache/spark/network/yarn/YarnShuffleService | grep Lists 57: invokestatic #138 // Method org/spark_project/guava/collect/Lists.newArrayList:()Ljava/util/ArrayList; ``` Guava is still shaded in the jar: ``` jar -tf common/network-yarn/target/scala-2.11/spark-2.3.0-SNAPSHOT-yarn-shuffle.jar | grep guava | head META-INF/maven/com.google.guava/ META-INF/maven/com.google.guava/guava/ META-INF/maven/com.google.guava/guava/pom.properties META-INF/maven/com.google.guava/guava/pom.xml org/spark_project/guava/ org/spark_project/guava/annotations/ org/spark_project/guava/annotations/Beta.class org/spark_project/guava/annotations/GwtCompatible.class org/spark_project/guava/annotations/GwtIncompatible.class org/spark_project/guava/annotations/VisibleForTesting.class ``` (not sure if the above META-INF/* is a problem or not) I took this jar, deployed it on a yarn cluster with shuffle service enabled, and made sure the YARN node managers came up. An application with a shuffle was run and it succeeded. Author: Mark Grover Closes #17990 from markgrover/spark-20756. (cherry picked from commit 36309110046a89d749a7c9746eaa16997de26922) Signed-off-by: Marcelo Vanzin --- common/network-yarn/pom.xml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 7a57e8964f6fc..fb6c241185897 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -87,6 +87,9 @@ *:* + + org.scala-lang:scala-library + @@ -98,7 +101,7 @@ - + com.fasterxml.jackson ${spark.shade.packageName}.com.fasterxml.jackson From f4538c95f73a6249b314e2241f1a3d44a8da4523 Mon Sep 17 00:00:00 2001 From: liuxian Date: Tue, 23 May 2017 10:09:18 +0900 Subject: [PATCH 029/151] [SPARK-20763][SQL][BACKPORT-2.1] The function of `month` and `day` return the value which is not we expected. What changes were proposed in this pull request? This PR is to backport #17997 to Spark 2.1 when the date before "1582-10-04", the function of month and day return the value which is not we expected. How was this patch tested? unit tests Author: liuxian Closes #18054 from 10110346/wip-lx-0522. --- .../spark/sql/catalyst/util/DateTimeUtils.scala | 9 ++++++++- .../expressions/DateExpressionsSuite.scala | 14 ++++++++++++++ 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 235ca8d2633a1..15e8772842968 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -511,7 +511,14 @@ object DateTimeUtils { */ private[this] def getYearAndDayInYear(daysSince1970: SQLDate): (Int, Int) = { // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) - val daysNormalized = daysSince1970 + toYearZero + var daysSince1970Tmp = daysSince1970 + // Since Julian calendar was replaced with the Gregorian calendar, + // the 10 days after Oct. 4 were skipped. + // (1582-10-04) -141428 days since 1970-01-01 + if (daysSince1970 <= -141428) { + daysSince1970Tmp -= 10 + } + val daysNormalized = daysSince1970Tmp + toYearZero val numOfQuarterCenturies = daysNormalized / daysIn400Years val daysInThis400 = daysNormalized % daysIn400Years + 1 val (years, dayInYear) = numYears(daysInThis400) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala index 35cea25ba0b7d..25a7aa5a9d837 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala @@ -60,6 +60,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } checkEvaluation(DayOfYear(Literal.create(null, DateType)), null) + checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 288) + checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 277) checkConsistencyBetweenInterpretedAndCodegen(DayOfYear, DateType) } @@ -80,6 +82,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + checkEvaluation(Year(Literal(new Date(sdf.parse("1582-01-01 13:10:15").getTime))), 1582) + checkEvaluation(Year(Literal(new Date(sdf.parse("1581-12-31 13:10:15").getTime))), 1581) checkConsistencyBetweenInterpretedAndCodegen(Year, DateType) } @@ -100,6 +104,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } + checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-10-01 13:10:15").getTime))), 4) + checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-09-30 13:10:15").getTime))), 3) checkConsistencyBetweenInterpretedAndCodegen(Quarter, DateType) } @@ -109,6 +115,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType)), 4) checkEvaluation(Month(Cast(Literal(ts), DateType)), 11) + checkEvaluation(Month(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 4) + checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 10) + checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 10) (2003 to 2004).foreach { y => (0 to 3).foreach { m => (0 to 2 * 24).foreach { i => @@ -130,6 +139,9 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(DayOfMonth(Cast(Literal(sdfDate.format(d)), DateType)), 8) checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType)), 8) + checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 28) + checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 15) + checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 4) (1999 to 2000).foreach { y => val c = Calendar.getInstance() c.set(y, 0, 1, 0, 0, 0) @@ -163,6 +175,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType)), 15) checkEvaluation(WeekOfYear(Cast(Literal(ts), DateType)), 45) checkEvaluation(WeekOfYear(Cast(Literal("2011-05-06"), DateType)), 18) + checkEvaluation(WeekOfYear(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 40) + checkEvaluation(WeekOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 40) checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType) } From 13adc0fc0e940a4ea8b703241666440357a597e3 Mon Sep 17 00:00:00 2001 From: Bago Amirbekian Date: Wed, 24 May 2017 22:55:38 +0800 Subject: [PATCH 030/151] [SPARK-20862][MLLIB][PYTHON] Avoid passing float to ndarray.reshape in LogisticRegressionModel ## What changes were proposed in this pull request? Fixed TypeError with python3 and numpy 1.12.1. Numpy's `reshape` no longer takes floats as arguments as of 1.12. Also, python3 uses float division for `/`, we should be using `//` to ensure that `_dataWithBiasSize` doesn't get set to a float. ## How was this patch tested? Existing tests run using python3 and numpy 1.12. Author: Bago Amirbekian Closes #18081 from MrBago/BF-py3floatbug. (cherry picked from commit bc66a77bbe2120cc21bd8da25194efca4cde13c3) Signed-off-by: Yanbo Liang --- python/pyspark/mllib/classification.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 9f53ed098202b..e04eeb2b60d71 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -171,7 +171,7 @@ def __init__(self, weights, intercept, numFeatures, numClasses): self._dataWithBiasSize = None self._weightsMatrix = None else: - self._dataWithBiasSize = self._coeff.size / (self._numClasses - 1) + self._dataWithBiasSize = self._coeff.size // (self._numClasses - 1) self._weightsMatrix = self._coeff.toArray().reshape(self._numClasses - 1, self._dataWithBiasSize) From 2f68631f523e4db08549497d9c3264a43137bbb1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 25 May 2017 00:35:40 +0800 Subject: [PATCH 031/151] [SPARK-20848][SQL] Shutdown the pool after reading parquet files ## What changes were proposed in this pull request? From JIRA: On each call to spark.read.parquet, a new ForkJoinPool is created. One of the threads in the pool is kept in the WAITING state, and never stopped, which leads to unbounded growth in number of threads. We should shutdown the pool after reading parquet files. ## How was this patch tested? Added a test to ParquetFileFormatSuite. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh Closes #18073 from viirya/SPARK-20848. (cherry picked from commit f72ad303f05a6d99513ea3b121375726b177199c) Signed-off-by: Wenchen Fan --- .../execution/datasources/parquet/ParquetFileFormat.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) 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 2b4892ee23ba3..70eb01cfc9fb0 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 @@ -496,7 +496,8 @@ object ParquetFileFormat extends Logging { partFiles: Seq[FileStatus], ignoreCorruptFiles: Boolean): Seq[Footer] = { val parFiles = partFiles.par - parFiles.tasksupport = new ForkJoinTaskSupport(new ForkJoinPool(8)) + val pool = new ForkJoinPool(8) + parFiles.tasksupport = new ForkJoinTaskSupport(pool) parFiles.flatMap { currentFile => try { // Skips row group information since we only need the schema. @@ -512,6 +513,8 @@ object ParquetFileFormat extends Logging { } else { throw new IOException(s"Could not read footer for file: $currentFile", e) } + } finally { + pool.shutdown() } }.seq } From c3302e81e3e74744ec8da66b18f81c233a15e918 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Thu, 25 May 2017 08:31:04 +0800 Subject: [PATCH 032/151] [SPARK-18406][CORE][BACKPORT-2.1] Race between end-of-task and completion iterator read lock release This is a backport PR of #18076 to 2.1. ## What changes were proposed in this pull request? When a TaskContext is not propagated properly to all child threads for the task, just like the reported cases in this issue, we fail to get to TID from TaskContext and that causes unable to release the lock and assertion failures. To resolve this, we have to explicitly pass the TID value to the `unlock` method. ## How was this patch tested? Add new failing regression test case in `RDDSuite`. Author: Xingbo Jiang Closes #18099 from jiangxb1987/completion-iterator-2.1. --- .../spark/network/BlockDataManager.scala | 2 +- .../spark/storage/BlockInfoManager.scala | 15 ++++++++----- .../apache/spark/storage/BlockManager.scala | 21 ++++++++++++++----- .../scala/org/apache/spark/rdd/RDDSuite.scala | 18 +++++++++++++++- 4 files changed, 44 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index 8f83668d79029..b3f8bfe8b1d48 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -46,5 +46,5 @@ trait BlockDataManager { /** * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. */ - def releaseLock(blockId: BlockId): Unit + def releaseLock(blockId: BlockId, taskAttemptId: Option[Long]): Unit } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala index dd8f5bacb9f6e..c0e18e54f410b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala @@ -281,22 +281,27 @@ private[storage] class BlockInfoManager extends Logging { /** * Release a lock on the given block. + * In case a TaskContext is not propagated properly to all child threads for the task, we fail to + * get the TID from TaskContext, so we have to explicitly pass the TID value to release the lock. + * + * See SPARK-18406 for more discussion of this issue. */ - def unlock(blockId: BlockId): Unit = synchronized { - logTrace(s"Task $currentTaskAttemptId releasing lock for $blockId") + def unlock(blockId: BlockId, taskAttemptId: Option[TaskAttemptId] = None): Unit = synchronized { + val taskId = taskAttemptId.getOrElse(currentTaskAttemptId) + logTrace(s"Task $taskId releasing lock for $blockId") val info = get(blockId).getOrElse { throw new IllegalStateException(s"Block $blockId not found") } if (info.writerTask != BlockInfo.NO_WRITER) { info.writerTask = BlockInfo.NO_WRITER - writeLocksByTask.removeBinding(currentTaskAttemptId, blockId) + writeLocksByTask.removeBinding(taskId, blockId) } else { assert(info.readerCount > 0, s"Block $blockId is not locked for reading") info.readerCount -= 1 - val countsForTask = readLocksByTask(currentTaskAttemptId) + val countsForTask = readLocksByTask(taskId) val newPinCountForTask: Int = countsForTask.remove(blockId, 1) - 1 assert(newPinCountForTask >= 0, - s"Task $currentTaskAttemptId release lock on block $blockId more times than it acquired it") + s"Task $taskId release lock on block $blockId more times than it acquired it") } notifyAll() } 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 35551e41213a3..6ff61bdd5b7b5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -454,6 +454,7 @@ private[spark] class BlockManager( case Some(info) => val level = info.level logDebug(s"Level for block $blockId is $level") + val taskAttemptId = Option(TaskContext.get()).map(_.taskAttemptId()) if (level.useMemory && memoryStore.contains(blockId)) { val iter: Iterator[Any] = if (level.deserialized) { memoryStore.getValues(blockId).get @@ -461,7 +462,12 @@ private[spark] class BlockManager( serializerManager.dataDeserializeStream( blockId, memoryStore.getBytes(blockId).get.toInputStream())(info.classTag) } - val ci = CompletionIterator[Any, Iterator[Any]](iter, releaseLock(blockId)) + // We need to capture the current taskId in case the iterator completion is triggered + // from a different thread which does not have TaskContext set; see SPARK-18406 for + // discussion. + val ci = CompletionIterator[Any, Iterator[Any]](iter, { + releaseLock(blockId, taskAttemptId) + }) Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) } else if (level.useDisk && diskStore.contains(blockId)) { val iterToReturn: Iterator[Any] = { @@ -478,7 +484,9 @@ private[spark] class BlockManager( serializerManager.dataDeserializeStream(blockId, stream)(info.classTag) } } - val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, releaseLock(blockId)) + val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, { + releaseLock(blockId, taskAttemptId) + }) Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) } else { handleLocalReadFailure(blockId) @@ -654,10 +662,13 @@ private[spark] class BlockManager( } /** - * Release a lock on the given block. + * Release a lock on the given block with explicit TID. + * The param `taskAttemptId` should be passed in case we can't get the correct TID from + * TaskContext, for example, the input iterator of a cached RDD iterates to the end in a child + * thread. */ - def releaseLock(blockId: BlockId): Unit = { - blockInfoManager.unlock(blockId) + def releaseLock(blockId: BlockId, taskAttemptId: Option[Long] = None): Unit = { + blockInfoManager.unlock(blockId, taskAttemptId) } /** diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index ad56715656c85..8d06f5468f4f1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -30,7 +30,7 @@ import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} class RDDSuite extends SparkFunSuite with SharedSparkContext { var tempDir: File = _ @@ -1082,6 +1082,22 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(totalPartitionCount == 10) } + test("SPARK-18406: race between end-of-task and completion iterator read lock release") { + val rdd = sc.parallelize(1 to 1000, 10) + rdd.cache() + + rdd.mapPartitions { iter => + ThreadUtils.runInNewThread("TestThread") { + // Iterate to the end of the input iterator, to cause the CompletionIterator completion to + // fire outside of the task's main thread. + while (iter.hasNext) { + iter.next() + } + iter + } + }.collect() + } + // NOTE // Below tests calling sc.stop() have to be the last tests in this suite. If there are tests // running after them and if they access sc those tests will fail as sc is already closed, because From 7015f6f0e7889616db9595b4e68b5a5b5ffe921a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 25 May 2017 09:55:45 +0800 Subject: [PATCH 033/151] [SPARK-20848][SQL][FOLLOW-UP] Shutdown the pool after reading parquet files ## What changes were proposed in this pull request? This is a follow-up to #18073. Taking a safer approach to shutdown the pool to prevent possible issue. Also using `ThreadUtils.newForkJoinPool` instead to set a better thread name. ## How was this patch tested? Manually test. Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Liang-Chi Hsieh Closes #18100 from viirya/SPARK-20848-followup. (cherry picked from commit 6b68d61cf31748a088778dfdd66491b2f89a3c7b) Signed-off-by: Wenchen Fan --- .../parquet/ParquetFileFormat.scala | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) 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 70eb01cfc9fb0..f303a014fa483 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 @@ -50,7 +50,7 @@ import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} class ParquetFileFormat extends FileFormat @@ -496,27 +496,29 @@ object ParquetFileFormat extends Logging { partFiles: Seq[FileStatus], ignoreCorruptFiles: Boolean): Seq[Footer] = { val parFiles = partFiles.par - val pool = new ForkJoinPool(8) + val pool = ThreadUtils.newForkJoinPool("readingParquetFooters", 8) parFiles.tasksupport = new ForkJoinTaskSupport(pool) - parFiles.flatMap { currentFile => - try { - // Skips row group information since we only need the schema. - // ParquetFileReader.readFooter throws RuntimeException, instead of IOException, - // when it can't read the footer. - Some(new Footer(currentFile.getPath(), - ParquetFileReader.readFooter( - conf, currentFile, SKIP_ROW_GROUPS))) - } catch { case e: RuntimeException => - if (ignoreCorruptFiles) { - logWarning(s"Skipped the footer in the corrupted file: $currentFile", e) - None - } else { - throw new IOException(s"Could not read footer for file: $currentFile", e) + try { + parFiles.flatMap { currentFile => + try { + // Skips row group information since we only need the schema. + // ParquetFileReader.readFooter throws RuntimeException, instead of IOException, + // when it can't read the footer. + Some(new Footer(currentFile.getPath(), + ParquetFileReader.readFooter( + conf, currentFile, SKIP_ROW_GROUPS))) + } catch { case e: RuntimeException => + if (ignoreCorruptFiles) { + logWarning(s"Skipped the footer in the corrupted file: $currentFile", e) + None + } else { + throw new IOException(s"Could not read footer for file: $currentFile", e) + } } - } finally { - pool.shutdown() - } - }.seq + }.seq + } finally { + pool.shutdown() + } } /** From 7fc2347b510d73fb55ab69c0579494b0761fb022 Mon Sep 17 00:00:00 2001 From: Xianyang Liu Date: Thu, 25 May 2017 15:47:59 +0800 Subject: [PATCH 034/151] [SPARK-20250][CORE] Improper OOM error when a task been killed while spilling data Currently, when a task is calling spill() but it receives a killing request from driver (e.g., speculative task), the `TaskMemoryManager` will throw an `OOM` exception. And we don't catch `Fatal` exception when a error caused by `Thread.interrupt`. So for `ClosedByInterruptException`, we should throw `RuntimeException` instead of `OutOfMemoryError`. https://issues.apache.org/jira/browse/SPARK-20250?jql=project%20%3D%20SPARK Existing unit tests. Author: Xianyang Liu Closes #18090 from ConeyLiu/SPARK-20250. (cherry picked from commit 731462a04f8e33ac507ad19b4270c783a012a33e) Signed-off-by: Wenchen Fan --- .../java/org/apache/spark/memory/TaskMemoryManager.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index c40974b54cb47..3385d0e6a31cf 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -19,6 +19,7 @@ import javax.annotation.concurrent.GuardedBy; import java.io.IOException; +import java.nio.channels.ClosedByInterruptException; import java.util.Arrays; import java.util.BitSet; import java.util.HashSet; @@ -156,6 +157,10 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { break; } } + } catch (ClosedByInterruptException e) { + // This called by user to kill a task (e.g: speculative task). + logger.error("error while calling spill() on " + c, e); + throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("error while calling spill() on " + c, e); throw new OutOfMemoryError("error while calling spill() on " + c + " : " @@ -174,6 +179,10 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { Utils.bytesToString(released), consumer); got += memoryManager.acquireExecutionMemory(required - got, taskAttemptId, mode); } + } catch (ClosedByInterruptException e) { + // This called by user to kill a task (e.g: speculative task). + logger.error("error while calling spill() on " + consumer, e); + throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("error while calling spill() on " + consumer, e); throw new OutOfMemoryError("error while calling spill() on " + consumer + " : " From 4f6fccf15d40da503a7c6a8722058c38d57178cc Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Thu, 25 May 2017 10:49:14 -0700 Subject: [PATCH 035/151] [SPARK-20874][EXAMPLES] Add Structured Streaming Kafka Source to examples project ## What changes were proposed in this pull request? Add Structured Streaming Kafka Source to the `examples` project so that people can run `bin/run-example StructuredKafkaWordCount ...`. ## How was this patch tested? manually tested it. Author: Shixiong Zhu Closes #18101 from zsxwing/add-missing-example-dep. (cherry picked from commit 98c3852986a2cb5f2d249d6c8ef602be283bd90e) Signed-off-by: Shixiong Zhu --- examples/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/examples/pom.xml b/examples/pom.xml index 8fa731fb340a2..f17e605250a15 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -90,6 +90,12 @@ ${project.version} provided + + org.apache.spark + spark-sql-kafka-0-10_${scala.binary.version} + ${project.version} + provided + org.apache.commons commons-math3 From 6e6adcceab9218463d8c7637350d7f5229cf648d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 26 May 2017 15:01:28 +0800 Subject: [PATCH 036/151] [SPARK-20868][CORE] UnsafeShuffleWriter should verify the position after FileChannel.transferTo ## What changes were proposed in this pull request? Long time ago we fixed a [bug](https://issues.apache.org/jira/browse/SPARK-3948) in shuffle writer about `FileChannel.transferTo`. We were not very confident about that fix, so we added a position check after the writing, try to discover the bug earlier. However this checking is missing in the new `UnsafeShuffleWriter`, this PR adds it. https://issues.apache.org/jira/browse/SPARK-18105 maybe related to that `FileChannel.transferTo` bug, hopefully we can find out the root cause after adding this position check. ## How was this patch tested? N/A Author: Wenchen Fan Closes #18091 from cloud-fan/shuffle. (cherry picked from commit d9ad78908f6189719cec69d34557f1a750d2e6af) Signed-off-by: Wenchen Fan --- .../shuffle/sort/UnsafeShuffleWriter.java | 15 ++-- .../scala/org/apache/spark/util/Utils.scala | 71 +++++++++++-------- 2 files changed, 47 insertions(+), 39 deletions(-) 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 8a1771848dee6..2fde5c300f072 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 @@ -422,17 +422,14 @@ private long[] mergeSpillsWithTransferTo(SpillInfo[] spills, File outputFile) th for (int partition = 0; partition < numPartitions; partition++) { for (int i = 0; i < spills.length; i++) { final long partitionLengthInSpill = spills[i].partitionLengths[partition]; - long bytesToTransfer = partitionLengthInSpill; final FileChannel spillInputChannel = spillInputChannels[i]; final long writeStartTime = System.nanoTime(); - while (bytesToTransfer > 0) { - final long actualBytesTransferred = spillInputChannel.transferTo( - spillInputChannelPositions[i], - bytesToTransfer, - mergedFileOutputChannel); - spillInputChannelPositions[i] += actualBytesTransferred; - bytesToTransfer -= actualBytesTransferred; - } + Utils.copyFileStreamNIO( + spillInputChannel, + mergedFileOutputChannel, + spillInputChannelPositions[i], + partitionLengthInSpill); + spillInputChannelPositions[i] += partitionLengthInSpill; writeMetrics.incWriteTime(System.nanoTime() - writeStartTime); bytesWrittenToMergedFile += partitionLengthInSpill; partitionLengths[partition] += partitionLengthInSpill; diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4cdfb9cbf39b7..56de802c423ca 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -21,7 +21,7 @@ import java.io._ import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo} import java.net._ import java.nio.ByteBuffer -import java.nio.channels.Channels +import java.nio.channels.{Channels, FileChannel} import java.nio.charset.StandardCharsets import java.nio.file.{Files, Paths} import java.util.{Locale, Properties, Random, UUID} @@ -58,7 +58,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} -import org.apache.spark.util.logging.RollingFileAppender /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) @@ -313,41 +312,22 @@ private[spark] object Utils extends Logging { * copying is disabled by default unless explicitly set transferToEnabled as true, * the parameter transferToEnabled should be configured by spark.file.transferTo = [true|false]. */ - def copyStream(in: InputStream, - out: OutputStream, - closeStreams: Boolean = false, - transferToEnabled: Boolean = false): Long = - { - var count = 0L + def copyStream( + in: InputStream, + out: OutputStream, + closeStreams: Boolean = false, + transferToEnabled: Boolean = false): Long = { tryWithSafeFinally { if (in.isInstanceOf[FileInputStream] && out.isInstanceOf[FileOutputStream] && transferToEnabled) { // When both streams are File stream, use transferTo to improve copy performance. val inChannel = in.asInstanceOf[FileInputStream].getChannel() val outChannel = out.asInstanceOf[FileOutputStream].getChannel() - val initialPos = outChannel.position() val size = inChannel.size() - - // In case transferTo method transferred less data than we have required. - while (count < size) { - count += inChannel.transferTo(count, size - count, outChannel) - } - - // Check the position after transferTo loop to see if it is in the right position and - // give user information if not. - // Position will not be increased to the expected length after calling transferTo in - // kernel version 2.6.32, this issue can be seen in - // https://bugs.openjdk.java.net/browse/JDK-7052359 - // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). - val finalPos = outChannel.position() - assert(finalPos == initialPos + size, - s""" - |Current position $finalPos do not equal to expected position ${initialPos + size} - |after transferTo, please check your kernel version to see if it is 2.6.32, - |this is a kernel bug which will lead to unexpected behavior when using transferTo. - |You can set spark.file.transferTo = false to disable this NIO feature. - """.stripMargin) + copyFileStreamNIO(inChannel, outChannel, 0, size) + size } else { + var count = 0L val buf = new Array[Byte](8192) var n = 0 while (n != -1) { @@ -357,8 +337,8 @@ private[spark] object Utils extends Logging { count += n } } + count } - count } { if (closeStreams) { try { @@ -370,6 +350,37 @@ private[spark] object Utils extends Logging { } } + def copyFileStreamNIO( + input: FileChannel, + output: FileChannel, + startPosition: Long, + bytesToCopy: Long): Unit = { + val initialPos = output.position() + var count = 0L + // In case transferTo method transferred less data than we have required. + while (count < bytesToCopy) { + count += input.transferTo(count + startPosition, bytesToCopy - count, output) + } + assert(count == bytesToCopy, + s"request to copy $bytesToCopy bytes, but actually copied $count bytes.") + + // Check the position after transferTo loop to see if it is in the right position and + // give user information if not. + // Position will not be increased to the expected length after calling transferTo in + // kernel version 2.6.32, this issue can be seen in + // https://bugs.openjdk.java.net/browse/JDK-7052359 + // This will lead to stream corruption issue when using sort-based shuffle (SPARK-3948). + val finalPos = output.position() + val expectedPos = initialPos + bytesToCopy + assert(finalPos == expectedPos, + s""" + |Current position $finalPos do not equal to expected position $expectedPos + |after transferTo, please check your kernel version to see if it is 2.6.32, + |this is a kernel bug which will lead to unexpected behavior when using transferTo. + |You can set spark.file.transferTo = false to disable this NIO feature. + """.stripMargin) + } + /** * Construct a URI container information used for authentication. * This also sets the default authenticator to properly negotiation the From ebd72f453aa0b4f68760d28b3e93e6dd33856659 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 26 May 2017 22:25:38 -0700 Subject: [PATCH 037/151] [SPARK-20843][CORE] Add a config to set driver terminate timeout ## What changes were proposed in this pull request? Add a `worker` configuration to set how long to wait before forcibly killing driver. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #18126 from zsxwing/SPARK-20843. (cherry picked from commit 6c1dbd6fc8d49acf7c1c902d2ebf89ed5e788a4e) Signed-off-by: Shixiong Zhu --- .../scala/org/apache/spark/deploy/worker/DriverRunner.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e878c10183f61..58a181128eb4d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -57,7 +57,8 @@ private[deploy] class DriverRunner( @volatile private[worker] var finalException: Option[Exception] = None // Timeout to wait for when trying to terminate a driver. - private val DRIVER_TERMINATE_TIMEOUT_MS = 10 * 1000 + private val DRIVER_TERMINATE_TIMEOUT_MS = + conf.getTimeAsMs("spark.worker.driverTerminateTimeout", "10s") // Decoupled for testing def setClock(_clock: Clock): Unit = { From 38f37c557367e474000d7d7ae659ab8a8df69097 Mon Sep 17 00:00:00 2001 From: "NICHOLAS T. MARION" Date: Wed, 10 May 2017 10:59:57 +0100 Subject: [PATCH 038/151] [SPARK-20393][WEBU UI] Strengthen Spark to prevent XSS vulnerabilities Add stripXSS and stripXSSMap to Spark Core's UIUtils. Calling these functions at any point that getParameter is called against a HttpServletRequest. Unit tests, IBM Security AppScan Standard no longer showing vulnerabilities, manual verification of WebUI pages. Author: NICHOLAS T. MARION Closes #17686 from n-marion/xss-fix. (cherry picked from commit b512233a457092b0e2a39d0b42cb021abc69d375) Signed-off-by: Sean Owen --- .../spark/deploy/history/HistoryPage.scala | 3 +- .../deploy/master/ui/ApplicationPage.scala | 3 +- .../spark/deploy/master/ui/MasterPage.scala | 6 ++- .../spark/deploy/worker/ui/LogPage.scala | 30 ++++++++------ .../scala/org/apache/spark/ui/UIUtils.scala | 21 ++++++++++ .../ui/exec/ExecutorThreadDumpPage.scala | 4 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 14 ++++--- .../org/apache/spark/ui/jobs/JobPage.scala | 3 +- .../org/apache/spark/ui/jobs/JobsTab.scala | 5 ++- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 15 +++---- .../org/apache/spark/ui/jobs/StageTable.scala | 15 +++---- .../org/apache/spark/ui/jobs/StagesTab.scala | 5 ++- .../org/apache/spark/ui/storage/RDDPage.scala | 13 ++++--- .../org/apache/spark/ui/UIUtilsSuite.scala | 39 +++++++++++++++++++ .../spark/deploy/mesos/ui/DriverPage.scala | 3 +- .../sql/execution/ui/ExecutionPage.scala | 3 +- .../ui/ThriftServerSessionPage.scala | 4 +- .../apache/spark/streaming/ui/BatchPage.scala | 5 ++- 19 files changed, 140 insertions(+), 54 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 0e7a6c24d4fa5..af14717633409 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -26,8 +26,9 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { + // stripXSS is called first to remove suspicious characters used in XSS attacks val requestedIncomplete = - Option(request.getParameter("showIncomplete")).getOrElse("false").toBoolean + Option(UIUtils.stripXSS(request.getParameter("showIncomplete"))).getOrElse("false").toBoolean val allAppsSize = parent.getApplicationList().count(_.completed != requestedIncomplete) val eventLogsUnderProcessCount = parent.getEventLogsUnderProcess() diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 18cff3125d6b4..57778701ff9eb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -33,7 +33,8 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { - val appId = request.getParameter("appId") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val appId = UIUtils.stripXSS(request.getParameter("appId")) val state = master.askWithRetry[MasterStateResponse](RequestMasterState) val app = state.activeApps.find(_.id == appId) .getOrElse(state.completedApps.find(_.id == appId).orNull) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 3fb860582cc17..d26e99cda31ae 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -57,8 +57,10 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = { if (parent.killEnabled && parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val id = Option(request.getParameter("id")) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val killFlag = + Option(UIUtils.stripXSS(request.getParameter("terminate"))).getOrElse("false").toBoolean + val id = Option(UIUtils.stripXSS(request.getParameter("id"))) if (id.isDefined && killFlag) { action(id.get) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 465c214362b25..63c0d7d418b60 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -35,13 +35,16 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val supportedLogTypes = Set("stderr", "stdout") private val defaultBytes = 100 * 1024 + // stripXSS is called first to remove suspicious characters used in XSS attacks def renderLog(request: HttpServletRequest): String = { - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) + val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) + val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) + val logType = UIUtils.stripXSS(request.getParameter("logType")) + val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) + val byteLength = + Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + .getOrElse(defaultBytes) val logDir = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => @@ -57,13 +60,16 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with pre + logText } + // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) + val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) + val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) + val logType = UIUtils.stripXSS(request.getParameter("logType")) + val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) + val byteLength = + Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + .getOrElse(defaultBytes) val (logDir, params, pageName) = (appId, executorId, driverId) match { case (Some(a), Some(e), None) => diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index d161843dd2230..40aa7ff8a7864 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -25,6 +25,8 @@ import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} +import org.apache.commons.lang3.StringEscapeUtils + import org.apache.spark.internal.Logging import org.apache.spark.ui.scope.RDDOperationGraph @@ -34,6 +36,8 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" + private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue(): SimpleDateFormat = @@ -524,4 +528,21 @@ private[spark] object UIUtils extends Logging { origHref } } + + /** + * Remove suspicious characters of user input to prevent Cross-Site scripting (XSS) attacks + * + * For more information about XSS testing: + * https://www.owasp.org/index.php/XSS_Filter_Evasion_Cheat_Sheet and + * https://www.owasp.org/index.php/Testing_for_Reflected_Cross_site_scripting_(OTG-INPVAL-001) + */ + def stripXSS(requestParameter: String): String = { + if (requestParameter == null) { + null + } else { + // Remove new lines and single quotes, followed by escaping HTML version 4.0 + StringEscapeUtils.escapeHtml4( + NEWLINE_AND_SINGLE_QUOTE_REGEX.replaceAllIn(requestParameter, "")) + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala index c6a07445f2a35..d1577d170a8aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorThreadDumpPage.scala @@ -27,8 +27,10 @@ private[ui] class ExecutorThreadDumpPage(parent: ExecutorsTab) extends WebUIPage private val sc = parent.sc + // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val executorId = Option(request.getParameter("executorId")).map { executorId => + val executorId = + Option(UIUtils.stripXSS(request.getParameter("executorId"))).map { executorId => UIUtils.decodeURLParameter(executorId) }.getOrElse { throw new IllegalArgumentException(s"Missing executorId parameter") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala index d9475c4c5d5f2..3f25a2deca4b5 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 @@ -220,18 +220,20 @@ private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { jobTag: String, jobs: Seq[JobUIData], killEnabled: Boolean): Seq[Node] = { - val allParameters = request.getParameterMap.asScala.toMap + // stripXSS is called to remove suspicious characters used in XSS attacks + val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS)) val parameterOtherTable = allParameters.filterNot(_._1.startsWith(jobTag)) .map(para => para._1 + "=" + para._2(0)) val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) val jobIdTitle = if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id" - val parameterJobPage = request.getParameter(jobTag + ".page") - val parameterJobSortColumn = request.getParameter(jobTag + ".sort") - val parameterJobSortDesc = request.getParameter(jobTag + ".desc") - val parameterJobPageSize = request.getParameter(jobTag + ".pageSize") - val parameterJobPrevPageSize = request.getParameter(jobTag + ".prevPageSize") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterJobPage = UIUtils.stripXSS(request.getParameter(jobTag + ".page")) + val parameterJobSortColumn = UIUtils.stripXSS(request.getParameter(jobTag + ".sort")) + val parameterJobSortDesc = UIUtils.stripXSS(request.getParameter(jobTag + ".desc")) + val parameterJobPageSize = UIUtils.stripXSS(request.getParameter(jobTag + ".pageSize")) + val parameterJobPrevPageSize = UIUtils.stripXSS(request.getParameter(jobTag + ".prevPageSize")) val jobPage = Option(parameterJobPage).map(_.toInt).getOrElse(1) val jobSortColumn = Option(parameterJobSortColumn).map { sortColumn => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 0ff9e5e9411ca..c0b1ed934666c 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 @@ -187,7 +187,8 @@ private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { val listener = parent.jobProgresslistener listener.synchronized { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index 620c54c2dc0a5..cc173381879a6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { @@ -40,7 +40,8 @@ private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - val jobId = Option(request.getParameter("id")).map(_.toInt) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) jobId.foreach { id => if (jobProgresslistener.activeJobs.contains(id)) { sc.foreach(_.cancelJob(id)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 8ee70d27cc09f..b164f32b62e97 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -31,7 +31,8 @@ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val poolName = Option(request.getParameter("poolname")).map { poolname => + // stripXSS is called first to remove suspicious characters used in XSS attacks + val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => UIUtils.decodeURLParameter(poolname) }.getOrElse { throw new IllegalArgumentException(s"Missing poolname parameter") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 412ddfa9fad35..1db4d8b003688 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 @@ -87,17 +87,18 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { def render(request: HttpServletRequest): Seq[Node] = { progressListener.synchronized { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterAttempt = request.getParameter("attempt") + val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - val parameterTaskPage = request.getParameter("task.page") - val parameterTaskSortColumn = request.getParameter("task.sort") - val parameterTaskSortDesc = request.getParameter("task.desc") - val parameterTaskPageSize = request.getParameter("task.pageSize") - val parameterTaskPrevPageSize = request.getParameter("task.prevPageSize") + val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) + val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) + val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) + val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) + val parameterTaskPrevPageSize = UIUtils.stripXSS(request.getParameter("task.prevPageSize")) val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e1fa9043b6a15..f08d7abb184d7 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -42,15 +42,17 @@ private[ui] class StageTableBase( isFairScheduler: Boolean, killEnabled: Boolean, isFailedStage: Boolean) { - val allParameters = request.getParameterMap().asScala.toMap + // stripXSS is called to remove suspicious characters used in XSS attacks + val allParameters = request.getParameterMap.asScala.toMap.mapValues(_.map(UIUtils.stripXSS)) val parameterOtherTable = allParameters.filterNot(_._1.startsWith(stageTag)) .map(para => para._1 + "=" + para._2(0)) - val parameterStagePage = request.getParameter(stageTag + ".page") - val parameterStageSortColumn = request.getParameter(stageTag + ".sort") - val parameterStageSortDesc = request.getParameter(stageTag + ".desc") - val parameterStagePageSize = request.getParameter(stageTag + ".pageSize") - val parameterStagePrevPageSize = request.getParameter(stageTag + ".prevPageSize") + val parameterStagePage = UIUtils.stripXSS(request.getParameter(stageTag + ".page")) + val parameterStageSortColumn = UIUtils.stripXSS(request.getParameter(stageTag + ".sort")) + val parameterStageSortDesc = UIUtils.stripXSS(request.getParameter(stageTag + ".desc")) + val parameterStagePageSize = UIUtils.stripXSS(request.getParameter(stageTag + ".pageSize")) + val parameterStagePrevPageSize = + UIUtils.stripXSS(request.getParameter(stageTag + ".prevPageSize")) val stagePage = Option(parameterStagePage).map(_.toInt).getOrElse(1) val stageSortColumn = Option(parameterStageSortColumn).map { sortColumn => @@ -512,4 +514,3 @@ private[ui] class StageDataSource( } } } - diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index c1f25114371f1..6baebb3eb3a51 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.{SparkUI, SparkUITab} +import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} /** Web UI showing progress status of all stages in the given SparkContext. */ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { @@ -39,7 +39,8 @@ private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages" def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - val stageId = Option(request.getParameter("id")).map(_.toInt) + // stripXSS is called first to remove suspicious characters used in XSS attacks + val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) stageId.foreach { id => if (progressListener.activeStages.contains(id)) { sc.foreach(_.cancelStage(id)) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 227e940c9c50c..68a0ef92a6062 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -31,14 +31,15 @@ private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val parameterId = request.getParameter("id") + // stripXSS is called first to remove suspicious characters used in XSS attacks + val parameterId = UIUtils.stripXSS(request.getParameter("id")) require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterBlockPage = request.getParameter("block.page") - val parameterBlockSortColumn = request.getParameter("block.sort") - val parameterBlockSortDesc = request.getParameter("block.desc") - val parameterBlockPageSize = request.getParameter("block.pageSize") - val parameterBlockPrevPageSize = request.getParameter("block.prevPageSize") + val parameterBlockPage = UIUtils.stripXSS(request.getParameter("block.page")) + val parameterBlockSortColumn = UIUtils.stripXSS(request.getParameter("block.sort")) + val parameterBlockSortDesc = UIUtils.stripXSS(request.getParameter("block.desc")) + val parameterBlockPageSize = UIUtils.stripXSS(request.getParameter("block.pageSize")) + val parameterBlockPrevPageSize = UIUtils.stripXSS(request.getParameter("block.prevPageSize")) val blockPage = Option(parameterBlockPage).map(_.toInt).getOrElse(1) val blockSortColumn = Option(parameterBlockSortColumn).getOrElse("Block Name") diff --git a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala index 6335d905c0fbf..5ae07a53271ab 100644 --- a/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala @@ -133,6 +133,45 @@ class UIUtilsSuite extends SparkFunSuite { assert(decoded2 === decodeURLParameter(decoded2)) } + test("SPARK-20393: Prevent newline characters in parameters.") { + val encoding = "Encoding:base64%0d%0a%0d%0aPGh0bWw%2bjcmlwdD48L2h0bWw%2b" + val stripEncoding = "Encoding:base64PGh0bWw%2bjcmlwdD48L2h0bWw%2b" + + assert(stripEncoding === stripXSS(encoding)) + } + + test("SPARK-20393: Prevent script from parameters running on page.") { + val scriptAlert = """>"'>