From 8b69b17f3fde2c890068a5a9ef490fe675cc80c1 Mon Sep 17 00:00:00 2001 From: zhoukang Date: Mon, 7 Aug 2017 12:51:39 +0100 Subject: [PATCH 01/20] [SPARK-21544][DEPLOY][TEST-MAVEN] Tests jar of some module should not upload twice ## What changes were proposed in this pull request? **For moudle below:** common/network-common streaming sql/core sql/catalyst **tests.jar will install or deploy twice.Like:** `[DEBUG] Installing org.apache.spark:spark-streaming_2.11/maven-metadata.xml to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/maven-metadata-local.xml [INFO] Installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar [DEBUG] Skipped re-installing /home/mi/Work/Spark/scala2.11/spark/streaming/target/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar to /home/mi/.m2/repository/org/apache/spark/spark-streaming_2.11/2.1.0-mdh2.1.0.1-SNAPSHOT/spark-streaming_2.11-2.1.0-mdh2.1.0.1-SNAPSHOT-tests.jar, seems unchanged` **The reason is below:** `[DEBUG] (f) artifact = org.apache.spark:spark-streaming_2.11:jar:2.1.0-mdh2.1.0.1-SNAPSHOT [DEBUG] (f) attachedArtifacts = [org.apache.spark:spark-streaming_2.11:test-jar:tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:jar:tests:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark -streaming_2.11:java-source:sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:java-source:test-sources:2.1.0-mdh2.1.0.1-SNAPSHOT, org.apache.spark:spark-streaming_2.11:javadoc:javadoc:2.1.0 -mdh2.1.0.1-SNAPSHOT]` when executing 'mvn deploy' to nexus during release.I will fail since release nexus can not be overrided. ## How was this patch tested? Execute 'mvn clean install -Pyarn -Phadoop-2.6 -Phadoop-provided -DskipTests' Author: zhoukang Closes #18745 from caneGuy/zhoukang/fix-installtwice. --- common/network-common/pom.xml | 2 +- sql/catalyst/pom.xml | 7 +------ sql/core/pom.xml | 7 +------ 3 files changed, 3 insertions(+), 13 deletions(-) diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 0254d0cefc368..ccd8504ba0d7b 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -126,7 +126,7 @@ maven-jar-plugin - test-jar-on-test-compile + prepare-test-jar test-compile test-jar diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0bbf7a95124cf..fce81493795c8 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -118,12 +118,7 @@ maven-jar-plugin - - test-jar - - - - test-jar-on-test-compile + prepare-test-jar test-compile test-jar diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 661c31ded7148..25004e5326097 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -174,12 +174,7 @@ maven-jar-plugin - - test-jar - - - - test-jar-on-test-compile + prepare-test-jar test-compile test-jar From bbfd6b5d24be5919a3ab1ac3eaec46e33201df39 Mon Sep 17 00:00:00 2001 From: Xiao Li Date: Tue, 8 Aug 2017 00:00:01 +0800 Subject: [PATCH 02/20] [SPARK-21647][SQL] Fix SortMergeJoin when using CROSS ### What changes were proposed in this pull request? author: BoleynSu closes https://github.com/apache/spark/pull/18836 ```Scala val df = Seq((1, 1)).toDF("i", "j") df.createOrReplaceTempView("T") withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { sql("select * from (select a.i from T a cross join T t where t.i = a.i) as t1 " + "cross join T t2 where t2.i = t1.i").explain(true) } ``` The above code could cause the following exception: ``` SortMergeJoinExec should not take Cross as the JoinType java.lang.IllegalArgumentException: SortMergeJoinExec should not take Cross as the JoinType at org.apache.spark.sql.execution.joins.SortMergeJoinExec.outputOrdering(SortMergeJoinExec.scala:100) ``` Our SortMergeJoinExec supports CROSS. We should not hit such an exception. This PR is to fix the issue. ### How was this patch tested? Modified the two existing test cases. Author: Xiao Li Author: Boleyn Su Closes #18863 from gatorsmile/pr-18836. --- .../execution/joins/SortMergeJoinExec.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 36 ++++++++++--------- 2 files changed, 21 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index 639b8e00c121b..f41fa14213df5 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 @@ -82,7 +82,7 @@ case class SortMergeJoinExec( override def outputOrdering: Seq[SortOrder] = joinType match { // For inner join, orders of both sides keys should be kept. - case Inner => + case _: InnerLike => val leftKeyOrdering = getKeyOrdering(leftKeys, left.outputOrdering) val rightKeyOrdering = getKeyOrdering(rightKeys, right.outputOrdering) leftKeyOrdering.zip(rightKeyOrdering).map { case (lKey, rKey) => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 4d155d538d637..63e17c7f372b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{execution, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOuter} +import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, LeftOuter, RightOuter} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Repartition} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.columnar.InMemoryRelation @@ -513,26 +513,30 @@ class PlannerSuite extends SharedSQLContext { } test("EnsureRequirements skips sort when either side of join keys is required after inner SMJ") { - val innerSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, Inner, None, planA, planB) - // Both left and right keys should be sorted after the SMJ. - Seq(orderingA, orderingB).foreach { ordering => - assertSortRequirementsAreSatisfied( - childPlan = innerSmj, - requiredOrdering = Seq(ordering), - shouldHaveSort = false) + Seq(Inner, Cross).foreach { joinType => + val innerSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, joinType, None, planA, planB) + // Both left and right keys should be sorted after the SMJ. + Seq(orderingA, orderingB).foreach { ordering => + assertSortRequirementsAreSatisfied( + childPlan = innerSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = false) + } } } test("EnsureRequirements skips sort when key order of a parent SMJ is propagated from its " + "child SMJ") { - val childSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, Inner, None, planA, planB) - val parentSmj = SortMergeJoinExec(exprB :: Nil, exprC :: Nil, Inner, None, childSmj, planC) - // After the second SMJ, exprA, exprB and exprC should all be sorted. - Seq(orderingA, orderingB, orderingC).foreach { ordering => - assertSortRequirementsAreSatisfied( - childPlan = parentSmj, - requiredOrdering = Seq(ordering), - shouldHaveSort = false) + Seq(Inner, Cross).foreach { joinType => + val childSmj = SortMergeJoinExec(exprA :: Nil, exprB :: Nil, joinType, None, planA, planB) + val parentSmj = SortMergeJoinExec(exprB :: Nil, exprC :: Nil, joinType, None, childSmj, planC) + // After the second SMJ, exprA, exprB and exprC should all be sorted. + Seq(orderingA, orderingB, orderingC).foreach { ordering => + assertSortRequirementsAreSatisfied( + childPlan = parentSmj, + requiredOrdering = Seq(ordering), + shouldHaveSort = false) + } } } From 4f7ec3a316fd2c6e0828d8777d29abb472fd7a14 Mon Sep 17 00:00:00 2001 From: Mac Date: Mon, 7 Aug 2017 17:16:03 +0100 Subject: [PATCH 03/20] [SPARK][DOCS] Added note on meaning of position to substring function ## What changes were proposed in this pull request? Enhanced some existing documentation Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Mac Closes #18710 from maclockard/maclockard-patch-1. --- python/pyspark/sql/functions.py | 4 +++- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 2 ++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 2c8c8e2d80f09..0e76182e0e02d 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1424,7 +1424,9 @@ def substring(str, pos, len): """ Substring starts at `pos` and is of length `len` when str is String type or returns the slice of byte array that starts at `pos` in byte and is of length `len` - when str is Binary type + when str is Binary type. + + .. note:: The position is not zero based, but 1 based index. >>> df = spark.createDataFrame([('abcd',)], ['s',]) >>> df.select(substring(df.s, 1, 2).alias('s')).collect() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index ccff00e570dbd..496619a43a503 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -2408,6 +2408,8 @@ object functions { * returns the slice of byte array that starts at `pos` in byte and is of length `len` * when str is Binary type * + * @note The position is not zero based, but 1 based index. + * * @group string_funcs * @since 1.5.0 */ From cce25b360ee9e39d9510134c73a1761475eaf4ac Mon Sep 17 00:00:00 2001 From: Jose Torres Date: Mon, 7 Aug 2017 12:27:16 -0700 Subject: [PATCH 04/20] [SPARK-21565][SS] Propagate metadata in attribute replacement. ## What changes were proposed in this pull request? Propagate metadata in attribute replacement during streaming execution. This is necessary for EventTimeWatermarks consuming replaced attributes. ## How was this patch tested? new unit test, which was verified to fail before the fix Author: Jose Torres Closes #18840 from joseph-torres/SPARK-21565. --- .../execution/streaming/StreamExecution.scala | 3 +- .../streaming/EventTimeWatermarkSuite.scala | 28 +++++++++++++++++++ 2 files changed, 30 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 5711262654a16..1528e7f469bef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -628,7 +628,8 @@ class StreamExecution( // Rewire the plan to use the new attributes that were returned by the source. val replacementMap = AttributeMap(replacements) val triggerLogicalPlan = withNewSources transformAllExpressions { - case a: Attribute if replacementMap.contains(a) => replacementMap(a) + case a: Attribute if replacementMap.contains(a) => + replacementMap(a).withMetadata(a.metadata) case ct: CurrentTimestamp => CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, ct.dataType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 552911f32ee43..4f19fa0bb4a97 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -391,6 +391,34 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche checkDataset[Long](df, 1L to 100L: _*) } + test("SPARK-21565: watermark operator accepts attributes from replacement") { + withTempDir { dir => + dir.delete() + + val df = Seq(("a", 100.0, new java.sql.Timestamp(100L))) + .toDF("symbol", "price", "eventTime") + df.write.json(dir.getCanonicalPath) + + val input = spark.readStream.schema(df.schema) + .json(dir.getCanonicalPath) + + val groupEvents = input + .withWatermark("eventTime", "2 seconds") + .groupBy("symbol", "eventTime") + .agg(count("price") as 'count) + .select("symbol", "eventTime", "count") + val q = groupEvents.writeStream + .outputMode("append") + .format("console") + .start() + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + } + private def assertNumStateRows(numTotalRows: Long): AssertOnQuery = AssertOnQuery { q => val progressWithData = q.recentProgress.filter(_.numInputRows > 0).lastOption.get assert(progressWithData.stateOperators(0).numRowsTotal === numTotalRows) From baf5cac0f8c35925c366464d7e0eb5f6023fce57 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 7 Aug 2017 13:04:04 -0700 Subject: [PATCH 05/20] [SPARK-21648][SQL] Fix confusing assert failure in JDBC source when parallel fetching parameters are not properly provided. ### What changes were proposed in this pull request? ```SQL CREATE TABLE mytesttable1 USING org.apache.spark.sql.jdbc OPTIONS ( url 'jdbc:mysql://${jdbcHostname}:${jdbcPort}/${jdbcDatabase}?user=${jdbcUsername}&password=${jdbcPassword}', dbtable 'mytesttable1', paritionColumn 'state_id', lowerBound '0', upperBound '52', numPartitions '53', fetchSize '10000' ) ``` The above option name `paritionColumn` is wrong. That mean, users did not provide the value for `partitionColumn`. In such case, users hit a confusing error. ``` AssertionError: assertion failed java.lang.AssertionError: assertion failed at scala.Predef$.assert(Predef.scala:156) at org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider.createRelation(JdbcRelationProvider.scala:39) at org.apache.spark.sql.execution.datasources.DataSource.resolveRelation(DataSource.scala:312) ``` ### How was this patch tested? Added a test case Author: gatorsmile Closes #18864 from gatorsmile/jdbcPartCol. --- .../datasources/jdbc/JDBCOptions.scala | 11 ++++++---- .../jdbc/JdbcRelationProvider.scala | 9 ++++++-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 22 +++++++++++++++++++ .../spark/sql/jdbc/JDBCWriteSuite.scala | 5 +++-- 4 files changed, 39 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 591096d5efd22..96a8a51da18e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -97,10 +97,13 @@ class JDBCOptions( val lowerBound = parameters.get(JDBC_LOWER_BOUND).map(_.toLong) // the upper bound of the partition column val upperBound = parameters.get(JDBC_UPPER_BOUND).map(_.toLong) - require(partitionColumn.isEmpty || - (lowerBound.isDefined && upperBound.isDefined && numPartitions.isDefined), - s"If '$JDBC_PARTITION_COLUMN' is specified then '$JDBC_LOWER_BOUND', '$JDBC_UPPER_BOUND'," + - s" and '$JDBC_NUM_PARTITIONS' are required.") + // numPartitions is also used for data source writing + require((partitionColumn.isEmpty && lowerBound.isEmpty && upperBound.isEmpty) || + (partitionColumn.isDefined && lowerBound.isDefined && upperBound.isDefined && + numPartitions.isDefined), + s"When reading JDBC data sources, users need to specify all or none for the following " + + s"options: '$JDBC_PARTITION_COLUMN', '$JDBC_LOWER_BOUND', '$JDBC_UPPER_BOUND', " + + s"and '$JDBC_NUM_PARTITIONS'") val fetchSize = { val size = parameters.getOrElse(JDBC_BATCH_FETCH_SIZE, "0").toInt require(size >= 0, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala index 74dcfb06f5c2b..37e7bb0a59bb6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcRelationProvider.scala @@ -29,6 +29,8 @@ class JdbcRelationProvider extends CreatableRelationProvider override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { + import JDBCOptions._ + val jdbcOptions = new JDBCOptions(parameters) val partitionColumn = jdbcOptions.partitionColumn val lowerBound = jdbcOptions.lowerBound @@ -36,10 +38,13 @@ class JdbcRelationProvider extends CreatableRelationProvider val numPartitions = jdbcOptions.numPartitions val partitionInfo = if (partitionColumn.isEmpty) { - assert(lowerBound.isEmpty && upperBound.isEmpty) + assert(lowerBound.isEmpty && upperBound.isEmpty, "When 'partitionColumn' is not specified, " + + s"'$JDBC_LOWER_BOUND' and '$JDBC_UPPER_BOUND' are expected to be empty") null } else { - assert(lowerBound.nonEmpty && upperBound.nonEmpty && numPartitions.nonEmpty) + assert(lowerBound.nonEmpty && upperBound.nonEmpty && numPartitions.nonEmpty, + s"When 'partitionColumn' is specified, '$JDBC_LOWER_BOUND', '$JDBC_UPPER_BOUND', and " + + s"'$JDBC_NUM_PARTITIONS' are also required") JDBCPartitioningInfo( partitionColumn.get, lowerBound.get, upperBound.get, numPartitions.get) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 24f46a6a057d9..4c43646889418 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -421,6 +421,28 @@ class JDBCSuite extends SparkFunSuite assert(e.contains("Invalid value `-1` for parameter `fetchsize`")) } + test("Missing partition columns") { + withView("tempPeople") { + val e = intercept[IllegalArgumentException] { + sql( + s""" + |CREATE OR REPLACE TEMPORARY VIEW tempPeople + |USING org.apache.spark.sql.jdbc + |OPTIONS ( + | url 'jdbc:h2:mem:testdb0;user=testUser;password=testPass', + | dbtable 'TEST.PEOPLE', + | lowerBound '0', + | upperBound '52', + | numPartitions '53', + | fetchSize '10000' ) + """.stripMargin.replaceAll("\n", " ")) + }.getMessage + assert(e.contains("When reading JDBC data sources, users need to specify all or none " + + "for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and " + + "'numPartitions'")) + } + } + test("Basic API with FetchSize") { (0 to 4).foreach { size => val properties = new Properties() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala index 2334d5ae32dc3..b7f97f204b24c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala @@ -324,8 +324,9 @@ class JDBCWriteSuite extends SharedSQLContext with BeforeAndAfter { .option("partitionColumn", "foo") .save() }.getMessage - assert(e.contains("If 'partitionColumn' is specified then 'lowerBound', 'upperBound'," + - " and 'numPartitions' are required.")) + assert(e.contains("When reading JDBC data sources, users need to specify all or none " + + "for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and " + + "'numPartitions'")) } test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { From fdcee028afa7a7ac0f8bd8f59ee4933d7caea064 Mon Sep 17 00:00:00 2001 From: Ajay Saini Date: Mon, 7 Aug 2017 17:03:20 -0700 Subject: [PATCH 06/20] [SPARK-21542][ML][PYTHON] Python persistence helper functions ## What changes were proposed in this pull request? Added DefaultParamsWriteable, DefaultParamsReadable, DefaultParamsWriter, and DefaultParamsReader to Python to support Python-only persistence of Json-serializable parameters. ## How was this patch tested? Instantiated an estimator with Json-serializable parameters (ex. LogisticRegression), saved it using the added helper functions, and loaded it back, and compared it to the original instance to make sure it is the same. This test was both done in the Python REPL and implemented in the unit tests. Note to reviewers: there are a few excess comments that I left in the code for clarity but will remove before the code is merged to master. Author: Ajay Saini Closes #18742 from ajaysaini725/PythonPersistenceHelperFunctions. --- .../org/apache/spark/ml/util/ReadWrite.scala | 37 ++- python/pyspark/ml/param/__init__.py | 11 + python/pyspark/ml/pipeline.py | 10 - python/pyspark/ml/tests.py | 34 ++ python/pyspark/ml/util.py | 302 ++++++++++++++++-- 5 files changed, 342 insertions(+), 52 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index b54e258cff2f8..65f142cfbbcb6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -96,21 +96,7 @@ abstract class MLWriter extends BaseReadWrite with Logging { @Since("1.6.0") @throws[IOException]("If the input path already exists but overwrite is not enabled.") def save(path: String): Unit = { - val hadoopConf = sc.hadoopConfiguration - val outputPath = new Path(path) - val fs = outputPath.getFileSystem(hadoopConf) - val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - if (fs.exists(qualifiedOutputPath)) { - if (shouldOverwrite) { - logInfo(s"Path $path already exists. It will be overwritten.") - // TODO: Revert back to the original content if save is not successful. - fs.delete(qualifiedOutputPath, true) - } else { - throw new IOException(s"Path $path already exists. To overwrite it, " + - s"please use write.overwrite().save(path) for Scala and use " + - s"write().overwrite().save(path) for Java and Python.") - } - } + new FileSystemOverwrite().handleOverwrite(path, shouldOverwrite, sc) saveImpl(path) } @@ -471,3 +457,24 @@ private[ml] object MetaAlgorithmReadWrite { List((instance.uid, instance)) ++ subStageMaps } } + +private[ml] class FileSystemOverwrite extends Logging { + + def handleOverwrite(path: String, shouldOverwrite: Boolean, sc: SparkContext): Unit = { + val hadoopConf = sc.hadoopConfiguration + val outputPath = new Path(path) + val fs = outputPath.getFileSystem(hadoopConf) + val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + if (fs.exists(qualifiedOutputPath)) { + if (shouldOverwrite) { + logInfo(s"Path $path already exists. It will be overwritten.") + // TODO: Revert back to the original content if save is not successful. + fs.delete(qualifiedOutputPath, true) + } else { + throw new IOException(s"Path $path already exists. To overwrite it, " + + s"please use write.overwrite().save(path) for Scala and use " + + s"write().overwrite().save(path) for Java and Python.") + } + } + } +} diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 4583ae85dc8fa..13342073b4988 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -384,6 +384,17 @@ def copy(self, extra=None): that._defaultParamMap = {} return self._copyValues(that, extra) + def set(self, param, value): + """ + Sets a parameter in the embedded param map. + """ + self._shouldOwn(param) + try: + value = param.typeConverter(value) + except ValueError as e: + raise ValueError('Invalid param value given for param "%s". %s' % (param.name, e)) + self._paramMap[param] = value + def _shouldOwn(self, param): """ Validates that the input param belongs to this Params instance. diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 4aac6a4466b54..a8dc76b846c24 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -132,11 +132,6 @@ def write(self): """Returns an MLWriter instance for this ML instance.""" return JavaMLWriter(self) - @since("2.0.0") - def save(self, path): - """Save this ML instance to the given path, a shortcut of `write().save(path)`.""" - self.write().save(path) - @classmethod @since("2.0.0") def read(cls): @@ -211,11 +206,6 @@ def write(self): """Returns an MLWriter instance for this ML instance.""" return JavaMLWriter(self) - @since("2.0.0") - def save(self, path): - """Save this ML instance to the given path, a shortcut of `write().save(path)`.""" - self.write().save(path) - @classmethod @since("2.0.0") def read(cls): diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index 3bd4d3737a056..6aecc7fe87074 100755 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -62,6 +62,7 @@ LinearRegression from pyspark.ml.stat import ChiSquareTest from pyspark.ml.tuning import * +from pyspark.ml.util import * from pyspark.ml.wrapper import JavaParams, JavaWrapper from pyspark.serializers import PickleSerializer from pyspark.sql import DataFrame, Row, SparkSession @@ -377,6 +378,12 @@ def test_params(self): with self.assertRaises(KeyError): testParams.getInputCol() + otherParam = Param(Params._dummy(), "otherParam", "Parameter used to test that " + + "set raises an error for a non-member parameter.", + typeConverter=TypeConverters.toString) + with self.assertRaises(ValueError): + testParams.set(otherParam, "value") + # Since the default is normally random, set it to a known number for debug str testParams._setDefault(seed=41) testParams.setSeed(43) @@ -1189,6 +1196,33 @@ def test_decisiontree_regressor(self): except OSError: pass + def test_default_read_write(self): + temp_path = tempfile.mkdtemp() + + lr = LogisticRegression() + lr.setMaxIter(50) + lr.setThreshold(.75) + writer = DefaultParamsWriter(lr) + + savePath = temp_path + "/lr" + writer.save(savePath) + + reader = DefaultParamsReadable.read() + lr2 = reader.load(savePath) + + self.assertEqual(lr.uid, lr2.uid) + self.assertEqual(lr.extractParamMap(), lr2.extractParamMap()) + + # test overwrite + lr.setThreshold(.8) + writer.overwrite().save(savePath) + + reader = DefaultParamsReadable.read() + lr3 = reader.load(savePath) + + self.assertEqual(lr.uid, lr3.uid) + self.assertEqual(lr.extractParamMap(), lr3.extractParamMap()) + class LDATest(SparkSessionTestCase): diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 7863edda7e7ab..67772910c0d38 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -15,16 +15,21 @@ # limitations under the License. # +import json import sys +import os +import time import uuid import warnings if sys.version > '3': basestring = str unicode = str + long = int from pyspark import SparkContext, since from pyspark.ml.common import inherit_doc +from pyspark.sql import SparkSession def _jvm(): @@ -61,33 +66,86 @@ def _randomUID(cls): @inherit_doc -class MLWriter(object): +class BaseReadWrite(object): """ - Utility class that can save ML instances. + Base class for MLWriter and MLReader. Stores information about the SparkContext + and SparkSession. - .. versionadded:: 2.0.0 + .. versionadded:: 2.3.0 """ - def save(self, path): - """Save the ML instance to the input path.""" - raise NotImplementedError("MLWriter is not yet implemented for type: %s" % type(self)) - - def overwrite(self): - """Overwrites if the output path already exists.""" - raise NotImplementedError("MLWriter is not yet implemented for type: %s" % type(self)) + def __init__(self): + self._sparkSession = None def context(self, sqlContext): """ - Sets the SQL context to use for saving. + Sets the Spark SQLContext to use for saving/loading. .. note:: Deprecated in 2.1 and will be removed in 3.0, use session instead. """ - raise NotImplementedError("MLWriter is not yet implemented for type: %s" % type(self)) + raise NotImplementedError("Read/Write is not yet implemented for type: %s" % type(self)) def session(self, sparkSession): - """Sets the Spark Session to use for saving.""" + """ + Sets the Spark Session to use for saving/loading. + """ + self._sparkSession = sparkSession + return self + + @property + def sparkSession(self): + """ + Returns the user-specified Spark Session or the default. + """ + if self._sparkSession is None: + self._sparkSession = SparkSession.builder.getOrCreate() + return self._sparkSession + + @property + def sc(self): + """ + Returns the underlying `SparkContext`. + """ + return self.sparkSession.sparkContext + + +@inherit_doc +class MLWriter(BaseReadWrite): + """ + Utility class that can save ML instances. + + .. versionadded:: 2.0.0 + """ + + def __init__(self): + super(MLWriter, self).__init__() + self.shouldOverwrite = False + + def _handleOverwrite(self, path): + from pyspark.ml.wrapper import JavaWrapper + + _java_obj = JavaWrapper._new_java_obj("org.apache.spark.ml.util.FileSystemOverwrite") + wrapper = JavaWrapper(_java_obj) + wrapper._call_java("handleOverwrite", path, True, self.sc._jsc.sc()) + + def save(self, path): + """Save the ML instance to the input path.""" + if self.shouldOverwrite: + self._handleOverwrite(path) + self.saveImpl(path) + + def saveImpl(self, path): + """ + save() handles overwriting and then calls this method. Subclasses should override this + method to implement the actual saving of the instance. + """ raise NotImplementedError("MLWriter is not yet implemented for type: %s" % type(self)) + def overwrite(self): + """Overwrites if the output path already exists.""" + self.shouldOverwrite = True + return self + @inherit_doc class JavaMLWriter(MLWriter): @@ -140,7 +198,7 @@ def write(self): raise NotImplementedError("MLWritable is not yet implemented for type: %r" % type(self)) def save(self, path): - """Save this ML instance to the given path, a shortcut of `write().save(path)`.""" + """Save this ML instance to the given path, a shortcut of 'write().save(path)'.""" self.write().save(path) @@ -156,29 +214,20 @@ def write(self): @inherit_doc -class MLReader(object): +class MLReader(BaseReadWrite): """ Utility class that can load ML instances. .. versionadded:: 2.0.0 """ + def __init__(self): + super(MLReader, self).__init__() + def load(self, path): """Load the ML instance from the input path.""" raise NotImplementedError("MLReader is not yet implemented for type: %s" % type(self)) - def context(self, sqlContext): - """ - Sets the SQL context to use for loading. - - .. note:: Deprecated in 2.1 and will be removed in 3.0, use session instead. - """ - raise NotImplementedError("MLReader is not yet implemented for type: %s" % type(self)) - - def session(self, sparkSession): - """Sets the Spark Session to use for loading.""" - raise NotImplementedError("MLReader is not yet implemented for type: %s" % type(self)) - @inherit_doc class JavaMLReader(MLReader): @@ -187,6 +236,7 @@ class JavaMLReader(MLReader): """ def __init__(self, clazz): + super(JavaMLReader, self).__init__() self._clazz = clazz self._jread = self._load_java_obj(clazz).read() @@ -283,3 +333,201 @@ def numFeatures(self): Returns the number of features the model was trained on. If unknown, returns -1 """ return self._call_java("numFeatures") + + +@inherit_doc +class DefaultParamsWritable(MLWritable): + """ + .. note:: DeveloperApi + + Helper trait for making simple :py:class:`Params` types writable. If a :py:class:`Params` + class stores all data as :py:class:`Param` values, then extending this trait will provide + a default implementation of writing saved instances of the class. + This only handles simple :py:class:`Param` types; e.g., it will not handle + :py:class:`Dataset`. See :py:class:`DefaultParamsReadable`, the counterpart to this trait. + + .. versionadded:: 2.3.0 + """ + + def write(self): + """Returns a DefaultParamsWriter instance for this class.""" + from pyspark.ml.param import Params + + if isinstance(self, Params): + return DefaultParamsWriter(self) + else: + raise TypeError("Cannot use DefautParamsWritable with type %s because it does not " + + " extend Params.", type(self)) + + +@inherit_doc +class DefaultParamsWriter(MLWriter): + """ + .. note:: DeveloperApi + + Specialization of :py:class:`MLWriter` for :py:class:`Params` types + + Class for writing Estimators and Transformers whose parameters are JSON-serializable. + + .. versionadded:: 2.3.0 + """ + + def __init__(self, instance): + super(DefaultParamsWriter, self).__init__() + self.instance = instance + + def saveImpl(self, path): + DefaultParamsWriter.saveMetadata(self.instance, path, self.sc) + + @staticmethod + def saveMetadata(instance, path, sc, extraMetadata=None, paramMap=None): + """ + Saves metadata + Params to: path + "/metadata" + - class + - timestamp + - sparkVersion + - uid + - paramMap + - (optionally, extra metadata) + :param extraMetadata: Extra metadata to be saved at same level as uid, paramMap, etc. + :param paramMap: If given, this is saved in the "paramMap" field. + """ + metadataPath = os.path.join(path, "metadata") + metadataJson = DefaultParamsWriter._get_metadata_to_save(instance, + sc, + extraMetadata, + paramMap) + sc.parallelize([metadataJson], 1).saveAsTextFile(metadataPath) + + @staticmethod + def _get_metadata_to_save(instance, sc, extraMetadata=None, paramMap=None): + """ + Helper for :py:meth:`DefaultParamsWriter.saveMetadata` which extracts the JSON to save. + This is useful for ensemble models which need to save metadata for many sub-models. + + .. note:: :py:meth:`DefaultParamsWriter.saveMetadata` for details on what this includes. + """ + uid = instance.uid + cls = instance.__module__ + '.' + instance.__class__.__name__ + params = instance.extractParamMap() + jsonParams = {} + if paramMap is not None: + jsonParams = paramMap + else: + for p in params: + jsonParams[p.name] = params[p] + basicMetadata = {"class": cls, "timestamp": long(round(time.time() * 1000)), + "sparkVersion": sc.version, "uid": uid, "paramMap": jsonParams} + if extraMetadata is not None: + basicMetadata.update(extraMetadata) + return json.dumps(basicMetadata, separators=[',', ':']) + + +@inherit_doc +class DefaultParamsReadable(MLReadable): + """ + .. note:: DeveloperApi + + Helper trait for making simple :py:class:`Params` types readable. + If a :py:class:`Params` class stores all data as :py:class:`Param` values, + then extending this trait will provide a default implementation of reading saved + instances of the class. This only handles simple :py:class:`Param` types; + e.g., it will not handle :py:class:`Dataset`. See :py:class:`DefaultParamsWritable`, + the counterpart to this trait. + + .. versionadded:: 2.3.0 + """ + + @classmethod + def read(cls): + """Returns a DefaultParamsReader instance for this class.""" + return DefaultParamsReader(cls) + + +@inherit_doc +class DefaultParamsReader(MLReader): + """ + .. note:: DeveloperApi + + Specialization of :py:class:`MLReader` for :py:class:`Params` types + + Default :py:class:`MLReader` implementation for transformers and estimators that + contain basic (json-serializable) params and no data. This will not handle + more complex params or types with data (e.g., models with coefficients). + + .. versionadded:: 2.3.0 + """ + + def __init__(self, cls): + super(DefaultParamsReader, self).__init__() + self.cls = cls + + @staticmethod + def __get_class(clazz): + """ + Loads Python class from its name. + """ + parts = clazz.split('.') + module = ".".join(parts[:-1]) + m = __import__(module) + for comp in parts[1:]: + m = getattr(m, comp) + return m + + def load(self, path): + metadata = DefaultParamsReader.loadMetadata(path, self.sc) + py_type = DefaultParamsReader.__get_class(metadata['class']) + instance = py_type() + instance._resetUid(metadata['uid']) + DefaultParamsReader.getAndSetParams(instance, metadata) + return instance + + @staticmethod + def loadMetadata(path, sc, expectedClassName=""): + """ + Load metadata saved using :py:meth:`DefaultParamsWriter.saveMetadata` + + :param expectedClassName: If non empty, this is checked against the loaded metadata. + """ + metadataPath = os.path.join(path, "metadata") + metadataStr = sc.textFile(metadataPath, 1).first() + loadedVals = DefaultParamsReader._parseMetaData(metadataStr, expectedClassName) + return loadedVals + + @staticmethod + def _parseMetaData(metadataStr, expectedClassName=""): + """ + Parse metadata JSON string produced by :py:meth`DefaultParamsWriter._get_metadata_to_save`. + This is a helper function for :py:meth:`DefaultParamsReader.loadMetadata`. + + :param metadataStr: JSON string of metadata + :param expectedClassName: If non empty, this is checked against the loaded metadata. + """ + metadata = json.loads(metadataStr) + className = metadata['class'] + if len(expectedClassName) > 0: + assert className == expectedClassName, "Error loading metadata: Expected " + \ + "class name {} but found class name {}".format(expectedClassName, className) + return metadata + + @staticmethod + def getAndSetParams(instance, metadata): + """ + Extract Params from metadata, and set them in the instance. + """ + for paramName in metadata['paramMap']: + param = instance.getParam(paramName) + paramValue = metadata['paramMap'][paramName] + instance.set(param, paramValue) + + @staticmethod + def loadParamsInstance(path, sc): + """ + Load a :py:class:`Params` instance from the given path, and return it. + This assumes the instance inherits from :py:class:`MLReadable`. + """ + metadata = DefaultParamsReader.loadMetadata(path, sc) + pythonClassName = metadata['class'].replace("org.apache.spark", "pyspark") + py_type = DefaultParamsReader.__get_class(pythonClassName) + instance = py_type.load(path) + return instance From f763d8464b32852d7fd33e962e5476a7f03bc6c6 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Tue, 8 Aug 2017 08:43:58 +0800 Subject: [PATCH 07/20] [SPARK-19270][FOLLOW-UP][ML] PySpark GLR model.summary should return a printable representation. ## What changes were proposed in this pull request? PySpark GLR ```model.summary``` should return a printable representation by calling Scala ```toString```. ## How was this patch tested? ``` from pyspark.ml.regression import GeneralizedLinearRegression dataset = spark.read.format("libsvm").load("data/mllib/sample_linear_regression_data.txt") glr = GeneralizedLinearRegression(family="gaussian", link="identity", maxIter=10, regParam=0.3) model = glr.fit(dataset) model.summary ``` Before this PR: ![image](https://user-images.githubusercontent.com/1962026/29021059-e221633e-7b96-11e7-8d77-5d53f89c81a9.png) After this PR: ![image](https://user-images.githubusercontent.com/1962026/29021097-fce80fa6-7b96-11e7-8ab4-7e113d447d5d.png) Author: Yanbo Liang Closes #18870 from yanboliang/spark-19270. --- python/pyspark/ml/regression.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 2cc623427edc8..72374acbe019f 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1745,6 +1745,9 @@ def pValues(self): """ return self._call_java("pValues") + def __repr__(self): + return self._call_java("toString") + if __name__ == "__main__": import doctest From 312bebfb6d9e3fc8d48d3c1f7509ba05059bd8b0 Mon Sep 17 00:00:00 2001 From: "Marcos P. Sanchez" Date: Mon, 7 Aug 2017 22:41:57 -0700 Subject: [PATCH 08/20] [SPARK-21640][FOLLOW-UP][SQL] added errorifexists on IllegalArgumentException message ## What changes were proposed in this pull request? This commit adds a new argument for IllegalArgumentException message. This recent commit added the argument: [https://github.com/apache/spark/commit/dcac1d57f0fd05605edf596c303546d83062a352](https://github.com/apache/spark/commit/dcac1d57f0fd05605edf596c303546d83062a352) ## How was this patch tested? Unit test have been passed Please review http://spark.apache.org/contributing.html before opening a pull request. Author: Marcos P. Sanchez Closes #18862 from mpenate/feature/exception-errorifexists. --- docs/sql-programming-guide.md | 2 +- .../src/main/scala/org/apache/spark/sql/DataFrameWriter.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 7f7cf59b7a9a8..2ac2383d699c4 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -514,7 +514,7 @@ new data. Scala/JavaAny LanguageMeaning SaveMode.ErrorIfExists (default) - "error" (default) + "error" or "errorifexists" (default) When saving a DataFrame to a data source, if data already exists, an exception is expected to be thrown. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 079f699a181f5..65c9ef40777a2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -73,7 +73,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { case "ignore" => SaveMode.Ignore case "error" | "errorifexists" | "default" => SaveMode.ErrorIfExists case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " + - "Accepted save modes are 'overwrite', 'append', 'ignore', 'error'.") + "Accepted save modes are 'overwrite', 'append', 'ignore', 'error', 'errorifexists'.") } this } From ee1304199bcd9c1d5fc94f5b06fdd5f6fe7336a1 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 8 Aug 2017 16:12:41 +0800 Subject: [PATCH 09/20] [SPARK-21567][SQL] Dataset should work with type alias ## What changes were proposed in this pull request? If we create a type alias for a type workable with Dataset, the type alias doesn't work with Dataset. A reproducible case looks like: object C { type TwoInt = (Int, Int) def tupleTypeAlias: TwoInt = (1, 1) } Seq(1).toDS().map(_ => ("", C.tupleTypeAlias)) It throws an exception like: type T1 is not a class scala.ScalaReflectionException: type T1 is not a class at scala.reflect.api.Symbols$SymbolApi$class.asClass(Symbols.scala:275) ... This patch accesses the dealias of type in many places in `ScalaReflection` to fix it. ## How was this patch tested? Added test case. Author: Liang-Chi Hsieh Closes #18813 from viirya/SPARK-21567. --- .../spark/sql/catalyst/ScalaReflection.scala | 27 ++++++++++--------- .../org/apache/spark/sql/DatasetSuite.scala | 24 +++++++++++++++++ 2 files changed, 38 insertions(+), 13 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 004b4ef8f69fe..17e595f9c5d8d 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 @@ -62,7 +62,7 @@ object ScalaReflection extends ScalaReflection { def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T]) private def dataTypeFor(tpe: `Type`): DataType = { - tpe match { + tpe.dealias match { case t if t <:< definitions.IntTpe => IntegerType case t if t <:< definitions.LongTpe => LongType case t if t <:< definitions.DoubleTpe => DoubleType @@ -94,7 +94,7 @@ object ScalaReflection extends ScalaReflection { * JVM form instead of the Scala Array that handles auto boxing. */ private def arrayClassFor(tpe: `Type`): ObjectType = { - val cls = tpe match { + val cls = tpe.dealias match { case t if t <:< definitions.IntTpe => classOf[Array[Int]] case t if t <:< definitions.LongTpe => classOf[Array[Long]] case t if t <:< definitions.DoubleTpe => classOf[Array[Double]] @@ -193,7 +193,7 @@ object ScalaReflection extends ScalaReflection { case _ => UpCast(expr, expected, walkedTypePath) } - tpe match { + tpe.dealias match { case t if !dataTypeFor(t).isInstanceOf[ObjectType] => getPath case t if t <:< localTypeOf[Option[_]] => @@ -469,7 +469,7 @@ object ScalaReflection extends ScalaReflection { } } - tpe match { + tpe.dealias match { case _ if !inputObject.dataType.isInstanceOf[ObjectType] => inputObject case t if t <:< localTypeOf[Option[_]] => @@ -643,7 +643,7 @@ object ScalaReflection extends ScalaReflection { * we also treat [[DefinedByConstructorParams]] as product type. */ def optionOfProductType(tpe: `Type`): Boolean = { - tpe match { + tpe.dealias match { case t if t <:< localTypeOf[Option[_]] => val TypeRef(_, _, Seq(optType)) = t definedByConstructorParams(optType) @@ -690,7 +690,7 @@ object ScalaReflection extends ScalaReflection { /* * Retrieves the runtime class corresponding to the provided type. */ - def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.typeSymbol.asClass) + def getClassFromType(tpe: Type): Class[_] = mirror.runtimeClass(tpe.dealias.typeSymbol.asClass) case class Schema(dataType: DataType, nullable: Boolean) @@ -705,7 +705,7 @@ object ScalaReflection extends ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = { - tpe match { + tpe.dealias match { case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() Schema(udt, nullable = true) @@ -775,7 +775,7 @@ object ScalaReflection extends ScalaReflection { * Whether the fields of the given type is defined entirely by its constructor parameters. */ def definedByConstructorParams(tpe: Type): Boolean = { - tpe <:< localTypeOf[Product] || tpe <:< localTypeOf[DefinedByConstructorParams] + tpe.dealias <:< localTypeOf[Product] || tpe.dealias <:< localTypeOf[DefinedByConstructorParams] } private val javaKeywords = Set("abstract", "assert", "boolean", "break", "byte", "case", "catch", @@ -829,7 +829,7 @@ trait ScalaReflection { * synthetic classes, emulating behaviour in Java bytecode. */ def getClassNameFromType(tpe: `Type`): String = { - tpe.erasure.typeSymbol.asClass.fullName + tpe.dealias.erasure.typeSymbol.asClass.fullName } /** @@ -848,9 +848,10 @@ trait ScalaReflection { * support inner class. */ def getConstructorParameters(tpe: Type): Seq[(String, Type)] = { - val formalTypeArgs = tpe.typeSymbol.asClass.typeParams - val TypeRef(_, _, actualTypeArgs) = tpe - val params = constructParams(tpe) + val dealiasedTpe = tpe.dealias + val formalTypeArgs = dealiasedTpe.typeSymbol.asClass.typeParams + val TypeRef(_, _, actualTypeArgs) = dealiasedTpe + val params = constructParams(dealiasedTpe) // if there are type variables to fill in, do the substitution (SomeClass[T] -> SomeClass[Int]) if (actualTypeArgs.nonEmpty) { params.map { p => @@ -864,7 +865,7 @@ trait ScalaReflection { } protected def constructParams(tpe: Type): Seq[Symbol] = { - val constructorSymbol = tpe.member(termNames.CONSTRUCTOR) + val constructorSymbol = tpe.dealias.member(termNames.CONSTRUCTOR) val params = if (constructorSymbol.isMethod) { constructorSymbol.asMethod.paramLists } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 40235e32d35da..6245b2eff9fa1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -34,6 +34,16 @@ import org.apache.spark.sql.types._ case class TestDataPoint(x: Int, y: Double, s: String, t: TestDataPoint2) case class TestDataPoint2(x: Int, s: String) +object TestForTypeAlias { + type TwoInt = (Int, Int) + type ThreeInt = (TwoInt, Int) + type SeqOfTwoInt = Seq[TwoInt] + + def tupleTypeAlias: TwoInt = (1, 1) + def nestedTupleTypeAlias: ThreeInt = ((1, 1), 2) + def seqOfTupleTypeAlias: SeqOfTwoInt = Seq((1, 1), (2, 2)) +} + class DatasetSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -1317,6 +1327,20 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkAnswer(df.orderBy($"id"), expected) checkAnswer(df.orderBy('id), expected) } + + test("SPARK-21567: Dataset should work with type alias") { + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.tupleTypeAlias)), + ("", (1, 1))) + + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.nestedTupleTypeAlias)), + ("", ((1, 1), 2))) + + checkDataset( + Seq(1).toDS().map(_ => ("", TestForTypeAlias.seqOfTupleTypeAlias)), + ("", Seq((1, 1), (2, 2)))) + } } case class WithImmutableMap(id: String, map_test: scala.collection.immutable.Map[Long, String]) From 08ef7d71875378e324dd23c6d2739e606799c818 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 8 Aug 2017 23:18:59 +0900 Subject: [PATCH 10/20] [MINOR][R][BUILD] More reliable detection of R version for Windows in AppVeyor ## What changes were proposed in this pull request? This PR proposes to use https://rversions.r-pkg.org/r-release-win instead of https://rversions.r-pkg.org/r-release to check R's version for Windows correctly. We met a syncing problem with Windows release (see #15709) before. To cut this short, it was ... - 3.3.2 release was released but not for Windows for few hours. - `https://rversions.r-pkg.org/r-release` returns the latest as 3.3.2 and the download link for 3.3.1 becomes `windows/base/old` by our script - 3.3.2 release for WIndows yet - 3.3.1 is still not in `windows/base/old` but `windows/base` as the latest - Failed to download with `windows/base/old` link and builds were broken I believe this problem is not only what we met. Please see https://github.com/krlmlr/r-appveyor/commit/01ce943929993bbf27facd2cdc20ae2e03808eb4 and also this `r-release-win` API came out between 3.3.1 and 3.3.2 (assuming to deal with this issue), please see `https://github.com/metacran/rversions.app/issues/2`. Using this API will prevent the problem although it looks quite rare assuming from the commit logs in https://github.com/metacran/rversions.app/commits/master. After 3.3.2, both `r-release-win` and `r-release` are being updated together. ## How was this patch tested? AppVeyor tests. Author: hyukjinkwon Closes #18859 from HyukjinKwon/use-reliable-link. --- dev/appveyor-install-dependencies.ps1 | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/appveyor-install-dependencies.ps1 b/dev/appveyor-install-dependencies.ps1 index cf82389173048..a357fbf59f6c8 100644 --- a/dev/appveyor-install-dependencies.ps1 +++ b/dev/appveyor-install-dependencies.ps1 @@ -26,7 +26,7 @@ Function InstallR { } $urlPath = "" - $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest http://rversions.r-pkg.org/r-release).Content).version + $latestVer = $(ConvertFrom-JSON $(Invoke-WebRequest http://rversions.r-pkg.org/r-release-win).Content).version If ($rVer -ne $latestVer) { $urlPath = ("old/" + $rVer + "/") } From 979bf946d5fc7c15c2fdaa2e6c4df07bbbb74d93 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 8 Aug 2017 11:02:54 -0700 Subject: [PATCH 11/20] [SPARK-20655][CORE] In-memory KVStore implementation. This change adds an in-memory implementation of KVStore that can be used by the live UI. The implementation is not fully optimized, neither for speed nor space, but should be fast enough for using in the listener bus. Author: Marcelo Vanzin Closes #18395 from vanzin/SPARK-20655. --- common/kvstore/pom.xml | 5 + .../apache/spark/kvstore/ArrayWrappers.java | 213 ++++++++++++ .../apache/spark/kvstore/InMemoryStore.java | 320 ++++++++++++++++++ .../org/apache/spark/kvstore/KVTypeInfo.java | 2 +- .../spark/kvstore/ArrayKeyIndexType.java | 44 +++ .../spark/kvstore/ArrayWrappersSuite.java | 59 ++++ .../spark/kvstore/InMemoryIteratorSuite.java | 27 ++ .../spark/kvstore/InMemoryStoreSuite.java | 161 +++++++++ .../apache/spark/kvstore/LevelDBSuite.java | 24 -- project/SparkBuild.scala | 3 +- 10 files changed, 832 insertions(+), 26 deletions(-) create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java create mode 100644 common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java create mode 100644 common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index d00cf2788b964..cf93d41cd77cf 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -35,6 +35,11 @@ + + org.apache.spark + spark-tags_${scala.binary.version} + + com.google.guava guava diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java new file mode 100644 index 0000000000000..5efa842e3aad7 --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java @@ -0,0 +1,213 @@ +/* + * 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.kvstore; + +import java.util.Arrays; + +import com.google.common.base.Preconditions; + +/** + * A factory for array wrappers so that arrays can be used as keys in a map, sorted or not. + * + * The comparator implementation makes two assumptions: + * - All elements are instances of Comparable + * - When comparing two arrays, they both contain elements of the same type in corresponding + * indices. + * + * Otherwise, ClassCastExceptions may occur. The equality method can compare any two arrays. + * + * This class is not efficient and is mostly meant to compare really small arrays, like those + * generally used as indices and keys in a KVStore. + */ +class ArrayWrappers { + + @SuppressWarnings("unchecked") + public static Comparable forArray(Object a) { + Preconditions.checkArgument(a.getClass().isArray()); + Comparable ret; + if (a instanceof int[]) { + ret = new ComparableIntArray((int[]) a); + } else if (a instanceof long[]) { + ret = new ComparableLongArray((long[]) a); + } else if (a instanceof byte[]) { + ret = new ComparableByteArray((byte[]) a); + } else { + Preconditions.checkArgument(!a.getClass().getComponentType().isPrimitive()); + ret = new ComparableObjectArray((Object[]) a); + } + return (Comparable) ret; + } + + private static class ComparableIntArray implements Comparable { + + private final int[] array; + + ComparableIntArray(int[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableIntArray)) { + return false; + } + return Arrays.equals(array, ((ComparableIntArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableIntArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = array[i] - other.array[i]; + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableLongArray implements Comparable { + + private final long[] array; + + ComparableLongArray(long[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableLongArray)) { + return false; + } + return Arrays.equals(array, ((ComparableLongArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + (int) array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableLongArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + long diff = array[i] - other.array[i]; + if (diff != 0) { + return diff > 0 ? 1 : -1; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableByteArray implements Comparable { + + private final byte[] array; + + ComparableByteArray(byte[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableByteArray)) { + return false; + } + return Arrays.equals(array, ((ComparableByteArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i]; + } + return code; + } + + @Override + public int compareTo(ComparableByteArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = array[i] - other.array[i]; + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + + private static class ComparableObjectArray implements Comparable { + + private final Object[] array; + + ComparableObjectArray(Object[] array) { + this.array = array; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof ComparableObjectArray)) { + return false; + } + return Arrays.equals(array, ((ComparableObjectArray) other).array); + } + + @Override + public int hashCode() { + int code = 0; + for (int i = 0; i < array.length; i++) { + code = (code * 31) + array[i].hashCode(); + } + return code; + } + + @Override + @SuppressWarnings("unchecked") + public int compareTo(ComparableObjectArray other) { + int len = Math.min(array.length, other.array.length); + for (int i = 0; i < len; i++) { + int diff = ((Comparable) array[i]).compareTo((Comparable) other.array[i]); + if (diff != 0) { + return diff; + } + } + + return array.length - other.array.length; + } + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java new file mode 100644 index 0000000000000..f3aeb822e72cd --- /dev/null +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java @@ -0,0 +1,320 @@ +/* + * 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.kvstore; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Objects; +import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; + +import org.apache.spark.annotation.Private; + +/** + * Implementation of KVStore that keeps data deserialized in memory. This store does not index + * data; instead, whenever iterating over an indexed field, the stored data is copied and sorted + * according to the index. This saves memory but makes iteration more expensive. + */ +@Private +public class InMemoryStore implements KVStore { + + private Object metadata; + private ConcurrentMap, InstanceList> data = new ConcurrentHashMap<>(); + + @Override + public T getMetadata(Class klass) { + return klass.cast(metadata); + } + + @Override + public void setMetadata(Object value) { + this.metadata = value; + } + + @Override + public long count(Class type) { + InstanceList list = data.get(type); + return list != null ? list.size() : 0; + } + + @Override + public long count(Class type, String index, Object indexedValue) throws Exception { + InstanceList list = data.get(type); + int count = 0; + Object comparable = asKey(indexedValue); + KVTypeInfo.Accessor accessor = list.getIndexAccessor(index); + for (Object o : view(type)) { + if (Objects.equal(comparable, asKey(accessor.get(o)))) { + count++; + } + } + return count; + } + + @Override + public T read(Class klass, Object naturalKey) { + InstanceList list = data.get(klass); + Object value = list != null ? list.get(naturalKey) : null; + if (value == null) { + throw new NoSuchElementException(); + } + return klass.cast(value); + } + + @Override + public void write(Object value) throws Exception { + InstanceList list = data.computeIfAbsent(value.getClass(), key -> { + try { + return new InstanceList(key); + } catch (Exception e) { + throw Throwables.propagate(e); + } + }); + list.put(value); + } + + @Override + public void delete(Class type, Object naturalKey) { + InstanceList list = data.get(type); + if (list != null) { + list.delete(naturalKey); + } + } + + @Override + public KVStoreView view(Class type){ + InstanceList list = data.get(type); + return list != null ? list.view(type) + : new InMemoryView<>(type, Collections.emptyList(), null); + } + + @Override + public void close() { + metadata = null; + data.clear(); + } + + @SuppressWarnings("unchecked") + private static Comparable asKey(Object in) { + if (in.getClass().isArray()) { + in = ArrayWrappers.forArray(in); + } + return (Comparable) in; + } + + private static class InstanceList { + + private final KVTypeInfo ti; + private final KVTypeInfo.Accessor naturalKey; + private final ConcurrentMap, Object> data; + + private int size; + + private InstanceList(Class type) throws Exception { + this.ti = new KVTypeInfo(type); + this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME); + this.data = new ConcurrentHashMap<>(); + this.size = 0; + } + + KVTypeInfo.Accessor getIndexAccessor(String indexName) { + return ti.getAccessor(indexName); + } + + public Object get(Object key) { + return data.get(asKey(key)); + } + + public void put(Object value) throws Exception { + Preconditions.checkArgument(ti.type().equals(value.getClass()), + "Unexpected type: %s", value.getClass()); + if (data.put(asKey(naturalKey.get(value)), value) == null) { + size++; + } + } + + public void delete(Object key) { + if (data.remove(asKey(key)) != null) { + size--; + } + } + + public int size() { + return size; + } + + @SuppressWarnings("unchecked") + public InMemoryView view(Class type) { + Preconditions.checkArgument(ti.type().equals(type), "Unexpected type: %s", type); + Collection all = (Collection) data.values(); + return new InMemoryView(type, all, ti); + } + + } + + private static class InMemoryView extends KVStoreView { + + private final Collection elements; + private final KVTypeInfo ti; + private final KVTypeInfo.Accessor natural; + + InMemoryView(Class type, Collection elements, KVTypeInfo ti) { + super(type); + this.elements = elements; + this.ti = ti; + this.natural = ti != null ? ti.getAccessor(KVIndex.NATURAL_INDEX_NAME) : null; + } + + @Override + public Iterator iterator() { + if (elements.isEmpty()) { + return new InMemoryIterator<>(elements.iterator()); + } + + try { + KVTypeInfo.Accessor getter = index != null ? ti.getAccessor(index) : null; + int modifier = ascending ? 1 : -1; + + final List sorted = copyElements(); + Collections.sort(sorted, (e1, e2) -> modifier * compare(e1, e2, getter)); + Stream stream = sorted.stream(); + + if (first != null) { + stream = stream.filter(e -> modifier * compare(e, getter, first) >= 0); + } + + if (last != null) { + stream = stream.filter(e -> modifier * compare(e, getter, last) <= 0); + } + + if (skip > 0) { + stream = stream.skip(skip); + } + + if (max < sorted.size()) { + stream = stream.limit((int) max); + } + + return new InMemoryIterator<>(stream.iterator()); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + /** + * Create a copy of the input elements, filtering the values for child indices if needed. + */ + private List copyElements() { + if (parent != null) { + KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index); + Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index."); + + return elements.stream() + .filter(e -> compare(e, parentGetter, parent) == 0) + .collect(Collectors.toList()); + } else { + return new ArrayList<>(elements); + } + } + + private int compare(T e1, T e2, KVTypeInfo.Accessor getter) { + try { + int diff = compare(e1, getter, getter.get(e2)); + if (diff == 0 && getter != natural) { + diff = compare(e1, natural, natural.get(e2)); + } + return diff; + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + private int compare(T e1, KVTypeInfo.Accessor getter, Object v2) { + try { + return asKey(getter.get(e1)).compareTo(asKey(v2)); + } catch (Exception e) { + throw Throwables.propagate(e); + } + } + + } + + private static class InMemoryIterator implements KVStoreIterator { + + private final Iterator iter; + + InMemoryIterator(Iterator iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public T next() { + return iter.next(); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public List next(int max) { + List list = new ArrayList<>(max); + while (hasNext() && list.size() < max) { + list.add(next()); + } + return list; + } + + @Override + public boolean skip(long n) { + long skipped = 0; + while (skipped < n) { + if (hasNext()) { + next(); + skipped++; + } else { + return false; + } + } + + return hasNext(); + } + + @Override + public void close() { + // no op. + } + + } + +} diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java index e1cc0ba3f5aa7..e3e61e0a0e452 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java @@ -89,7 +89,7 @@ private void checkIndex(KVIndex idx, Map indices) { "Duplicate index %s for type %s.", idx.value(), type.getName()); } - public Class getType() { + public Class type() { return type; } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java new file mode 100644 index 0000000000000..d5938acc3e80e --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.kvstore; + +import java.util.Arrays; + +public class ArrayKeyIndexType { + + @KVIndex + public int[] key; + + @KVIndex("id") + public String[] id; + + @Override + public boolean equals(Object o) { + if (o instanceof ArrayKeyIndexType) { + ArrayKeyIndexType other = (ArrayKeyIndexType) o; + return Arrays.equals(key, other.key) && Arrays.equals(id, other.id); + } + return false; + } + + @Override + public int hashCode() { + return key.hashCode(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java new file mode 100644 index 0000000000000..f9b4774820ea0 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java @@ -0,0 +1,59 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.kvstore; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class ArrayWrappersSuite { + + @Test + public void testGenericArrayKey() { + byte[] b1 = new byte[] { 0x01, 0x02, 0x03 }; + byte[] b2 = new byte[] { 0x01, 0x02 }; + int[] i1 = new int[] { 1, 2, 3 }; + int[] i2 = new int[] { 1, 2 }; + String[] s1 = new String[] { "1", "2", "3" }; + String[] s2 = new String[] { "1", "2" }; + + assertEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(b2)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(i1)); + assertNotEquals(ArrayWrappers.forArray(b1), ArrayWrappers.forArray(s1)); + + assertEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(i1)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(i2)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(i1), ArrayWrappers.forArray(s1)); + + assertEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(s1)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(s2)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(b1)); + assertNotEquals(ArrayWrappers.forArray(s1), ArrayWrappers.forArray(i1)); + + assertEquals(0, ArrayWrappers.forArray(b1).compareTo(ArrayWrappers.forArray(b1))); + assertTrue(ArrayWrappers.forArray(b1).compareTo(ArrayWrappers.forArray(b2)) > 0); + + assertEquals(0, ArrayWrappers.forArray(i1).compareTo(ArrayWrappers.forArray(i1))); + assertTrue(ArrayWrappers.forArray(i1).compareTo(ArrayWrappers.forArray(i2)) > 0); + + assertEquals(0, ArrayWrappers.forArray(s1).compareTo(ArrayWrappers.forArray(s1))); + assertTrue(ArrayWrappers.forArray(s1).compareTo(ArrayWrappers.forArray(s2)) > 0); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java new file mode 100644 index 0000000000000..57ee4f6dd7cb6 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java @@ -0,0 +1,27 @@ +/* + * 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.kvstore; + +public class InMemoryIteratorSuite extends DBIteratorSuite { + + @Override + protected KVStore createStore() { + return new InMemoryStore(); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java new file mode 100644 index 0000000000000..6a7915f9385e0 --- /dev/null +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java @@ -0,0 +1,161 @@ +/* + * 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.kvstore; + +import java.util.NoSuchElementException; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class InMemoryStoreSuite { + + @Test + public void testObjectWriteReadDelete() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + try { + store.read(CustomType1.class, t.key); + fail("Expected exception for non-existant object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + + store.write(t); + assertEquals(t, store.read(t.getClass(), t.key)); + assertEquals(1L, store.count(t.getClass())); + + store.delete(t.getClass(), t.key); + try { + store.read(t.getClass(), t.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + + @Test + public void testMultipleObjectWriteReadDelete() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t1 = new CustomType1(); + t1.key = "key1"; + t1.id = "id"; + t1.name = "name1"; + + CustomType1 t2 = new CustomType1(); + t2.key = "key2"; + t2.id = "id"; + t2.name = "name2"; + + store.write(t1); + store.write(t2); + + assertEquals(t1, store.read(t1.getClass(), t1.key)); + assertEquals(t2, store.read(t2.getClass(), t2.key)); + assertEquals(2L, store.count(t1.getClass())); + + store.delete(t1.getClass(), t1.key); + assertEquals(t2, store.read(t2.getClass(), t2.key)); + store.delete(t2.getClass(), t2.key); + try { + store.read(t2.getClass(), t2.key); + fail("Expected exception for deleted object."); + } catch (NoSuchElementException nsee) { + // Expected. + } + } + + @Test + public void testMetadata() throws Exception { + KVStore store = new InMemoryStore(); + assertNull(store.getMetadata(CustomType1.class)); + + CustomType1 t = new CustomType1(); + t.id = "id"; + t.name = "name"; + + store.setMetadata(t); + assertEquals(t, store.getMetadata(CustomType1.class)); + + store.setMetadata(null); + assertNull(store.getMetadata(CustomType1.class)); + } + + @Test + public void testUpdate() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t = new CustomType1(); + t.key = "key"; + t.id = "id"; + t.name = "name"; + + store.write(t); + + t.name = "anotherName"; + + store.write(t); + assertEquals(1, store.count(t.getClass())); + assertSame(t, store.read(t.getClass(), t.key)); + } + + @Test + public void testArrayIndices() throws Exception { + KVStore store = new InMemoryStore(); + + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { 1, 2 }; + o.id = new String[] { "3", "4" }; + + store.write(o); + assertEquals(o, store.read(ArrayKeyIndexType.class, o.key)); + assertEquals(o, store.view(ArrayKeyIndexType.class).index("id").first(o.id).iterator().next()); + } + + @Test + public void testBasicIteration() throws Exception { + KVStore store = new InMemoryStore(); + + CustomType1 t1 = new CustomType1(); + t1.key = "1"; + t1.id = "id1"; + t1.name = "name1"; + store.write(t1); + + CustomType1 t2 = new CustomType1(); + t2.key = "2"; + t2.id = "id2"; + t2.name = "name2"; + store.write(t2); + + assertEquals(t1.id, store.view(t1.getClass()).iterator().next().id); + assertEquals(t2.id, store.view(t1.getClass()).skip(1).iterator().next().id); + assertEquals(t2.id, store.view(t1.getClass()).skip(1).max(1).iterator().next().id); + assertEquals(t1.id, + store.view(t1.getClass()).first(t1.key).max(1).iterator().next().id); + assertEquals(t2.id, + store.view(t1.getClass()).first(t2.key).max(1).iterator().next().id); + assertFalse(store.view(t1.getClass()).first(t2.id).skip(1).iterator().hasNext()); + } + +} diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java index 42bff610457e7..86c85c1b7a032 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java @@ -283,28 +283,4 @@ public int hashCode() { } - public static class ArrayKeyIndexType { - - @KVIndex - public int[] key; - - @KVIndex("id") - public String[] id; - - @Override - public boolean equals(Object o) { - if (o instanceof ArrayKeyIndexType) { - ArrayKeyIndexType other = (ArrayKeyIndexType) o; - return Arrays.equals(key, other.key) && Arrays.equals(id, other.id); - } - return false; - } - - @Override - public int hashCode() { - return key.hashCode(); - } - - } - } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b9db1df2d1919..371a171aa98e3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -230,7 +230,8 @@ object SparkBuild extends PomBuild { javacOptions in Compile ++= Seq( "-encoding", "UTF-8", - "-source", javacJVMVersion.value + "-source", javacJVMVersion.value, + "-Xlint:unchecked" ), // This -target option cannot be set in the Compile configuration scope since `javadoc` doesn't // play nicely with it; see https://github.com/sbt/sbt/issues/355#issuecomment-3817629 for From 2c1bfb497f31ff402796b57b617a9075c6044e4d Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 8 Aug 2017 14:33:27 -0700 Subject: [PATCH 12/20] [SPARK-21671][CORE] Move kvstore to "util" sub-package, add private annotation. Author: Marcelo Vanzin Closes #18886 from vanzin/SPARK-21671. --- .../org/apache/spark/{ => util}/kvstore/ArrayWrappers.java | 2 +- .../org/apache/spark/{ => util}/kvstore/InMemoryStore.java | 2 +- .../java/org/apache/spark/{ => util}/kvstore/KVIndex.java | 5 ++++- .../java/org/apache/spark/{ => util}/kvstore/KVStore.java | 5 ++++- .../org/apache/spark/{ => util}/kvstore/KVStoreIterator.java | 5 ++++- .../apache/spark/{ => util}/kvstore/KVStoreSerializer.java | 5 ++++- .../org/apache/spark/{ => util}/kvstore/KVStoreView.java | 5 ++++- .../java/org/apache/spark/{ => util}/kvstore/KVTypeInfo.java | 5 ++++- .../java/org/apache/spark/{ => util}/kvstore/LevelDB.java | 5 ++++- .../org/apache/spark/{ => util}/kvstore/LevelDBIterator.java | 2 +- .../org/apache/spark/{ => util}/kvstore/LevelDBTypeInfo.java | 2 +- .../{ => util}/kvstore/UnsupportedStoreVersionException.java | 5 ++++- .../apache/spark/{ => util}/kvstore/ArrayKeyIndexType.java | 2 +- .../apache/spark/{ => util}/kvstore/ArrayWrappersSuite.java | 2 +- .../org/apache/spark/{ => util}/kvstore/CustomType1.java | 2 +- .../org/apache/spark/{ => util}/kvstore/DBIteratorSuite.java | 2 +- .../spark/{ => util}/kvstore/InMemoryIteratorSuite.java | 2 +- .../apache/spark/{ => util}/kvstore/InMemoryStoreSuite.java | 2 +- .../apache/spark/{ => util}/kvstore/LevelDBBenchmark.java | 2 +- .../spark/{ => util}/kvstore/LevelDBIteratorSuite.java | 2 +- .../org/apache/spark/{ => util}/kvstore/LevelDBSuite.java | 2 +- .../spark/{ => util}/kvstore/LevelDBTypeInfoSuite.java | 2 +- 22 files changed, 46 insertions(+), 22 deletions(-) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/ArrayWrappers.java (99%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/InMemoryStore.java (99%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVIndex.java (97%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVStore.java (97%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVStoreIterator.java (93%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVStoreSerializer.java (96%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVStoreView.java (97%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/KVTypeInfo.java (98%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/LevelDB.java (99%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/LevelDBIterator.java (99%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/LevelDBTypeInfo.java (99%) rename common/kvstore/src/main/java/org/apache/spark/{ => util}/kvstore/UnsupportedStoreVersionException.java (91%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/ArrayKeyIndexType.java (96%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/ArrayWrappersSuite.java (98%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/CustomType1.java (97%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/DBIteratorSuite.java (99%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/InMemoryIteratorSuite.java (96%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/InMemoryStoreSuite.java (99%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/LevelDBBenchmark.java (99%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/LevelDBIteratorSuite.java (97%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/LevelDBSuite.java (99%) rename common/kvstore/src/test/java/org/apache/spark/{ => util}/kvstore/LevelDBTypeInfoSuite.java (99%) diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java similarity index 99% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java index 5efa842e3aad7..9bc8c55bd5389 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/ArrayWrappers.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/ArrayWrappers.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.Arrays; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java similarity index 99% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java index f3aeb822e72cd..9cae5da5d2600 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/InMemoryStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.ArrayList; import java.util.Collection; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java similarity index 97% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java index 0cffefe07c25d..80f4921107246 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVIndex.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVIndex.java @@ -15,13 +15,15 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.spark.annotation.Private; + /** * Tags a field to be indexed when storing an object. * @@ -46,6 +48,7 @@ * of those values. *

*/ +@Private @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.FIELD, ElementType.METHOD}) public @interface KVIndex { diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java similarity index 97% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java index c7808ea3c3881..72d06a8ca8070 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.Closeable; +import org.apache.spark.annotation.Private; + /** * Abstraction for a local key/value store for storing app data. * @@ -59,6 +61,7 @@ * KVStore instances are thread-safe for both reads and writes. *

*/ +@Private public interface KVStore extends Closeable { /** diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java similarity index 93% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java index 3efdec9ed32be..28a432b26d98e 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreIterator.java @@ -15,11 +15,13 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.Iterator; import java.util.List; +import org.apache.spark.annotation.Private; + /** * An iterator for KVStore. * @@ -28,6 +30,7 @@ * explicitly close iterators after they're used. *

*/ +@Private public interface KVStoreIterator extends Iterator, AutoCloseable { /** diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java similarity index 96% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java index b84ec91cf67a0..bd8d9486acde5 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreSerializer.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreSerializer.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -25,6 +25,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.spark.annotation.Private; + /** * Serializer used to translate between app-defined types and the LevelDB store. * @@ -33,6 +35,7 @@ * and integers to be written as values directly, which will be written as UTF-8 strings. *

*/ +@Private public class KVStoreSerializer { /** diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java similarity index 97% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java index 8cd1f52892293..8ea79bbe160d9 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import com.google.common.base.Preconditions; +import org.apache.spark.annotation.Private; + /** * A configurable view that allows iterating over values in a {@link KVStore}. * @@ -33,6 +35,7 @@ * to be closed explicitly unless all elements are read. *

*/ +@Private public abstract class KVStoreView implements Iterable { final Class type; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java similarity index 98% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java index e3e61e0a0e452..a2b077e4531ee 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -25,9 +25,12 @@ import com.google.common.base.Preconditions; +import org.apache.spark.annotation.Private; + /** * Wrapper around types managed in a KVStore, providing easy access to their indexed fields. */ +@Private public class KVTypeInfo { private final Class type; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java similarity index 99% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 27141358dc0f2..310febc352ef8 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.File; import java.io.IOException; @@ -36,9 +36,12 @@ import org.iq80.leveldb.Options; import org.iq80.leveldb.WriteBatch; +import org.apache.spark.annotation.Private; + /** * Implementation of KVStore that uses LevelDB as the underlying data store. */ +@Private public class LevelDB implements KVStore { @VisibleForTesting diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java similarity index 99% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java index 263d45c242106..a2181f3874f86 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.IOException; import java.util.ArrayList; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java similarity index 99% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java index 722f54e6f9c66..93aa0bbc9dc8f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/LevelDBTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBTypeInfo.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.lang.reflect.Array; import java.util.Collection; diff --git a/common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java similarity index 91% rename from common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java rename to common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java index 2ed246e4f4c97..75a33b7c75de4 100644 --- a/common/kvstore/src/main/java/org/apache/spark/kvstore/UnsupportedStoreVersionException.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/UnsupportedStoreVersionException.java @@ -15,13 +15,16 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.IOException; +import org.apache.spark.annotation.Private; + /** * Exception thrown when the store implementation is not compatible with the underlying data. */ +@Private public class UnsupportedStoreVersionException extends IOException { } diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java similarity index 96% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java index d5938acc3e80e..32030fb4115c3 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayKeyIndexType.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayKeyIndexType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.Arrays; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java similarity index 98% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java index f9b4774820ea0..b1c8927d0761d 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/ArrayWrappersSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/ArrayWrappersSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import org.junit.Test; import static org.junit.Assert.*; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java similarity index 97% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java index afb72b8689223..92b643b0cb928 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/CustomType1.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/CustomType1.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import com.google.common.base.Objects; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java similarity index 99% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java index 3a418189ecfec..9a81f86812cde 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/DBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/DBIteratorSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.Arrays; import java.util.ArrayList; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java similarity index 96% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java index 57ee4f6dd7cb6..27dde6a9fbea1 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryIteratorSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; public class InMemoryIteratorSuite extends DBIteratorSuite { diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java similarity index 99% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java index 6a7915f9385e0..510b3058a4e3c 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/InMemoryStoreSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.util.NoSuchElementException; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java similarity index 99% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java index 5e33606b12dd4..268d025f5f06a 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBBenchmark.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBBenchmark.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.File; import java.util.ArrayList; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java similarity index 97% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java index 93409712986ca..f8195da58cf9f 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBIteratorSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBIteratorSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.File; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java similarity index 99% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 86c85c1b7a032..2b07d249d2022 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import java.io.File; import java.util.Arrays; diff --git a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java similarity index 99% rename from common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java rename to common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java index 8e6196506c6a8..38db3bedaef6a 100644 --- a/common/kvstore/src/test/java/org/apache/spark/kvstore/LevelDBTypeInfoSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBTypeInfoSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.kvstore; +package org.apache.spark.util.kvstore; import static java.nio.charset.StandardCharsets.UTF_8; From fb54a564d75aea835f57bc147b83a76d1da0a01f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 8 Aug 2017 18:15:29 -0700 Subject: [PATCH 13/20] [SPARK-20433][BUILD] Bump jackson from 2.6.5 to 2.6.7.1 ## What changes were proposed in this pull request? Taking over https://github.com/apache/spark/pull/18789 ; Closes #18789 Update Jackson to 2.6.7 uniformly, and some components to 2.6.7.1, to get some fixes and prep for Scala 2.12 ## How was this patch tested? Existing tests Author: Sean Owen Closes #18881 from srowen/SPARK-20433. --- dev/deps/spark-deps-hadoop-2.6 | 10 +++++----- dev/deps/spark-deps-hadoop-2.7 | 10 +++++----- pom.xml | 12 +++++++++--- sql/core/pom.xml | 1 - 4 files changed, 19 insertions(+), 14 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 76e37646af0ec..a41183accf718 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -86,14 +86,14 @@ htrace-core-3.0.4.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.6.5.jar -jackson-core-2.6.5.jar +jackson-annotations-2.6.7.jar +jackson-core-2.6.7.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.6.5.jar +jackson-databind-2.6.7.1.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.6.5.jar -jackson-module-scala_2.11-2.6.5.jar +jackson-module-paranamer-2.6.7.jar +jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar janino-3.0.0.jar java-xmlbuilder-1.0.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index f4173477d05d3..5e1321be14d0c 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -86,14 +86,14 @@ htrace-core-3.1.0-incubating.jar httpclient-4.5.2.jar httpcore-4.4.4.jar ivy-2.4.0.jar -jackson-annotations-2.6.5.jar -jackson-core-2.6.5.jar +jackson-annotations-2.6.7.jar +jackson-core-2.6.7.jar jackson-core-asl-1.9.13.jar -jackson-databind-2.6.5.jar +jackson-databind-2.6.7.1.jar jackson-jaxrs-1.9.13.jar jackson-mapper-asl-1.9.13.jar -jackson-module-paranamer-2.6.5.jar -jackson-module-scala_2.11-2.6.5.jar +jackson-module-paranamer-2.6.7.jar +jackson-module-scala_2.11-2.6.7.1.jar jackson-xc-1.9.13.jar janino-3.0.0.jar java-xmlbuilder-1.0.jar diff --git a/pom.xml b/pom.xml index d54a9c4f19667..95dd52acdfd5a 100644 --- a/pom.xml +++ b/pom.xml @@ -158,7 +158,8 @@ 2.11.8 2.11 1.9.13 - 2.6.5 + 2.6.7 + 2.6.7.1 1.1.2.6 1.1.2 1.2.0-incubating @@ -615,7 +616,7 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.jackson.version} + ${fasterxml.jackson.databind.version} com.fasterxml.jackson.core @@ -627,7 +628,7 @@ com.fasterxml.jackson.module jackson-module-scala_${scala.binary.version} - ${fasterxml.jackson.version} + ${fasterxml.jackson.databind.version} com.google.guava @@ -635,6 +636,11 @@ + + com.fasterxml.jackson.module + jackson-module-paranamer + ${fasterxml.jackson.version} + com.fasterxml.jackson.module jackson-module-jaxb-annotations diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 25004e5326097..a16411ec304a9 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -101,7 +101,6 @@ com.fasterxml.jackson.core jackson-databind - ${fasterxml.jackson.version} org.apache.arrow From 6edfff055caea81dc3a98a6b4081313a0c0b0729 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 8 Aug 2017 20:20:26 -0700 Subject: [PATCH 14/20] [SPARK-21596][SS] Ensure places calling HDFSMetadataLog.get check the return value ## What changes were proposed in this pull request? When I was investigating a flaky test, I realized that many places don't check the return value of `HDFSMetadataLog.get(batchId: Long): Option[T]`. When a batch is supposed to be there, the caller just ignores None rather than throwing an error. If some bug causes a query doesn't generate a batch metadata file, this behavior will hide it and allow the query continuing to run and finally delete metadata logs and make it hard to debug. This PR ensures that places calling HDFSMetadataLog.get always check the return value. ## How was this patch tested? Jenkins Author: Shixiong Zhu Closes #18799 from zsxwing/SPARK-21596. --- .../streaming/CompactibleFileStreamLog.scala | 16 +++++- .../streaming/FileStreamSourceLog.scala | 5 +- .../execution/streaming/HDFSMetadataLog.scala | 57 ++++++++++++++++++- .../execution/streaming/StreamExecution.scala | 17 ++++-- .../streaming/HDFSMetadataLogSuite.scala | 17 ++++++ .../sql/streaming/FileStreamSourceSuite.scala | 1 + 6 files changed, 102 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index e37033b19a8eb..77bc0ba5548dd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -169,7 +169,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( */ private def compact(batchId: Long, logs: Array[T]): Boolean = { val validBatches = getValidBatchesBeforeCompactionBatch(batchId, compactInterval) - val allLogs = validBatches.flatMap(batchId => super.get(batchId)).flatten ++ logs + val allLogs = validBatches.map { id => + super.get(id).getOrElse { + throw new IllegalStateException( + s"${batchIdToPath(id)} doesn't exist when compacting batch $batchId " + + s"(compactInterval: $compactInterval)") + } + }.flatten ++ logs // Return false as there is another writer. super.add(batchId, compactLogs(allLogs).toArray) } @@ -186,7 +192,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( if (latestId >= 0) { try { val logs = - getAllValidBatches(latestId, compactInterval).flatMap(id => super.get(id)).flatten + getAllValidBatches(latestId, compactInterval).map { id => + super.get(id).getOrElse { + throw new IllegalStateException( + s"${batchIdToPath(id)} doesn't exist " + + s"(latestId: $latestId, compactInterval: $compactInterval)") + } + }.flatten return compactLogs(logs).toArray } catch { case e: IOException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala index 33e6a1d5d6e18..8628471fdb925 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala @@ -115,7 +115,10 @@ class FileStreamSourceLog( Map.empty[Long, Option[Array[FileEntry]]] } - (existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1) + val batches = + (existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1) + HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + batches } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index 46bfc297931fb..5f8973fd09460 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -123,7 +123,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: serialize(metadata, output) return Some(tempPath) } finally { - IOUtils.closeQuietly(output) + output.close() } } catch { case e: FileAlreadyExistsException => @@ -211,13 +211,17 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: } override def get(startId: Option[Long], endId: Option[Long]): Array[(Long, T)] = { + assert(startId.isEmpty || endId.isEmpty || startId.get <= endId.get) val files = fileManager.list(metadataPath, batchFilesFilter) val batchIds = files .map(f => pathToBatchId(f.getPath)) .filter { batchId => (endId.isEmpty || batchId <= endId.get) && (startId.isEmpty || batchId >= startId.get) - } - batchIds.sorted.map(batchId => (batchId, get(batchId))).filter(_._2.isDefined).map { + }.sorted + + verifyBatchIds(batchIds, startId, endId) + + batchIds.map(batchId => (batchId, get(batchId))).filter(_._2.isDefined).map { case (batchId, metadataOption) => (batchId, metadataOption.get) } @@ -437,4 +441,51 @@ object HDFSMetadataLog { } } } + + /** + * Verify if batchIds are continuous and between `startId` and `endId`. + * + * @param batchIds the sorted ids to verify. + * @param startId the start id. If it's set, batchIds should start with this id. + * @param endId the start id. If it's set, batchIds should end with this id. + */ + def verifyBatchIds(batchIds: Seq[Long], startId: Option[Long], endId: Option[Long]): Unit = { + // Verify that we can get all batches between `startId` and `endId`. + if (startId.isDefined || endId.isDefined) { + if (batchIds.isEmpty) { + throw new IllegalStateException(s"batch ${startId.orElse(endId).get} doesn't exist") + } + if (startId.isDefined) { + val minBatchId = batchIds.head + assert(minBatchId >= startId.get) + if (minBatchId != startId.get) { + val missingBatchIds = startId.get to minBatchId + throw new IllegalStateException( + s"batches (${missingBatchIds.mkString(", ")}) don't exist " + + s"(startId: $startId, endId: $endId)") + } + } + + if (endId.isDefined) { + val maxBatchId = batchIds.last + assert(maxBatchId <= endId.get) + if (maxBatchId != endId.get) { + val missingBatchIds = maxBatchId to endId.get + throw new IllegalStateException( + s"batches (${missingBatchIds.mkString(", ")}) don't exist " + + s"(startId: $startId, endId: $endId)") + } + } + } + + if (batchIds.nonEmpty) { + val minBatchId = batchIds.head + val maxBatchId = batchIds.last + val missingBatchIds = (minBatchId to maxBatchId).toSet -- batchIds + if (missingBatchIds.nonEmpty) { + throw new IllegalStateException(s"batches (${missingBatchIds.mkString(", ")}) " + + s"don't exist (startId: $startId, endId: $endId)") + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 1528e7f469bef..9bc114f138562 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -438,7 +438,10 @@ class StreamExecution( availableOffsets = nextOffsets.toStreamProgress(sources) /* Initialize committed offsets to a committed batch, which at this * is the second latest batch id in the offset log. */ - offsetLog.get(latestBatchId - 1).foreach { secondLatestBatchId => + if (latestBatchId != 0) { + val secondLatestBatchId = offsetLog.get(latestBatchId - 1).getOrElse { + throw new IllegalStateException(s"batch ${latestBatchId - 1} doesn't exist") + } committedOffsets = secondLatestBatchId.toStreamProgress(sources) } @@ -565,10 +568,14 @@ class StreamExecution( // Now that we've updated the scheduler's persistent checkpoint, it is safe for the // sources to discard data from the previous batch. - val prevBatchOff = offsetLog.get(currentBatchId - 1) - if (prevBatchOff.isDefined) { - prevBatchOff.get.toStreamProgress(sources).foreach { - case (src, off) => src.commit(off) + if (currentBatchId != 0) { + val prevBatchOff = offsetLog.get(currentBatchId - 1) + if (prevBatchOff.isDefined) { + prevBatchOff.get.toStreamProgress(sources).foreach { + case (src, off) => src.commit(off) + } + } else { + throw new IllegalStateException(s"batch $currentBatchId doesn't exist") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 7689bc03a4ccf..48e70e48b1799 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -259,6 +259,23 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { fm.rename(path2, path3) } } + + test("verifyBatchIds") { + import HDFSMetadataLog.verifyBatchIds + verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), Some(3L)) + verifyBatchIds(Seq(1L), Some(1L), Some(1L)) + verifyBatchIds(Seq(1L, 2L, 3L), None, Some(3L)) + verifyBatchIds(Seq(1L, 2L, 3L), Some(1L), None) + verifyBatchIds(Seq(1L, 2L, 3L), None, None) + + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), None)) + intercept[IllegalStateException](verifyBatchIds(Seq(), None, Some(1L))) + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(1L), Some(1L))) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), None)) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L))) + intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L))) + intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L))) + } } /** FakeFileSystem to test fallback of the HDFSMetadataLog from FileContext to FileSystem API */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index 2108b118bf059..e2ec690d90e52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -1314,6 +1314,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val metadataLog = new FileStreamSourceLog(FileStreamSourceLog.VERSION, spark, dir.getAbsolutePath) assert(metadataLog.add(0, Array(FileEntry(s"$scheme:///file1", 100L, 0)))) + assert(metadataLog.add(1, Array(FileEntry(s"$scheme:///file2", 200L, 0)))) val newSource = new FileStreamSource(spark, s"$scheme:///", "parquet", StructType(Nil), Nil, dir.getAbsolutePath, Map.empty) From 031910b0ec24526d044fd31c05430dcda42b5be3 Mon Sep 17 00:00:00 2001 From: Xingbo Jiang Date: Wed, 9 Aug 2017 13:23:49 +0800 Subject: [PATCH 15/20] [SPARK-21608][SPARK-9221][SQL] Window rangeBetween() API should allow literal boundary ## What changes were proposed in this pull request? Window rangeBetween() API should allow literal boundary, that means, the window range frame can calculate frame of double/date/timestamp. Example of the use case can be: ``` SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData ``` This PR refactors the Window `rangeBetween` and `rowsBetween` API, while the legacy user code should still be valid. ## How was this patch tested? Add new test cases both in `DataFrameWindowFunctionsSuite` and in `window.sql`. Author: Xingbo Jiang Closes #18814 from jiangxb1987/literal-boundary. --- .../sql/catalyst/analysis/TypeCoercion.scala | 9 +- .../expressions/windowExpressions.scala | 8 +- .../sql/execution/window/WindowExec.scala | 9 +- .../apache/spark/sql/expressions/Window.scala | 63 +++++++- .../spark/sql/expressions/WindowSpec.scala | 69 ++++++++- .../org/apache/spark/sql/functions.scala | 27 ++++ .../resources/sql-tests/inputs/window.sql | 22 ++- .../sql-tests/results/window.sql.out | 141 +++++++++++++----- .../sql/DataFrameWindowFunctionsSuite.scala | 53 ++++++- 9 files changed, 335 insertions(+), 66 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 25af014f67fe9..06d8350db9891 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -821,9 +821,12 @@ object TypeCoercion { } private def createBoundaryCast(boundary: Expression, dt: DataType): Expression = { - boundary match { - case e: SpecialFrameBoundary => e - case e: Expression if e.dataType != dt && Cast.canCast(e.dataType, dt) => Cast(e, dt) + (boundary, dt) match { + case (e: SpecialFrameBoundary, _) => e + case (e, _: DateType) => e + case (e, _: TimestampType) => e + case (e: Expression, t) if e.dataType != t && Cast.canCast(e.dataType, t) => + Cast(e, t) case _ => boundary } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index a829dccfd3e36..e11e3a105f597 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -89,7 +89,11 @@ case class WindowSpecDefinition( elements.mkString("(", " ", ")") } - private def isValidFrameType(ft: DataType): Boolean = orderSpec.head.dataType == ft + private def isValidFrameType(ft: DataType): Boolean = (orderSpec.head.dataType, ft) match { + case (DateType, IntegerType) => true + case (TimestampType, CalendarIntervalType) => true + case (a, b) => a == b + } } /** @@ -129,7 +133,7 @@ case object RowFrame extends FrameType { * of the current row. */ case object RangeFrame extends FrameType { - override def inputType: AbstractDataType = NumericType + override def inputType: AbstractDataType = TypeCollection.NumericAndInterval override def sql: String = "RANGE" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index 0766e37826cb2..f8bb667e30064 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -25,7 +25,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType} /** * This class calculates and outputs (windowed) aggregates over the rows in a single (sorted) @@ -139,7 +141,12 @@ case class WindowExec( } // Create the projection which returns the current 'value' modified by adding the offset. - val boundExpr = Add(expr, Cast(boundOffset, expr.dataType)) + val boundExpr = (expr.dataType, boundOffset.dataType) match { + case (DateType, IntegerType) => DateAdd(expr, boundOffset) + case (TimestampType, CalendarIntervalType) => + TimeAdd(expr, boundOffset, Some(conf.sessionLocalTimeZone)) + case (a, b) if a== b => Add(expr, boundOffset) + } val bound = newMutableProjection(boundExpr :: Nil, child.output) // Construct the ordering. This is used to compare the result of current value projection diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index cd79128d8f375..1caa243f8d118 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -75,7 +75,7 @@ object Window { } /** - * Value representing the last row in the partition, equivalent to "UNBOUNDED PRECEDING" in SQL. + * Value representing the first row in the partition, equivalent to "UNBOUNDED PRECEDING" in SQL. * This can be used to specify the frame boundaries: * * {{{ @@ -167,17 +167,17 @@ object Window { * current row. * * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using integral - * values directly. + * and `Window.currentRow` to specify special boundary values, rather than using long values + * directly. * * A range-based boundary is based on the actual value of the ORDER BY * expression(s). An offset is used to alter the value of the ORDER BY expression, for * instance if the current order by expression has a value of 10 and the lower bound offset * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical data type. An exception can be made when the offset is 0, - * because no value modification is needed, in this case multiple and non-numeric ORDER BY - * expression are allowed. + * expression must have a numerical data type. An exception can be made when the offset is + * unbounded, because no value modification is needed, in this case multiple and non-numeric + * ORDER BY expression are allowed. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -210,6 +210,57 @@ object Window { spec.rangeBetween(start, end) } + /** + * Creates a [[WindowSpec]] with the frame boundaries defined, + * from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative to the current row. For example, "lit(0)" means + * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the + * five off after the current row. + * + * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from + * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not + * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical/date/timestamp data type. An exception can be made when the + * offset is unbounded, because no value modification is needed, in this case multiple and + * non-numerical/date/timestamp data type ORDER BY expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. + * @param end boundary end, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * @since 2.3.0 + */ + def rangeBetween(start: Column, end: Column): WindowSpec = { + spec.rangeBetween(start, end) + } + private[sql] def spec: WindowSpec = { new WindowSpec(Seq.empty, Seq.empty, UnspecifiedFrame) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index f8b404de77a4a..4c41aa3c5fb67 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -146,22 +146,22 @@ class WindowSpec private[sql]( /** * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). * - * Both `start` and `end` are relative from the current row. For example, "0" means "current row", - * while "-1" means one off before the current row, and "5" means the five off after the - * current row. + * Both `start` and `end` are relative from the current row. For example, "0" means + * "current row", while "-1" means one off before the current row, and "5" means the five off + * after the current row. * * We recommend users use `Window.unboundedPreceding`, `Window.unboundedFollowing`, - * and `Window.currentRow` to specify special boundary values, rather than using integral - * values directly. + * and `Window.currentRow` to specify special boundary values, rather than using long values + * directly. * * A range-based boundary is based on the actual value of the ORDER BY * expression(s). An offset is used to alter the value of the ORDER BY expression, for * instance if the current order by expression has a value of 10 and the lower bound offset * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical data type. An exception can be made when the offset is 0, - * because no value modification is needed, in this case multiple and non-numeric ORDER BY - * expression are allowed. + * expression must have a numerical data type. An exception can be made when the offset is + * unbounded, because no value modification is needed, in this case multiple and non-numeric + * ORDER BY expression are allowed. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -209,6 +209,59 @@ class WindowSpec private[sql]( SpecifiedWindowFrame(RangeFrame, boundaryStart, boundaryEnd)) } + /** + * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). + * + * Both `start` and `end` are relative to the current row. For example, "lit(0)" means + * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the + * five off after the current row. + * + * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from + * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not + * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. + * + * A range-based boundary is based on the actual value of the ORDER BY + * expression(s). An offset is used to alter the value of the ORDER BY expression, for + * instance if the current order by expression has a value of 10 and the lower bound offset + * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a + * number of constraints on the ORDER BY expressions: there can be only one expression and this + * expression must have a numerical/date/timestamp data type. An exception can be made when the + * offset is unbounded, because no value modification is needed, in this case multiple and + * non-numerical/date/timestamp data type ORDER BY expression are allowed. + * + * {{{ + * import org.apache.spark.sql.expressions.Window + * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) + * .toDF("id", "category") + * val byCategoryOrderedById = + * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) + * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() + * + * +---+--------+---+ + * | id|category|sum| + * +---+--------+---+ + * | 1| b| 3| + * | 2| b| 5| + * | 3| b| 3| + * | 1| a| 4| + * | 1| a| 4| + * | 2| a| 2| + * +---+--------+---+ + * }}} + * + * @param start boundary start, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. + * @param end boundary end, inclusive. The frame is unbounded if the expression is + * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * @since 2.3.0 + */ + def rangeBetween(start: Column, end: Column): WindowSpec = { + new WindowSpec( + partitionSpec, + orderSpec, + SpecifiedWindowFrame(RangeFrame, start.expr, end.expr)) + } + /** * Converts this [[WindowSpec]] into a [[Column]] with an aggregate expression. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 496619a43a503..14ab8a2665141 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -777,6 +778,32 @@ object functions { ////////////////////////////////////////////////////////////////////////////////////////////// // Window functions ////////////////////////////////////////////////////////////////////////////////////////////// + /** + * Window function: returns the special frame boundary that represents the first row in the + * window partition. + * + * @group window_funcs + * @since 2.3.0 + */ + def unboundedPreceding(): Column = Column(UnboundedPreceding) + + /** + * Window function: returns the special frame boundary that represents the last row in the + * window partition. + * + * @group window_funcs + * @since 2.3.0 + */ + def unboundedFollowing(): Column = Column(UnboundedFollowing) + + /** + * Window function: returns the special frame boundary that represents the current row in the + * window partition. + * + * @group window_funcs + * @since 2.3.0 + */ + def currentRow(): Column = Column(CurrentRow) /** * Window function: returns the cumulative distribution of values within a window partition, diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index 342e5719e9a60..c4bea34ec4cf3 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -1,8 +1,15 @@ -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, "a"), (1, 1L, "a"), (1, 2L, "a"), (2, 2147483650L, "a"), (1, null, "b"), (2, 3L, "b"), -(3, 2147483650L, "b"), (null, null, null), (3, 1L, null) -AS testData(val, val_long, cate); +(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val ROWS CURRENT ROW) FROM testData @@ -19,6 +26,13 @@ SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; SELECT val_long, cate, sum(val_long) OVER(PARTITION BY cate ORDER BY val_long RANGE BETWEEN CURRENT ROW AND 2147483648 FOLLOWING) FROM testData ORDER BY cate, val_long; +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double; +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date; +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp; -- RangeBetween with reverse OrderBy SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC @@ -31,7 +45,7 @@ SELECT val, cate, count(val) OVER(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; -SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_date +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val; SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val; diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 97511068b323c..73ad27e5bf8ce 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,12 +1,19 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 19 +-- Number of queries: 22 -- !query 0 CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, "a"), (1, 1L, "a"), (1, 2L, "a"), (2, 2147483650L, "a"), (1, null, "b"), (2, 3L, "b"), -(3, 2147483650L, "b"), (null, null, null), (3, 1L, null) -AS testData(val, val_long, cate) +(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, null, null, null, null, null), +(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query 0 schema struct<> -- !query 0 output @@ -109,11 +116,63 @@ NULL b NULL -- !query 7 +SELECT val_double, cate, sum(val_double) OVER(PARTITION BY cate ORDER BY val_double +RANGE BETWEEN CURRENT ROW AND 2.5 FOLLOWING) FROM testData ORDER BY cate, val_double +-- !query 7 schema +struct +-- !query 7 output +NULL NULL NULL +1.0 NULL 1.0 +1.0 a 4.5 +1.0 a 4.5 +2.5 a 2.5 +100.001 a 100.001 +1.0 b 4.3 +3.3 b 3.3 +100.001 b 100.001 + + +-- !query 8 +SELECT val_date, cate, max(val_date) OVER(PARTITION BY cate ORDER BY val_date +RANGE BETWEEN CURRENT ROW AND 2 FOLLOWING) FROM testData ORDER BY cate, val_date +-- !query 8 schema +struct +-- !query 8 output +NULL NULL NULL +2017-08-01 NULL 2017-08-01 +2017-08-01 a 2017-08-02 +2017-08-01 a 2017-08-02 +2017-08-02 a 2017-08-02 +2020-12-31 a 2020-12-31 +2017-08-01 b 2017-08-03 +2017-08-03 b 2017-08-03 +2020-12-31 b 2020-12-31 + + +-- !query 9 +SELECT val_timestamp, cate, avg(val_timestamp) OVER(PARTITION BY cate ORDER BY val_timestamp +RANGE BETWEEN CURRENT ROW AND interval 23 days 4 hours FOLLOWING) FROM testData +ORDER BY cate, val_timestamp +-- !query 9 schema +struct +-- !query 9 output +NULL NULL NULL +2017-07-31 17:00:00 NULL 1.5015456E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-07-31 17:00:00 a 1.5016970666666667E9 +2017-08-05 23:13:20 a 1.502E9 +2020-12-30 16:00:00 a 1.6093728E9 +2017-07-31 17:00:00 b 1.5022728E9 +2017-08-17 13:00:00 b 1.503E9 +2020-12-30 16:00:00 b 1.6093728E9 + + +-- !query 10 SELECT val, cate, sum(val) OVER(PARTITION BY cate ORDER BY val DESC RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 7 schema +-- !query 10 schema struct --- !query 7 output +-- !query 10 output NULL NULL NULL 3 NULL 3 NULL a NULL @@ -125,62 +184,62 @@ NULL a NULL 3 b 5 --- !query 8 +-- !query 11 SELECT val, cate, count(val) OVER(PARTITION BY cate ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 8 schema +-- !query 11 schema struct<> --- !query 8 output +-- !query 11 output org.apache.spark.sql.AnalysisException cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not followes the lower bound 'unboundedfollowing$()'.; line 1 pos 33 --- !query 9 +-- !query 12 SELECT val, cate, count(val) OVER(PARTITION BY cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 9 schema +-- !query 12 schema struct<> --- !query 9 output +-- !query 12 output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY testdata.`cate` RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame cannot be used in an unordered window specification.; line 1 pos 33 --- !query 10 +-- !query 13 SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val, cate RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 10 schema +-- !query 13 schema struct<> --- !query 10 output +-- !query 13 output org.apache.spark.sql.AnalysisException cannot resolve '(PARTITION BY testdata.`cate` ORDER BY testdata.`val` ASC NULLS FIRST, testdata.`cate` ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: val#x ASC NULLS FIRST,cate#x ASC NULLS FIRST; line 1 pos 33 --- !query 11 -SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_date +-- !query 14 +SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY current_timestamp RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING) FROM testData ORDER BY cate, val --- !query 11 schema +-- !query 14 schema struct<> --- !query 11 output +-- !query 14 output org.apache.spark.sql.AnalysisException -cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_date() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'DateType' used in the order specification does not match the data type 'IntegerType' which is used in the range frame.; line 1 pos 33 +cannot resolve '(PARTITION BY testdata.`cate` ORDER BY current_timestamp() ASC NULLS FIRST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'TimestampType' used in the order specification does not match the data type 'IntegerType' which is used in the range frame.; line 1 pos 33 --- !query 12 +-- !query 15 SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING) FROM testData ORDER BY cate, val --- !query 12 schema +-- !query 15 schema struct<> --- !query 12 output +-- !query 15 output org.apache.spark.sql.AnalysisException cannot resolve 'RANGE BETWEEN 1 FOLLOWING AND 1 PRECEDING' due to data type mismatch: The lower bound of a window frame must be less than or equal to the upper bound; line 1 pos 33 --- !query 13 +-- !query 16 SELECT val, cate, count(val) OVER(PARTITION BY cate ORDER BY val RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cate, val --- !query 13 schema +-- !query 16 schema struct<> --- !query 13 output +-- !query 16 output org.apache.spark.sql.catalyst.parser.ParseException Frame bound value must be a literal.(line 2, pos 30) @@ -191,7 +250,7 @@ RANGE BETWEEN CURRENT ROW AND current_date PRECEDING) FROM testData ORDER BY cat ------------------------------^^^ --- !query 14 +-- !query 17 SELECT val, cate, max(val) OVER w AS max, min(val) OVER w AS min, @@ -218,9 +277,9 @@ approx_count_distinct(val) OVER w AS approx_count_distinct FROM testData WINDOW w AS (PARTITION BY cate ORDER BY val) ORDER BY cate, val --- !query 14 schema +-- !query 17 schema struct --- !query 14 output +-- !query 17 output NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.5 0.0 1 1 NULL NULL 0 3 NULL 3 3 3 1 3 3.0 NaN NULL 3 NULL 3 3 3 2 2 1.0 1.0 2 2 0.0 NaN 1 NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0.0 1 1 NULL NULL 0 @@ -232,11 +291,11 @@ NULL a NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL NULL 1 1 0.25 0. 3 b 3 1 1 3 6 2.0 1.0 1 1 1 3 3 3 3 3 1.0 1.0 2 3 0.6666666666666666 1.0 3 --- !query 15 +-- !query 18 SELECT val, cate, avg(null) OVER(PARTITION BY cate ORDER BY val) FROM testData ORDER BY cate, val --- !query 15 schema +-- !query 18 schema struct --- !query 15 output +-- !query 18 output NULL NULL NULL 3 NULL NULL NULL a NULL @@ -248,20 +307,20 @@ NULL a NULL 3 b NULL --- !query 16 +-- !query 19 SELECT val, cate, row_number() OVER(PARTITION BY cate) FROM testData ORDER BY cate, val --- !query 16 schema +-- !query 19 schema struct<> --- !query 16 output +-- !query 19 output org.apache.spark.sql.AnalysisException Window function row_number() requires window to be ordered, please add ORDER BY clause. For example SELECT row_number()(value_expr) OVER (PARTITION BY window_partition ORDER BY window_ordering) from table; --- !query 17 +-- !query 20 SELECT val, cate, sum(val) OVER(), avg(val) OVER() FROM testData ORDER BY cate, val --- !query 17 schema +-- !query 20 schema struct --- !query 17 output +-- !query 20 output NULL NULL 13 1.8571428571428572 3 NULL 13 1.8571428571428572 NULL a 13 1.8571428571428572 @@ -273,7 +332,7 @@ NULL a 13 1.8571428571428572 3 b 13 1.8571428571428572 --- !query 18 +-- !query 21 SELECT val, cate, first_value(false) OVER w AS first_value, first_value(true, true) OVER w AS first_value_ignore_null, @@ -284,9 +343,9 @@ last_value(false, false) OVER w AS last_value_contain_null FROM testData WINDOW w AS () ORDER BY cate, val --- !query 18 schema +-- !query 21 schema struct --- !query 18 output +-- !query 21 output NULL NULL false true false false true false 3 NULL false true false false true false NULL a false true false false true false diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 9806e57f08744..ea725af8d1ad8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * Window function testing for DataFrame API. @@ -172,7 +175,7 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { assert(e.message.contains("Boundary end is not a valid integer: 2147483648")) } - test("range between should accept integer/long values as boundary") { + test("range between should accept int/long values as boundary") { val df = Seq((1L, "1"), (1L, "1"), (2147483650L, "1"), (3L, "2"), (2L, "1"), (2147483650L, "2")) .toDF("key", "value") @@ -191,6 +194,54 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { Window.partitionBy($"value").orderBy($"key").rangeBetween(-2147483649L, 0))), Seq(Row(1, 2), Row(1, 2), Row(2, 3), Row(2147483650L, 2), Row(2147483650L, 4), Row(3, 1)) ) + + def dt(date: String): Date = Date.valueOf(date) + + val df2 = Seq((dt("2017-08-01"), "1"), (dt("2017-08-01"), "1"), (dt("2020-12-31"), "1"), + (dt("2017-08-03"), "2"), (dt("2017-08-02"), "1"), (dt("2020-12-31"), "2")) + .toDF("key", "value") + checkAnswer( + df2.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key").rangeBetween(lit(0), lit(2)))), + Seq(Row(dt("2017-08-01"), 3), Row(dt("2017-08-01"), 3), Row(dt("2020-12-31"), 1), + Row(dt("2017-08-03"), 1), Row(dt("2017-08-02"), 1), Row(dt("2020-12-31"), 1)) + ) + } + + test("range between should accept double values as boundary") { + val df = Seq((1.0D, "1"), (1.0D, "1"), (100.001D, "1"), + (3.3D, "2"), (2.02D, "1"), (100.001D, "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key") + .rangeBetween(currentRow, lit(2.5D)))), + Seq(Row(1.0, 3), Row(1.0, 3), Row(100.001, 1), Row(3.3, 1), Row(2.02, 1), Row(100.001, 1)) + ) + } + + test("range between should accept interval values as boundary") { + def ts(timestamp: Long): Timestamp = new Timestamp(timestamp * 1000) + + val df = Seq((ts(1501545600), "1"), (ts(1501545600), "1"), (ts(1609372800), "1"), + (ts(1503000000), "2"), (ts(1502000000), "1"), (ts(1609372800), "2")) + .toDF("key", "value") + df.createOrReplaceTempView("window_table") + checkAnswer( + df.select( + $"key", + count("key").over( + Window.partitionBy($"value").orderBy($"key") + .rangeBetween(currentRow, + lit(CalendarInterval.fromString("interval 23 days 4 hours"))))), + Seq(Row(ts(1501545600), 3), Row(ts(1501545600), 3), Row(ts(1609372800), 1), + Row(ts(1503000000), 1), Row(ts(1502000000), 1), Row(ts(1609372800), 1)) + ) } test("aggregation and rows between with unbounded") { From f016f5c8f6c6aae674e9905a5c0b0bede09163a4 Mon Sep 17 00:00:00 2001 From: pgandhi Date: Wed, 9 Aug 2017 13:46:06 +0800 Subject: [PATCH 16/20] [SPARK-21503][UI] Spark UI shows incorrect task status for a killed Executor Process The executor tab on Spark UI page shows task as completed when an executor process that is running that task is killed using the kill command. Added the case ExecutorLostFailure which was previously not there, thus, the default case would be executed in which case, task would be marked as completed. This case will consider all those cases where executor connection to Spark Driver was lost due to killing the executor process, network connection etc. ## How was this patch tested? Manually Tested the fix by observing the UI change before and after. Before: screen shot-before After: screen shot-after Please review http://spark.apache.org/contributing.html before opening a pull request. Author: pgandhi Author: pgandhi999 Closes #18707 from pgandhi999/master. --- .../apache/spark/ui/exec/ExecutorsTab.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala index aabf6e0c63c02..770da2226fe04 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui.exec import scala.collection.mutable.{LinkedHashMap, ListBuffer} -import org.apache.spark.{ExceptionFailure, Resubmitted, SparkConf, SparkContext} +import org.apache.spark.{Resubmitted, SparkConf, SparkContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageStatus, StorageStatusListener} @@ -131,17 +131,17 @@ class ExecutorsListener(storageStatusListener: StorageStatusListener, conf: Spar if (info != null) { val eid = info.executorId val taskSummary = executorToTaskSummary.getOrElseUpdate(eid, ExecutorTaskSummary(eid)) - taskEnd.reason match { - case Resubmitted => - // Note: For resubmitted tasks, we continue to use the metrics that belong to the - // first attempt of this task. This may not be 100% accurate because the first attempt - // could have failed half-way through. The correct fix would be to keep track of the - // metrics added by each attempt, but this is much more complicated. - return - case _: ExceptionFailure => - taskSummary.tasksFailed += 1 - case _ => - taskSummary.tasksComplete += 1 + // Note: For resubmitted tasks, we continue to use the metrics that belong to the + // first attempt of this task. This may not be 100% accurate because the first attempt + // could have failed half-way through. The correct fix would be to keep track of the + // metrics added by each attempt, but this is much more complicated. + if (taskEnd.reason == Resubmitted) { + return + } + if (info.successful) { + taskSummary.tasksComplete += 1 + } else { + taskSummary.tasksFailed += 1 } if (taskSummary.tasksActive >= 1) { taskSummary.tasksActive -= 1 From ae8a2b14966b1dfa10e620bb24ca6560778c20e7 Mon Sep 17 00:00:00 2001 From: Anderson Osagie Date: Wed, 9 Aug 2017 14:35:27 +0800 Subject: [PATCH 17/20] [SPARK-21176][WEB UI] Use a single ProxyServlet to proxy all workers and applications ## What changes were proposed in this pull request? Currently, each application and each worker creates their own proxy servlet. Each proxy servlet is backed by its own HTTP client and a relatively large number of selector threads. This is excessive but was fixed (to an extent) by https://github.com/apache/spark/pull/18437. However, a single HTTP client (backed by a single selector thread) should be enough to handle all proxy requests. This PR creates a single proxy servlet no matter how many applications and workers there are. ## How was this patch tested? . The unit tests for rewriting proxied locations and headers were updated. I then spun up a 100 node cluster to ensure that proxy'ing worked correctly jiangxb1987 Please let me know if there's anything else I can do to help push this thru. Thanks! Author: Anderson Osagie Closes #18499 from aosagie/fix/minimize-proxy-threads. --- .../apache/spark/deploy/master/Master.scala | 15 ++----- .../spark/deploy/master/ui/MasterWebUI.scala | 21 +++++---- .../org/apache/spark/ui/JettyUtils.scala | 45 ++++++++++--------- .../scala/org/apache/spark/ui/UISuite.scala | 20 ++++----- 4 files changed, 46 insertions(+), 55 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 4cc580eb75ca2..e030cac60a8e4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -133,6 +133,7 @@ private[deploy] class Master( masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort if (reverseProxy) { masterWebUiUrl = conf.get("spark.ui.reverseProxyUrl", masterWebUiUrl) + webUi.addProxy() logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " + s"Applications UIs are available at $masterWebUiUrl") } @@ -769,9 +770,6 @@ private[deploy] class Master( workers += worker idToWorker(worker.id) = worker addressToWorker(workerAddress) = worker - if (reverseProxy) { - webUi.addProxyTargets(worker.id, worker.webUiAddress) - } true } @@ -780,9 +778,7 @@ private[deploy] class Master( worker.setState(WorkerState.DEAD) idToWorker -= worker.id addressToWorker -= worker.endpoint.address - if (reverseProxy) { - webUi.removeProxyTargets(worker.id) - } + for (exec <- worker.executors.values) { logInfo("Telling app of lost executor: " + exec.id) exec.application.driver.send(ExecutorUpdated( @@ -844,9 +840,6 @@ private[deploy] class Master( endpointToApp(app.driver) = app addressToApp(appAddress) = app waitingApps += app - if (reverseProxy) { - webUi.addProxyTargets(app.id, app.desc.appUiUrl) - } } private def finishApplication(app: ApplicationInfo) { @@ -860,9 +853,7 @@ private[deploy] class Master( idToApp -= app.id endpointToApp -= app.driver addressToApp -= app.driver.address - if (reverseProxy) { - webUi.removeProxyTargets(app.id) - } + if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach { a => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index e42f41b97202a..35b7ddd46e4db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,10 +17,7 @@ package org.apache.spark.deploy.master.ui -import scala.collection.mutable.HashMap - -import org.eclipse.jetty.servlet.ServletContextHandler - +import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging import org.apache.spark.ui.{SparkUI, WebUI} @@ -38,7 +35,6 @@ class MasterWebUI( val masterEndpointRef = master.self val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) - private val proxyHandlers = new HashMap[String, ServletContextHandler] initialize() @@ -54,16 +50,19 @@ class MasterWebUI( "/driver/kill", "/", masterPage.handleDriverKillRequest, httpMethods = Set("POST"))) } - def addProxyTargets(id: String, target: String): Unit = { - val endTarget = target.stripSuffix("/") - val handler = createProxyHandler("/proxy/" + id, endTarget) + def addProxy(): Unit = { + val handler = createProxyHandler(idToUiAddress) attachHandler(handler) - proxyHandlers(id) = handler } - def removeProxyTargets(id: String): Unit = { - proxyHandlers.remove(id).foreach(detachHandler) + def idToUiAddress(id: String): Option[String] = { + val state = masterEndpointRef.askSync[MasterStateResponse](RequestMasterState) + val maybeWorkerUiAddress = state.workers.find(_.id == id).map(_.webUiAddress) + val maybeAppUiAddress = state.activeApps.find(_.id == id).map(_.desc.appUiUrl) + + maybeWorkerUiAddress.orElse(maybeAppUiAddress) } + } private[master] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 0fa96713eaf95..880cf08d34016 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -194,28 +194,32 @@ private[spark] object JettyUtils extends Logging { } /** Create a handler for proxying request to Workers and Application Drivers */ - def createProxyHandler( - prefix: String, - target: String): ServletContextHandler = { + def createProxyHandler(idToUiAddress: String => Option[String]): ServletContextHandler = { val servlet = new ProxyServlet { override def rewriteTarget(request: HttpServletRequest): String = { - val rewrittenURI = createProxyURI( - prefix, target, request.getRequestURI(), request.getQueryString()) - if (rewrittenURI == null) { - return null - } - if (!validateDestination(rewrittenURI.getHost(), rewrittenURI.getPort())) { - return null + val path = request.getPathInfo + if (path == null) return null + + val prefixTrailingSlashIndex = path.indexOf('/', 1) + val prefix = if (prefixTrailingSlashIndex == -1) { + path + } else { + path.substring(0, prefixTrailingSlashIndex) } - rewrittenURI.toString() + val id = prefix.drop(1) + + // Query master state for id's corresponding UI address + // If that address exists, turn it into a valid, target URI string or return null + idToUiAddress(id) + .map(createProxyURI(prefix, _, path, request.getQueryString)) + .filter(uri => uri != null && validateDestination(uri.getHost, uri.getPort)) + .map(_.toString) + .orNull } override def newHttpClient(): HttpClient = { // SPARK-21176: Use the Jetty logic to calculate the number of selector threads (#CPUs/2), // but limit it to 8 max. - // Otherwise, it might happen that we exhaust the threadpool since in reverse proxy mode - // a proxy is instantiated for each executor. If the head node has many processors, this - // can quickly add up to an unreasonably high number of threads. val numSelectors = math.max(1, math.min(8, Runtime.getRuntime().availableProcessors() / 2)) new HttpClient(new HttpClientTransportOverHTTP(numSelectors), null) } @@ -226,8 +230,8 @@ private[spark] object JettyUtils extends Logging { headerName: String, headerValue: String): String = { if (headerName.equalsIgnoreCase("location")) { - val newHeader = createProxyLocationHeader( - prefix, headerValue, clientRequest, serverResponse.getRequest().getURI()) + val newHeader = createProxyLocationHeader(headerValue, clientRequest, + serverResponse.getRequest().getURI()) if (newHeader != null) { return newHeader } @@ -239,8 +243,8 @@ private[spark] object JettyUtils extends Logging { val contextHandler = new ServletContextHandler val holder = new ServletHolder(servlet) - contextHandler.setContextPath(prefix) - contextHandler.addServlet(holder, "/") + contextHandler.setContextPath("/proxy") + contextHandler.addServlet(holder, "/*") contextHandler } @@ -438,7 +442,7 @@ private[spark] object JettyUtils extends Logging { val rest = path.substring(prefix.length()) if (!rest.isEmpty()) { - if (!rest.startsWith("/")) { + if (!rest.startsWith("/") && !uri.endsWith("/")) { uri.append("/") } uri.append(rest) @@ -458,14 +462,13 @@ private[spark] object JettyUtils extends Logging { } def createProxyLocationHeader( - prefix: String, headerValue: String, clientRequest: HttpServletRequest, targetUri: URI): String = { val toReplace = targetUri.getScheme() + "://" + targetUri.getAuthority() if (headerValue.startsWith(toReplace)) { clientRequest.getScheme() + "://" + clientRequest.getHeader("host") + - prefix + headerValue.substring(toReplace.length()) + clientRequest.getPathInfo() + headerValue.substring(toReplace.length()) } else { null } diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 0c3d4caeeabf9..0428903bb4d8b 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -200,36 +200,34 @@ class UISuite extends SparkFunSuite { } test("verify proxy rewrittenURI") { - val prefix = "/proxy/worker-id" + val prefix = "/worker-id" val target = "http://localhost:8081" - val path = "/proxy/worker-id/json" + val path = "/worker-id/json" var rewrittenURI = JettyUtils.createProxyURI(prefix, target, path, null) assert(rewrittenURI.toString() === "http://localhost:8081/json") rewrittenURI = JettyUtils.createProxyURI(prefix, target, path, "test=done") assert(rewrittenURI.toString() === "http://localhost:8081/json?test=done") - rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/proxy/worker-id", null) + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id", null) assert(rewrittenURI.toString() === "http://localhost:8081") - rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/proxy/worker-id/test%2F", null) + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id/test%2F", null) assert(rewrittenURI.toString() === "http://localhost:8081/test%2F") - rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/proxy/worker-id/%F0%9F%98%84", null) + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-id/%F0%9F%98%84", null) assert(rewrittenURI.toString() === "http://localhost:8081/%F0%9F%98%84") - rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/proxy/worker-noid/json", null) + rewrittenURI = JettyUtils.createProxyURI(prefix, target, "/worker-noid/json", null) assert(rewrittenURI === null) } test("verify rewriting location header for reverse proxy") { val clientRequest = mock(classOf[HttpServletRequest]) var headerValue = "http://localhost:4040/jobs" - val prefix = "/proxy/worker-id" val targetUri = URI.create("http://localhost:4040") when(clientRequest.getScheme()).thenReturn("http") when(clientRequest.getHeader("host")).thenReturn("localhost:8080") - var newHeader = JettyUtils.createProxyLocationHeader( - prefix, headerValue, clientRequest, targetUri) + when(clientRequest.getPathInfo()).thenReturn("/proxy/worker-id") + var newHeader = JettyUtils.createProxyLocationHeader(headerValue, clientRequest, targetUri) assert(newHeader.toString() === "http://localhost:8080/proxy/worker-id/jobs") headerValue = "http://localhost:4041/jobs" - newHeader = JettyUtils.createProxyLocationHeader( - prefix, headerValue, clientRequest, targetUri) + newHeader = JettyUtils.createProxyLocationHeader(headerValue, clientRequest, targetUri) assert(newHeader === null) } From b35660dd0e930f4b484a079d9e2516b0a7dacf1d Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 9 Aug 2017 14:44:10 +0800 Subject: [PATCH 18/20] [SPARK-21523][ML] update breeze to 0.13.2 for an emergency bugfix in strong wolfe line search ## What changes were proposed in this pull request? Update breeze to 0.13.1 for an emergency bugfix in strong wolfe line search https://github.com/scalanlp/breeze/pull/651 ## How was this patch tested? N/A Author: WeichenXu Closes #18797 from WeichenXu123/update-breeze. --- dev/deps/spark-deps-hadoop-2.6 | 4 ++-- dev/deps/spark-deps-hadoop-2.7 | 4 ++-- .../ml/regression/AFTSurvivalRegression.scala | 2 ++ .../ml/regression/AFTSurvivalRegressionSuite.scala | 1 - .../org/apache/spark/ml/util/MLTestingUtils.scala | 1 - .../spark/mllib/optimization/LBFGSSuite.scala | 4 ++-- pom.xml | 2 +- python/pyspark/ml/regression.py | 14 +++++++------- 8 files changed, 16 insertions(+), 16 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index a41183accf718..d7587fbcb4b1b 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -22,8 +22,8 @@ avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.11-0.13.1.jar -breeze_2.11-0.13.1.jar +breeze-macros_2.11-0.13.2.jar +breeze_2.11-0.13.2.jar calcite-avatica-1.2.0-incubating.jar calcite-core-1.2.0-incubating.jar calcite-linq4j-1.2.0-incubating.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 5e1321be14d0c..887eeca9e422a 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -22,8 +22,8 @@ avro-mapred-1.7.7-hadoop2.jar base64-2.3.8.jar bcprov-jdk15on-1.51.jar bonecp-0.8.0.RELEASE.jar -breeze-macros_2.11-0.13.1.jar -breeze_2.11-0.13.1.jar +breeze-macros_2.11-0.13.2.jar +breeze_2.11-0.13.2.jar calcite-avatica-1.2.0-incubating.jar calcite-core-1.2.0-incubating.jar calcite-linq4j-1.2.0-incubating.jar diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 094853b6f4802..0891994530f88 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -553,6 +553,8 @@ private class AFTAggregator( val ti = data.label val delta = data.censor + require(ti > 0.0, "The lifetime or label should be greater than 0.") + val localFeaturesStd = bcFeaturesStd.value val margin = { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala index fb39e50a83552..02e5c6d294f44 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/AFTSurvivalRegressionSuite.scala @@ -364,7 +364,6 @@ class AFTSurvivalRegressionSuite test("should support all NumericType censors, and not support other types") { val df = spark.createDataFrame(Seq( - (0, Vectors.dense(0)), (1, Vectors.dense(1)), (2, Vectors.dense(2)), (3, Vectors.dense(3)), diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index bef79e634f75f..aef81c8c173a0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -160,7 +160,6 @@ object MLTestingUtils extends SparkFunSuite { featuresColName: String = "features", censorColName: String = "censor"): Map[NumericType, DataFrame] = { val df = spark.createDataFrame(Seq( - (0, Vectors.dense(0)), (1, Vectors.dense(1)), (2, Vectors.dense(2)), (3, Vectors.dense(3)), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala index 3d6a9f8d84cac..69c303ee932e0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/LBFGSSuite.scala @@ -191,8 +191,8 @@ class LBFGSSuite extends SparkFunSuite with MLlibTestSparkContext with Matchers // With smaller convergenceTol, it takes more steps. assert(lossLBFGS3.length > lossLBFGS2.length) - // Based on observation, lossLBFGS3 runs 7 iterations, no theoretically guaranteed. - assert(lossLBFGS3.length == 7) + // Based on observation, lossLBFGS3 runs 6 iterations, no theoretically guaranteed. + assert(lossLBFGS3.length == 6) assert((lossLBFGS3(4) - lossLBFGS3(5)) / lossLBFGS3(4) < convergenceTol) } diff --git a/pom.xml b/pom.xml index 95dd52acdfd5a..500fa1c421127 100644 --- a/pom.xml +++ b/pom.xml @@ -684,7 +684,7 @@ org.scalanlp breeze_${scala.binary.version} - 0.13.1 + 0.13.2 diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 72374acbe019f..9d5b768091cf4 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -1123,7 +1123,7 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([ ... (1.0, Vectors.dense(1.0), 1.0), - ... (0.0, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) + ... (1e-40, Vectors.sparse(1, [], []), 0.0)], ["label", "features", "censor"]) >>> aftsr = AFTSurvivalRegression() >>> model = aftsr.fit(df) >>> model.predict(Vectors.dense(6.3)) @@ -1131,12 +1131,12 @@ class AFTSurvivalRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredi >>> model.predictQuantiles(Vectors.dense(6.3)) DenseVector([0.0101, 0.0513, 0.1054, 0.2877, 0.6931, 1.3863, 2.3026, 2.9957, 4.6052]) >>> model.transform(df).show() - +-----+---------+------+----------+ - |label| features|censor|prediction| - +-----+---------+------+----------+ - | 1.0| [1.0]| 1.0| 1.0| - | 0.0|(1,[],[])| 0.0| 1.0| - +-----+---------+------+----------+ + +-------+---------+------+----------+ + | label| features|censor|prediction| + +-------+---------+------+----------+ + | 1.0| [1.0]| 1.0| 1.0| + |1.0E-40|(1,[],[])| 0.0| 1.0| + +-------+---------+------+----------+ ... >>> aftsr_path = temp_path + "/aftsr" >>> aftsr.save(aftsr_path) From 6426adffaf152651c30d481bb925d5025fd6130a Mon Sep 17 00:00:00 2001 From: 10087686 Date: Wed, 9 Aug 2017 18:45:38 +0800 Subject: [PATCH 19/20] [SPARK-21663][TESTS] test("remote fetch below max RPC message size") should call masterTracker.stop() in MapOutputTrackerSuite MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: 10087686 ## What changes were proposed in this pull request? After Unit tests end,there should be call masterTracker.stop() to free resource; (Please fill in changes proposed in this fix) ## How was this patch tested? Run Unit tests; (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Author: 10087686 Closes #18867 from wangjiaochun/mapout. --- .../test/scala/org/apache/spark/MapOutputTrackerSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index bc3d23e3fbb29..493ae514e78fe 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -176,7 +176,8 @@ class MapOutputTrackerSuite extends SparkFunSuite { val masterTracker = newTrackerMaster(newConf) val rpcEnv = createRpcEnv("spark") val masterEndpoint = new MapOutputTrackerMasterEndpoint(rpcEnv, masterTracker, newConf) - rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) + masterTracker.trackerEndpoint = + rpcEnv.setupEndpoint(MapOutputTracker.ENDPOINT_NAME, masterEndpoint) // Message size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) @@ -191,7 +192,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { verify(rpcCallContext, timeout(30000)).reply(any()) assert(0 == masterTracker.getNumCachedSerializedBroadcast) -// masterTracker.stop() // this throws an exception + masterTracker.stop() rpcEnv.shutdown() } From 83fe3b5e10f8dc62245ea37143abb96be0f39805 Mon Sep 17 00:00:00 2001 From: vinodkc Date: Wed, 9 Aug 2017 15:18:38 +0200 Subject: [PATCH 20/20] [SPARK-21665][CORE] Need to close resources after use ## What changes were proposed in this pull request? Resources in Core - SparkSubmitArguments.scala, Spark-launcher - AbstractCommandBuilder.java, resource-managers- YARN - Client.scala are released ## How was this patch tested? No new test cases added, Unit test have been passed Author: vinodkc Closes #18880 from vinodkc/br_fixresouceleak. --- .../spark/deploy/SparkSubmitArguments.scala | 9 +++++---- .../spark/launcher/AbstractCommandBuilder.java | 16 +++------------- 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 3721b98d68685..a7722e4f86023 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -207,11 +207,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S uriScheme match { case "file" => try { - val jar = new JarFile(uri.getPath) - // Note that this might still return null if no main-class is set; we catch that later - mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + Utils.tryWithResource(new JarFile(uri.getPath)) { jar => + // Note that this might still return null if no main-class is set; we catch that later + mainClass = jar.getManifest.getMainAttributes.getValue("Main-Class") + } } catch { - case e: Exception => + case _: Exception => SparkSubmit.printErrorAndExit(s"Cannot load main class from JAR $primaryResource") } case _ => diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 860ab35852331..44028c58ac489 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -291,24 +291,14 @@ private Properties loadPropertiesFile() throws IOException { } if (propsFile.isFile()) { - FileInputStream fd = null; - try { - fd = new FileInputStream(propsFile); - props.load(new InputStreamReader(fd, StandardCharsets.UTF_8)); + try (InputStreamReader isr = new InputStreamReader( + new FileInputStream(propsFile), StandardCharsets.UTF_8)) { + props.load(isr); for (Map.Entry e : props.entrySet()) { e.setValue(e.getValue().toString().trim()); } - } finally { - if (fd != null) { - try { - fd.close(); - } catch (IOException e) { - // Ignore. - } - } } } - return props; }