From 54e702c0dd585e5d6e929b427e31a34614ae257c Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Sun, 14 Jun 2020 14:26:11 -0700 Subject: [PATCH 01/28] [SPARK-31970][CORE] Make MDC configuration step be consistent between setLocalProperty and log4j.properties ### What changes were proposed in this pull request? This PR proposes to use "mdc.XXX" as the consistent key for both `sc.setLocalProperty` and `log4j.properties` when setting up configurations for MDC. ### Why are the changes needed? It's weird that we use "mdc.XXX" as key to set MDC value via `sc.setLocalProperty` while we use "XXX" as key to set MDC pattern in log4j.properties. It could also bring extra burden to the user. ### Does this PR introduce _any_ user-facing change? No, as MDC feature is added in version 3.1, which hasn't been released. ### How was this patch tested? Tested manually. Closes #28801 from Ngone51/consistent-mdc. Authored-by: yi.wu Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/executor/Executor.scala | 7 ++----- docs/configuration.md | 8 ++++---- 2 files changed, 6 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 93d1acdd2d156..c8b1afeebac0d 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -323,10 +323,7 @@ private[spark] class Executor( val threadName = s"Executor task launch worker for task $taskId" val taskName = taskDescription.name val mdcProperties = taskDescription.properties.asScala - .filter(_._1.startsWith("mdc.")).map { item => - val key = item._1.substring(4) - (key, item._2) - }.toSeq + .filter(_._1.startsWith("mdc.")).toSeq /** If specified, this task has been killed and this option contains the reason. */ @volatile private var reasonIfKilled: Option[String] = None @@ -705,7 +702,7 @@ private[spark] class Executor( MDC.clear() mdc.foreach { case (key, value) => MDC.put(key, value) } // avoid overriding the takName by the user - MDC.put("taskName", taskName) + MDC.put("mdc.taskName", taskName) } /** diff --git a/docs/configuration.md b/docs/configuration.md index 420942f7b7bbb..706c2552b1d17 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2955,11 +2955,11 @@ Spark uses [log4j](http://logging.apache.org/log4j/) for logging. You can config `log4j.properties` file in the `conf` directory. One way to start is to copy the existing `log4j.properties.template` located there. -By default, Spark adds 1 record to the MDC (Mapped Diagnostic Context): `taskName`, which shows something -like `task 1.0 in stage 0.0`. You can add `%X{taskName}` to your patternLayout in +By default, Spark adds 1 record to the MDC (Mapped Diagnostic Context): `mdc.taskName`, which shows something +like `task 1.0 in stage 0.0`. You can add `%X{mdc.taskName}` to your patternLayout in order to print it in the logs. -Moreover, you can use `spark.sparkContext.setLocalProperty("mdc." + name, "value")` to add user specific data into MDC. -The key in MDC will be the string after the `mdc.` prefix. +Moreover, you can use `spark.sparkContext.setLocalProperty(s"mdc.$name", "value")` to add user specific data into MDC. +The key in MDC will be the string of "mdc.$name". # Overriding configuration directory From f5f6eee3045e90e02fc7e999f616b5a021d7c724 Mon Sep 17 00:00:00 2001 From: iRakson Date: Sun, 14 Jun 2020 16:41:59 -0500 Subject: [PATCH 02/28] [SPARK-31642][FOLLOWUP] Fix Sorting for duration column and make Status column sortable ### What changes were proposed in this pull request? In #28485 pagination support for tables of Structured Streaming Tab was added. It missed 2 things: * For sorting duration column, `String` was used which sometimes gives wrong results(consider `"3 ms"` and `"12 ms"`). Now we first sort the duration column and then convert it to readable String * Status column was not made sortable. ### Why are the changes needed? To fix the wrong result for sorting and making Status column sortable. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? After changes: Screenshot 2020-06-08 at 2 18 48 PM Closes #28752 from iRakson/ssTests. Authored-by: iRakson Signed-off-by: Sean Owen --- .../spark/sql/streaming/ui/StreamingQueryPage.scala | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala index b969e41e4e55c..b98fdf16eef31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ui/StreamingQueryPage.scala @@ -152,7 +152,7 @@ class StreamingQueryPagedTable( val headerAndCss: Seq[(String, Boolean, Option[String])] = { Seq( ("Name", true, None), - ("Status", false, None), + ("Status", true, None), ("ID", true, None), ("Run ID", true, None), ("Start Time", true, None), @@ -197,7 +197,7 @@ class StreamingQueryPagedTable( {streamingQuery.id} {streamingQuery.runId} {SparkUIUtils.formatDate(streamingQuery.startTimestamp)} - {query.duration} + {SparkUIUtils.formatDurationVerbose(query.duration)} {withNoProgress(streamingQuery, {query.avgInput.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {query.avgProcess.formatted("%.2f")}, "NaN")} {withNoProgress(streamingQuery, {streamingQuery.lastProgress.batchId}, "NaN")} @@ -207,7 +207,7 @@ class StreamingQueryPagedTable( } case class StructuredStreamingRow( - duration: String, + duration: Long, avgInput: Double, avgProcess: Double, streamingUIData: StreamingQueryUIData) @@ -224,12 +224,12 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St private def streamingRow(query: StreamingQueryUIData): StructuredStreamingRow = { val duration = if (isActive) { - SparkUIUtils.formatDurationVerbose(System.currentTimeMillis() - query.startTimestamp) + System.currentTimeMillis() - query.startTimestamp } else { withNoProgress(query, { val endTimeMs = query.lastProgress.timestamp - SparkUIUtils.formatDurationVerbose(parseProgressTimestamp(endTimeMs) - query.startTimestamp) - }, "-") + parseProgressTimestamp(endTimeMs) - query.startTimestamp + }, 0) } val avgInput = (query.recentProgress.map(p => withNumberInvalid(p.inputRowsPerSecond)).sum / @@ -244,6 +244,7 @@ class StreamingQueryDataSource(uiData: Seq[StreamingQueryUIData], sortColumn: St private def ordering(sortColumn: String, desc: Boolean): Ordering[StructuredStreamingRow] = { val ordering: Ordering[StructuredStreamingRow] = sortColumn match { case "Name" => Ordering.by(q => UIUtils.getQueryName(q.streamingUIData)) + case "Status" => Ordering.by(q => UIUtils.getQueryStatus(q.streamingUIData)) case "ID" => Ordering.by(_.streamingUIData.id) case "Run ID" => Ordering.by(_.streamingUIData.runId) case "Start Time" => Ordering.by(_.streamingUIData.startTimestamp) From 8282bbf12d4e174986a649023ce3984aae7d7755 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 15 Jun 2020 11:01:56 +0900 Subject: [PATCH 03/28] [SPARK-27633][SQL] Remove redundant aliases in NestedColumnAliasing ## What changes were proposed in this pull request? In NestedColumnAliasing rule, we create aliases for nested field access in project list. We considered that top level parent field and nested fields under it were both accessed. In the case, we don't create the aliases because they are redundant. There is another case, where a nested parent field and nested fields under it were both accessed, which we don't consider now. We don't need to create aliases in this case too. ## How was this patch tested? Added test. Closes #24525 from viirya/SPARK-27633. Lead-authored-by: Liang-Chi Hsieh Co-authored-by: Liang-Chi Hsieh Signed-off-by: HyukjinKwon --- .../optimizer/NestedColumnAliasing.scala | 13 +++++++- .../optimizer/NestedColumnAliasingSuite.scala | 31 +++++++++++++++++++ 2 files changed, 43 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala index 7f49d8db4852d..118f41f9cd232 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -154,9 +154,20 @@ object NestedColumnAliasing { .filter(!_.references.subsetOf(exclusiveAttrSet)) .groupBy(_.references.head) .flatMap { case (attr, nestedFields: Seq[ExtractValue]) => + // Remove redundant `ExtractValue`s if they share the same parent nest field. + // For example, when `a.b` and `a.b.c` are in project list, we only need to alias `a.b`. + // We only need to deal with two `ExtractValue`: `GetArrayStructFields` and + // `GetStructField`. Please refer to the method `collectRootReferenceAndExtractValue`. + val dedupNestedFields = nestedFields.filter { + case e @ (_: GetStructField | _: GetArrayStructFields) => + val child = e.children.head + nestedFields.forall(f => child.find(_.semanticEquals(f)).isEmpty) + case _ => true + } + // Each expression can contain multiple nested fields. // Note that we keep the original names to deliver to parquet in a case-sensitive way. - val nestedFieldToAlias = nestedFields.distinct.map { f => + val nestedFieldToAlias = dedupNestedFields.distinct.map { f => val exprId = NamedExpression.newExprId (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala index 7b1735a6f04ab..0425846637f30 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -261,6 +261,37 @@ class NestedColumnAliasingSuite extends SchemaPruningTest { comparePlans(optimized, expected) } + test("SPARK-27633: Do not generate redundant aliases if parent nested field is aliased too") { + val nestedRelation = LocalRelation('a.struct('b.struct('c.int, + 'd.struct('f.int, 'g.int)), 'e.int)) + + // `a.b` + val first = 'a.getField("b") + // `a.b.c` + 1 + val second = 'a.getField("b").getField("c") + Literal(1) + // `a.b.d.f` + val last = 'a.getField("b").getField("d").getField("f") + + val query = nestedRelation + .limit(5) + .select(first, second, last) + .analyze + + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = nestedRelation + .select(first.as(aliases(0))) + .limit(5) + .select($"${aliases(0)}".as("a.b"), + ($"${aliases(0)}".getField("c") + Literal(1)).as("(a.b.c + 1)"), + $"${aliases(0)}".getField("d").getField("f").as("a.b.d.f")) + .analyze + + comparePlans(optimized, expected) + } + test("Nested field pruning for Project and Generate") { val query = contact .generate(Explode('friends.getField("first")), outputNames = Seq("explode")) From a0187cd6b59a6b6bb2cadc6711bb663d4d35a844 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 15 Jun 2020 06:10:24 +0000 Subject: [PATCH 04/28] [SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber ### What changes were proposed in this pull request? This PR brings 02f32cfae4082696a6c013fb6beb150040be3228 back which reverted by 4a25200cd74561152e42fb92b02e860f2d5c7d4e because of maven test failure diffs newly made: 1. add a missing log4j file to test resources 2. Call `SessionState.detachSession()` to clean the thread local one in `afterAll`. 3. Not use dedicated JVMs for sbt test runner too ### Why are the changes needed? fix the maven test ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? add new tests Closes #28797 from yaooqinn/SPARK-31926-NEW. Authored-by: Kent Yao Signed-off-by: Wenchen Fan --- project/SparkBuild.scala | 1 - .../src/test/resources/log4j.properties | 65 +++++++++++++++++++ .../thriftserver/SharedThriftServer.scala | 50 ++++++++++---- .../ThriftServerQueryTestSuite.scala | 3 + .../ThriftServerWithSparkContextSuite.scala | 11 +++- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- .../cli/thrift/ThriftBinaryCLIService.java | 11 +++- .../service/cli/thrift/ThriftCLIService.java | 3 + .../cli/thrift/ThriftHttpCLIService.java | 21 ++++-- 11 files changed, 170 insertions(+), 30 deletions(-) create mode 100644 sql/hive-thriftserver/src/test/resources/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 0035f1d95a90d..04a3fc4b63050 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,7 +480,6 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", - "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..21975ba818142 --- /dev/null +++ b/sql/hive-thriftserver/src/test/resources/log4j.properties @@ -0,0 +1,65 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Set everything to be logged to the file hive-thriftserver/target/unit-tests.log +log4j.rootLogger=DEBUG, CA, FA + +#Console Appender +log4j.appender.CA=org.apache.log4j.ConsoleAppender +log4j.appender.CA.layout=org.apache.log4j.PatternLayout +log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n +log4j.appender.CA.Threshold = WARN + + +#File Appender +log4j.appender.FA=org.apache.log4j.FileAppender +log4j.appender.FA.append=false +log4j.appender.FA.file=target/unit-tests.log +log4j.appender.FA.layout=org.apache.log4j.PatternLayout +log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Set the logger level of File Appender to WARN +log4j.appender.FA.Threshold = DEBUG + +# Some packages are noisy for no good reason. +log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false +log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF + +log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF + +log4j.additivity.hive.log=false +log4j.logger.hive.log=OFF + +log4j.additivity.parquet.hadoop.ParquetRecordReader=false +log4j.logger.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false +log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF + +log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false +log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF + +log4j.additivity.hive.ql.metadata.Hive=false +log4j.logger.hive.ql.metadata.Hive=OFF + +log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false +log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR + +# Parquet related logging +log4j.logger.org.apache.parquet.CorruptStatistics=ERROR +log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index e002bc0117c8b..1c33abff0780c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,6 +24,7 @@ import scala.concurrent.duration._ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession @@ -33,6 +34,8 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 + def mode: ServerMode.Value + override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -50,14 +53,21 @@ trait SharedThriftServer extends SharedSparkSession { hiveServer2.stop() } finally { super.afterAll() + SessionState.detachSession() } } + protected def jdbcUri: String = if (mode == ServerMode.http) { + s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" + } else { + s"jdbc:hive2://localhost:$serverPort/" + } + protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } + fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } val statements = connections.map(_.createStatement()) try { @@ -69,23 +79,35 @@ trait SharedThriftServer extends SharedSparkSession { } private def startThriftServer(attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") + logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could + // randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService if t.getPortNumber != 0 => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") - case _ => - } + sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") + sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) + + try { + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: mode=$mode, port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } + } + } catch { + case e: Exception => + logError("Error start hive server with Context ", e) + if (hiveServer2 != null) { + hiveServer2.stop() + } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 15cc3109da3f7..553f10a275bce 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,6 +54,9 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { + + override def mode: ServerMode.Value = ServerMode.binary + override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 3e1fce78ae71c..d6420dee41adb 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -class ThriftServerWithSparkContextSuite extends SharedThriftServer { +trait ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,3 +42,12 @@ class ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } + + +class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.binary +} + +class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { + override def mode: ServerMode.Value = ServerMode.http +} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index e1ee503b81209..00bdf7e19126e 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -45,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -100,6 +101,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 8fce9d9383438..783e5795aca76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,6 +175,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -633,6 +634,8 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 1099a00b67eb7..bd64c777c1d76 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -53,13 +54,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -150,6 +146,19 @@ public void run() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index a7de9c0f3d0d2..ce79e3c8228a6 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +47,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - public void run() { + protected void initializeServer() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,6 +102,14 @@ public void run() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + @Override + public void run() { + try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index d41c3b493bb47..e46799a1c427d 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,6 +176,7 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { + initializeServer(); new Thread(this).start(); isStarted = true; } @@ -670,6 +671,8 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } + protected abstract void initializeServer(); + @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 73d5f84476af0..ab9ed5b1f371e 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,6 +28,7 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; +import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -54,13 +55,8 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ @Override - public void run() { + protected void initializeServer() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -151,6 +147,19 @@ public void run() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); + } catch (Exception t) { + throw new ServiceException("Error initializing " + getName(), t); + } + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { httpServer.join(); } catch (Throwable t) { LOG.error( From 9d95f1b010293c63eff14ed28773c00c35207dee Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 15 Jun 2020 07:25:56 +0000 Subject: [PATCH 05/28] [SPARK-31992][SQL] Benchmark the EXCEPTION rebase mode ### What changes were proposed in this pull request? - Modify `DateTimeRebaseBenchmark` to benchmark the default date-time rebasing mode - `EXCEPTION` for saving/loading dates/timestamps from/to parquet files. The mode is benchmarked for modern timestamps after 1900-01-01 00:00:00Z and dates after 1582-10-15. - Regenerate benchmark results in the environment: | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge | | AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) | | Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | ### Why are the changes needed? The `EXCEPTION` rebasing mode is the default mode of the SQL configs `spark.sql.legacy.parquet.datetimeRebaseModeInRead` and `spark.sql.legacy.parquet.datetimeRebaseModeInWrite`. The changes are needed to improve benchmark coverage for default settings. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the benchmark and check results manually. Closes #28829 from MaxGekk/benchmark-exception-mode. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../DateTimeRebaseBenchmark-jdk11-results.txt | 156 ++++++++++-------- .../DateTimeRebaseBenchmark-results.txt | 156 ++++++++++-------- .../benchmark/DateTimeRebaseBenchmark.scala | 28 ++-- 3 files changed, 183 insertions(+), 157 deletions(-) diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt index 5d107c1e50fd8..05896a4d69b47 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk11-results.txt @@ -6,97 +6,109 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 20802 20802 0 4.8 208.0 1.0X -before 1582, noop 10728 10728 0 9.3 107.3 1.9X -after 1582, rebase off 32924 32924 0 3.0 329.2 0.6X -after 1582, rebase on 32627 32627 0 3.1 326.3 0.6X -before 1582, rebase off 21576 21576 0 4.6 215.8 1.0X -before 1582, rebase on 23115 23115 0 4.3 231.2 0.9X +after 1582, noop 20023 20023 0 5.0 200.2 1.0X +before 1582, noop 10729 10729 0 9.3 107.3 1.9X +after 1582, rebase EXCEPTION 31834 31834 0 3.1 318.3 0.6X +after 1582, rebase LEGACY 31997 31997 0 3.1 320.0 0.6X +after 1582, rebase CORRECTED 31712 31712 0 3.2 317.1 0.6X +before 1582, rebase LEGACY 23663 23663 0 4.2 236.6 0.8X +before 1582, rebase CORRECTED 22749 22749 0 4.4 227.5 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase off 12880 12984 178 7.8 128.8 1.0X -after 1582, vec off, rebase on 13118 13255 174 7.6 131.2 1.0X -after 1582, vec on, rebase off 3645 3698 76 27.4 36.4 3.5X -after 1582, vec on, rebase on 3709 3727 15 27.0 37.1 3.5X -before 1582, vec off, rebase off 13014 13051 36 7.7 130.1 1.0X -before 1582, vec off, rebase on 14195 14242 48 7.0 142.0 0.9X -before 1582, vec on, rebase off 3680 3773 92 27.2 36.8 3.5X -before 1582, vec on, rebase on 4310 4381 87 23.2 43.1 3.0X +after 1582, vec off, rebase EXCEPTION 12984 13262 257 7.7 129.8 1.0X +after 1582, vec off, rebase LEGACY 13278 13330 50 7.5 132.8 1.0X +after 1582, vec off, rebase CORRECTED 13202 13255 50 7.6 132.0 1.0X +after 1582, vec on, rebase EXCEPTION 3823 3853 40 26.2 38.2 3.4X +after 1582, vec on, rebase LEGACY 3846 3876 27 26.0 38.5 3.4X +after 1582, vec on, rebase CORRECTED 3775 3838 62 26.5 37.7 3.4X +before 1582, vec off, rebase LEGACY 13671 13692 26 7.3 136.7 0.9X +before 1582, vec off, rebase CORRECTED 13387 13476 106 7.5 133.9 1.0X +before 1582, vec on, rebase LEGACY 4477 4484 7 22.3 44.8 2.9X +before 1582, vec on, rebase CORRECTED 3729 3773 50 26.8 37.3 3.5X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 3026 3026 0 33.1 30.3 1.0X -before 1900, noop 2995 2995 0 33.4 30.0 1.0X -after 1900, rebase off 24294 24294 0 4.1 242.9 0.1X -after 1900, rebase on 24480 24480 0 4.1 244.8 0.1X -before 1900, rebase off 31120 31120 0 3.2 311.2 0.1X -before 1900, rebase on 31201 31201 0 3.2 312.0 0.1X +after 1900, noop 3020 3020 0 33.1 30.2 1.0X +before 1900, noop 3013 3013 0 33.2 30.1 1.0X +after 1900, rebase EXCEPTION 28796 28796 0 3.5 288.0 0.1X +after 1900, rebase LEGACY 28869 28869 0 3.5 288.7 0.1X +after 1900, rebase CORRECTED 28522 28522 0 3.5 285.2 0.1X +before 1900, rebase LEGACY 30594 30594 0 3.3 305.9 0.1X +before 1900, rebase CORRECTED 30743 30743 0 3.3 307.4 0.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 18283 18309 39 5.5 182.8 1.0X -after 1900, vec off, rebase on 18235 18269 53 5.5 182.4 1.0X -after 1900, vec on, rebase off 9563 9589 27 10.5 95.6 1.9X -after 1900, vec on, rebase on 9463 9554 81 10.6 94.6 1.9X -before 1900, vec off, rebase off 21377 21469 118 4.7 213.8 0.9X -before 1900, vec off, rebase on 21265 21422 156 4.7 212.7 0.9X -before 1900, vec on, rebase off 12481 12524 46 8.0 124.8 1.5X -before 1900, vec on, rebase on 12360 12482 105 8.1 123.6 1.5X +after 1900, vec off, rebase EXCEPTION 19325 19468 135 5.2 193.3 1.0X +after 1900, vec off, rebase LEGACY 19568 19602 30 5.1 195.7 1.0X +after 1900, vec off, rebase CORRECTED 19532 19538 6 5.1 195.3 1.0X +after 1900, vec on, rebase EXCEPTION 9884 9990 94 10.1 98.8 2.0X +after 1900, vec on, rebase LEGACY 9933 9985 49 10.1 99.3 1.9X +after 1900, vec on, rebase CORRECTED 9967 10043 76 10.0 99.7 1.9X +before 1900, vec off, rebase LEGACY 24162 24198 37 4.1 241.6 0.8X +before 1900, vec off, rebase CORRECTED 24034 24056 20 4.2 240.3 0.8X +before 1900, vec on, rebase LEGACY 12548 12625 72 8.0 125.5 1.5X +before 1900, vec on, rebase CORRECTED 12580 12660 115 7.9 125.8 1.5X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2984 2984 0 33.5 29.8 1.0X -before 1900, noop 3003 3003 0 33.3 30.0 1.0X -after 1900, rebase off 15814 15814 0 6.3 158.1 0.2X -after 1900, rebase on 16250 16250 0 6.2 162.5 0.2X -before 1900, rebase off 16026 16026 0 6.2 160.3 0.2X -before 1900, rebase on 19735 19735 0 5.1 197.3 0.2X +after 1900, noop 3159 3159 0 31.7 31.6 1.0X +before 1900, noop 3038 3038 0 32.9 30.4 1.0X +after 1900, rebase EXCEPTION 16885 16885 0 5.9 168.8 0.2X +after 1900, rebase LEGACY 17171 17171 0 5.8 171.7 0.2X +after 1900, rebase CORRECTED 17353 17353 0 5.8 173.5 0.2X +before 1900, rebase LEGACY 20579 20579 0 4.9 205.8 0.2X +before 1900, rebase CORRECTED 17544 17544 0 5.7 175.4 0.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15292 15351 57 6.5 152.9 1.0X -after 1900, vec off, rebase on 15753 15886 173 6.3 157.5 1.0X -after 1900, vec on, rebase off 4879 4923 52 20.5 48.8 3.1X -after 1900, vec on, rebase on 5018 5038 18 19.9 50.2 3.0X -before 1900, vec off, rebase off 15257 15311 53 6.6 152.6 1.0X -before 1900, vec off, rebase on 18459 18537 90 5.4 184.6 0.8X -before 1900, vec on, rebase off 4929 4946 15 20.3 49.3 3.1X -before 1900, vec on, rebase on 8254 8339 93 12.1 82.5 1.9X +after 1900, vec off, rebase EXCEPTION 16304 16345 58 6.1 163.0 1.0X +after 1900, vec off, rebase LEGACY 16503 16585 75 6.1 165.0 1.0X +after 1900, vec off, rebase CORRECTED 16413 16463 44 6.1 164.1 1.0X +after 1900, vec on, rebase EXCEPTION 5017 5034 29 19.9 50.2 3.2X +after 1900, vec on, rebase LEGACY 5060 5094 30 19.8 50.6 3.2X +after 1900, vec on, rebase CORRECTED 4969 4971 1 20.1 49.7 3.3X +before 1900, vec off, rebase LEGACY 19767 20001 203 5.1 197.7 0.8X +before 1900, vec off, rebase CORRECTED 16421 16465 38 6.1 164.2 1.0X +before 1900, vec on, rebase LEGACY 8535 8608 64 11.7 85.4 1.9X +before 1900, vec on, rebase CORRECTED 5044 5077 32 19.8 50.4 3.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2987 2987 0 33.5 29.9 1.0X -before 1900, noop 3002 3002 0 33.3 30.0 1.0X -after 1900, rebase off 15215 15215 0 6.6 152.1 0.2X -after 1900, rebase on 15577 15577 0 6.4 155.8 0.2X -before 1900, rebase off 15505 15505 0 6.4 155.1 0.2X -before 1900, rebase on 19143 19143 0 5.2 191.4 0.2X +after 1900, noop 2995 2995 0 33.4 29.9 1.0X +before 1900, noop 2981 2981 0 33.5 29.8 1.0X +after 1900, rebase EXCEPTION 16196 16196 0 6.2 162.0 0.2X +after 1900, rebase LEGACY 16550 16550 0 6.0 165.5 0.2X +after 1900, rebase CORRECTED 16908 16908 0 5.9 169.1 0.2X +before 1900, rebase LEGACY 20087 20087 0 5.0 200.9 0.1X +before 1900, rebase CORRECTED 17171 17171 0 5.8 171.7 0.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15330 15436 113 6.5 153.3 1.0X -after 1900, vec off, rebase on 15515 15549 30 6.4 155.1 1.0X -after 1900, vec on, rebase off 6056 6074 19 16.5 60.6 2.5X -after 1900, vec on, rebase on 6376 6390 14 15.7 63.8 2.4X -before 1900, vec off, rebase off 15490 15523 36 6.5 154.9 1.0X -before 1900, vec off, rebase on 18613 18685 118 5.4 186.1 0.8X -before 1900, vec on, rebase off 6065 6109 41 16.5 60.6 2.5X -before 1900, vec on, rebase on 9052 9082 32 11.0 90.5 1.7X +after 1900, vec off, rebase EXCEPTION 16688 16787 88 6.0 166.9 1.0X +after 1900, vec off, rebase LEGACY 17383 17462 73 5.8 173.8 1.0X +after 1900, vec off, rebase CORRECTED 17317 17329 11 5.8 173.2 1.0X +after 1900, vec on, rebase EXCEPTION 6342 6348 6 15.8 63.4 2.6X +after 1900, vec on, rebase LEGACY 6500 6521 18 15.4 65.0 2.6X +after 1900, vec on, rebase CORRECTED 6164 6172 11 16.2 61.6 2.7X +before 1900, vec off, rebase LEGACY 20575 20665 81 4.9 205.7 0.8X +before 1900, vec off, rebase CORRECTED 17239 17290 61 5.8 172.4 1.0X +before 1900, vec on, rebase LEGACY 9310 9373 60 10.7 93.1 1.8X +before 1900, vec on, rebase CORRECTED 6091 6105 16 16.4 60.9 2.7X ================================================================================================ @@ -107,36 +119,36 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 20653 20653 0 4.8 206.5 1.0X -before 1582, noop 10707 10707 0 9.3 107.1 1.9X -after 1582 28288 28288 0 3.5 282.9 0.7X -before 1582 19196 19196 0 5.2 192.0 1.1X +after 1582, noop 19583 19583 0 5.1 195.8 1.0X +before 1582, noop 10711 10711 0 9.3 107.1 1.8X +after 1582 27864 27864 0 3.6 278.6 0.7X +before 1582 19648 19648 0 5.1 196.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 10596 10621 37 9.4 106.0 1.0X -after 1582, vec on 3886 3938 61 25.7 38.9 2.7X -before 1582, vec off 10955 10984 26 9.1 109.6 1.0X -before 1582, vec on 4236 4258 24 23.6 42.4 2.5X +after 1582, vec off 10383 10560 192 9.6 103.8 1.0X +after 1582, vec on 3844 3864 33 26.0 38.4 2.7X +before 1582, vec off 10867 10916 48 9.2 108.7 1.0X +before 1582, vec on 4158 4170 12 24.0 41.6 2.5X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2988 2988 0 33.5 29.9 1.0X -before 1900, noop 3007 3007 0 33.3 30.1 1.0X -after 1900 18082 18082 0 5.5 180.8 0.2X -before 1900 22669 22669 0 4.4 226.7 0.1X +after 1900, noop 2989 2989 0 33.5 29.9 1.0X +before 1900, noop 3000 3000 0 33.3 30.0 1.0X +after 1900 19426 19426 0 5.1 194.3 0.2X +before 1900 23282 23282 0 4.3 232.8 0.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 12029 12035 9 8.3 120.3 1.0X -after 1900, vec on 5194 5197 3 19.3 51.9 2.3X -before 1900, vec off 14853 14875 23 6.7 148.5 0.8X -before 1900, vec on 7797 7836 60 12.8 78.0 1.5X +after 1900, vec off 12089 12102 15 8.3 120.9 1.0X +after 1900, vec on 5210 5325 100 19.2 52.1 2.3X +before 1900, vec off 15320 15373 46 6.5 153.2 0.8X +before 1900, vec on 7937 7970 48 12.6 79.4 1.5X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt index f2af3cc000f04..3e94d6c6fcfa7 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt @@ -6,97 +6,109 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 23567 23567 0 4.2 235.7 1.0X -before 1582, noop 10570 10570 0 9.5 105.7 2.2X -after 1582, rebase off 35335 35335 0 2.8 353.3 0.7X -after 1582, rebase on 35645 35645 0 2.8 356.5 0.7X -before 1582, rebase off 21824 21824 0 4.6 218.2 1.1X -before 1582, rebase on 22532 22532 0 4.4 225.3 1.0X +after 1582, noop 23300 23300 0 4.3 233.0 1.0X +before 1582, noop 10585 10585 0 9.4 105.9 2.2X +after 1582, rebase EXCEPTION 35215 35215 0 2.8 352.1 0.7X +after 1582, rebase LEGACY 34927 34927 0 2.9 349.3 0.7X +after 1582, rebase CORRECTED 35479 35479 0 2.8 354.8 0.7X +before 1582, rebase LEGACY 22767 22767 0 4.4 227.7 1.0X +before 1582, rebase CORRECTED 22527 22527 0 4.4 225.3 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase off 13194 13266 81 7.6 131.9 1.0X -after 1582, vec off, rebase on 13402 13466 89 7.5 134.0 1.0X -after 1582, vec on, rebase off 3627 3657 29 27.6 36.3 3.6X -after 1582, vec on, rebase on 3818 3839 26 26.2 38.2 3.5X -before 1582, vec off, rebase off 13075 13146 115 7.6 130.7 1.0X -before 1582, vec off, rebase on 13794 13804 13 7.2 137.9 1.0X -before 1582, vec on, rebase off 3655 3675 21 27.4 36.6 3.6X -before 1582, vec on, rebase on 4579 4634 72 21.8 45.8 2.9X +after 1582, vec off, rebase EXCEPTION 13480 13577 94 7.4 134.8 1.0X +after 1582, vec off, rebase LEGACY 13466 13586 118 7.4 134.7 1.0X +after 1582, vec off, rebase CORRECTED 13526 13558 41 7.4 135.3 1.0X +after 1582, vec on, rebase EXCEPTION 3759 3778 28 26.6 37.6 3.6X +after 1582, vec on, rebase LEGACY 3957 4004 57 25.3 39.6 3.4X +after 1582, vec on, rebase CORRECTED 3739 3755 25 26.7 37.4 3.6X +before 1582, vec off, rebase LEGACY 13986 14038 67 7.1 139.9 1.0X +before 1582, vec off, rebase CORRECTED 13453 13491 49 7.4 134.5 1.0X +before 1582, vec on, rebase LEGACY 4716 4724 10 21.2 47.2 2.9X +before 1582, vec on, rebase CORRECTED 3701 3750 50 27.0 37.0 3.6X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2671 2671 0 37.4 26.7 1.0X -before 1900, noop 2685 2685 0 37.2 26.8 1.0X -after 1900, rebase off 23899 23899 0 4.2 239.0 0.1X -after 1900, rebase on 24030 24030 0 4.2 240.3 0.1X -before 1900, rebase off 30178 30178 0 3.3 301.8 0.1X -before 1900, rebase on 30127 30127 0 3.3 301.3 0.1X +after 1900, noop 2790 2790 0 35.8 27.9 1.0X +before 1900, noop 2812 2812 0 35.6 28.1 1.0X +after 1900, rebase EXCEPTION 24789 24789 0 4.0 247.9 0.1X +after 1900, rebase LEGACY 24539 24539 0 4.1 245.4 0.1X +after 1900, rebase CORRECTED 24543 24543 0 4.1 245.4 0.1X +before 1900, rebase LEGACY 30496 30496 0 3.3 305.0 0.1X +before 1900, rebase CORRECTED 30428 30428 0 3.3 304.3 0.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 16613 16685 75 6.0 166.1 1.0X -after 1900, vec off, rebase on 16487 16541 47 6.1 164.9 1.0X -after 1900, vec on, rebase off 8840 8870 49 11.3 88.4 1.9X -after 1900, vec on, rebase on 8795 8813 20 11.4 87.9 1.9X -before 1900, vec off, rebase off 20400 20441 62 4.9 204.0 0.8X -before 1900, vec off, rebase on 20430 20481 60 4.9 204.3 0.8X -before 1900, vec on, rebase off 12211 12290 73 8.2 122.1 1.4X -before 1900, vec on, rebase on 12231 12321 95 8.2 122.3 1.4X +after 1900, vec off, rebase EXCEPTION 17106 17192 75 5.8 171.1 1.0X +after 1900, vec off, rebase LEGACY 17273 17337 55 5.8 172.7 1.0X +after 1900, vec off, rebase CORRECTED 17073 17215 128 5.9 170.7 1.0X +after 1900, vec on, rebase EXCEPTION 8903 8976 117 11.2 89.0 1.9X +after 1900, vec on, rebase LEGACY 8793 8876 84 11.4 87.9 1.9X +after 1900, vec on, rebase CORRECTED 8820 8878 53 11.3 88.2 1.9X +before 1900, vec off, rebase LEGACY 20997 21069 82 4.8 210.0 0.8X +before 1900, vec off, rebase CORRECTED 20874 20946 90 4.8 208.7 0.8X +before 1900, vec on, rebase LEGACY 12024 12090 58 8.3 120.2 1.4X +before 1900, vec on, rebase CORRECTED 12020 12069 64 8.3 120.2 1.4X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2836 2836 0 35.3 28.4 1.0X -before 1900, noop 2812 2812 0 35.6 28.1 1.0X -after 1900, rebase off 15976 15976 0 6.3 159.8 0.2X -after 1900, rebase on 16197 16197 0 6.2 162.0 0.2X -before 1900, rebase off 16140 16140 0 6.2 161.4 0.2X -before 1900, rebase on 20410 20410 0 4.9 204.1 0.1X +after 1900, noop 2939 2939 0 34.0 29.4 1.0X +before 1900, noop 2917 2917 0 34.3 29.2 1.0X +after 1900, rebase EXCEPTION 15954 15954 0 6.3 159.5 0.2X +after 1900, rebase LEGACY 16402 16402 0 6.1 164.0 0.2X +after 1900, rebase CORRECTED 16541 16541 0 6.0 165.4 0.2X +before 1900, rebase LEGACY 20500 20500 0 4.9 205.0 0.1X +before 1900, rebase CORRECTED 16764 16764 0 6.0 167.6 0.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15297 15324 40 6.5 153.0 1.0X -after 1900, vec off, rebase on 15771 15832 59 6.3 157.7 1.0X -after 1900, vec on, rebase off 4922 4949 32 20.3 49.2 3.1X -after 1900, vec on, rebase on 5392 5411 17 18.5 53.9 2.8X -before 1900, vec off, rebase off 15227 15385 141 6.6 152.3 1.0X -before 1900, vec off, rebase on 19611 19658 41 5.1 196.1 0.8X -before 1900, vec on, rebase off 4965 5013 54 20.1 49.6 3.1X -before 1900, vec on, rebase on 9847 9873 43 10.2 98.5 1.6X +after 1900, vec off, rebase EXCEPTION 15607 15655 81 6.4 156.1 1.0X +after 1900, vec off, rebase LEGACY 15616 15676 54 6.4 156.2 1.0X +after 1900, vec off, rebase CORRECTED 15634 15732 108 6.4 156.3 1.0X +after 1900, vec on, rebase EXCEPTION 5041 5057 16 19.8 50.4 3.1X +after 1900, vec on, rebase LEGACY 5516 5539 29 18.1 55.2 2.8X +after 1900, vec on, rebase CORRECTED 5087 5104 28 19.7 50.9 3.1X +before 1900, vec off, rebase LEGACY 19262 19338 79 5.2 192.6 0.8X +before 1900, vec off, rebase CORRECTED 15718 15755 53 6.4 157.2 1.0X +before 1900, vec on, rebase LEGACY 10147 10240 114 9.9 101.5 1.5X +before 1900, vec on, rebase CORRECTED 5062 5080 21 19.8 50.6 3.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2818 2818 0 35.5 28.2 1.0X -before 1900, noop 2805 2805 0 35.6 28.1 1.0X -after 1900, rebase off 15182 15182 0 6.6 151.8 0.2X -after 1900, rebase on 15614 15614 0 6.4 156.1 0.2X -before 1900, rebase off 15404 15404 0 6.5 154.0 0.2X -before 1900, rebase on 19747 19747 0 5.1 197.5 0.1X +after 1900, noop 2915 2915 0 34.3 29.2 1.0X +before 1900, noop 2894 2894 0 34.6 28.9 1.0X +after 1900, rebase EXCEPTION 15545 15545 0 6.4 155.4 0.2X +after 1900, rebase LEGACY 15840 15840 0 6.3 158.4 0.2X +after 1900, rebase CORRECTED 16324 16324 0 6.1 163.2 0.2X +before 1900, rebase LEGACY 20359 20359 0 4.9 203.6 0.1X +before 1900, rebase CORRECTED 16292 16292 0 6.1 162.9 0.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase off 15622 15649 24 6.4 156.2 1.0X -after 1900, vec off, rebase on 15572 15677 119 6.4 155.7 1.0X -after 1900, vec on, rebase off 6345 6358 15 15.8 63.5 2.5X -after 1900, vec on, rebase on 6780 6834 92 14.8 67.8 2.3X -before 1900, vec off, rebase off 15540 15584 38 6.4 155.4 1.0X -before 1900, vec off, rebase on 19590 19653 55 5.1 195.9 0.8X -before 1900, vec on, rebase off 6374 6381 10 15.7 63.7 2.5X -before 1900, vec on, rebase on 10530 10544 25 9.5 105.3 1.5X +after 1900, vec off, rebase EXCEPTION 15857 16015 223 6.3 158.6 1.0X +after 1900, vec off, rebase LEGACY 16174 16231 63 6.2 161.7 1.0X +after 1900, vec off, rebase CORRECTED 16353 16400 67 6.1 163.5 1.0X +after 1900, vec on, rebase EXCEPTION 6449 6459 9 15.5 64.5 2.5X +after 1900, vec on, rebase LEGACY 7028 7035 6 14.2 70.3 2.3X +after 1900, vec on, rebase CORRECTED 6585 6623 37 15.2 65.8 2.4X +before 1900, vec off, rebase LEGACY 19929 20027 95 5.0 199.3 0.8X +before 1900, vec off, rebase CORRECTED 16401 16451 49 6.1 164.0 1.0X +before 1900, vec on, rebase LEGACY 10517 10563 40 9.5 105.2 1.5X +before 1900, vec on, rebase CORRECTED 6659 6675 26 15.0 66.6 2.4X ================================================================================================ @@ -107,36 +119,36 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 23825 23825 0 4.2 238.2 1.0X -before 1582, noop 10501 10501 0 9.5 105.0 2.3X -after 1582 32134 32134 0 3.1 321.3 0.7X -before 1582 19947 19947 0 5.0 199.5 1.2X +after 1582, noop 22782 22782 0 4.4 227.8 1.0X +before 1582, noop 10555 10555 0 9.5 105.6 2.2X +after 1582 31497 31497 0 3.2 315.0 0.7X +before 1582 19803 19803 0 5.0 198.0 1.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 10034 10056 22 10.0 100.3 1.0X -after 1582, vec on 3664 3698 30 27.3 36.6 2.7X -before 1582, vec off 10472 10502 30 9.5 104.7 1.0X -before 1582, vec on 4052 4098 42 24.7 40.5 2.5X +after 1582, vec off 10180 10214 44 9.8 101.8 1.0X +after 1582, vec on 3785 3804 24 26.4 37.8 2.7X +before 1582, vec off 10537 10582 39 9.5 105.4 1.0X +before 1582, vec on 4117 4146 25 24.3 41.2 2.5X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2812 2812 0 35.6 28.1 1.0X -before 1900, noop 2801 2801 0 35.7 28.0 1.0X -after 1900 18290 18290 0 5.5 182.9 0.2X -before 1900 22344 22344 0 4.5 223.4 0.1X +after 1900, noop 2853 2853 0 35.1 28.5 1.0X +before 1900, noop 2999 2999 0 33.3 30.0 1.0X +after 1900 16757 16757 0 6.0 167.6 0.2X +before 1900 21542 21542 0 4.6 215.4 0.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 11257 11279 32 8.9 112.6 1.0X -after 1900, vec on 5296 5310 15 18.9 53.0 2.1X -before 1900, vec off 14700 14758 72 6.8 147.0 0.8X -before 1900, vec on 8576 8665 150 11.7 85.8 1.3X +after 1900, vec off 12212 12254 39 8.2 122.1 1.0X +after 1900, vec on 5369 5390 35 18.6 53.7 2.3X +before 1900, vec off 15661 15705 73 6.4 156.6 0.8X +before 1900, vec on 8720 8744 29 11.5 87.2 1.4X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala index d6167f98b5a51..7caaa5376db7f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeRebaseBenchmark.scala @@ -25,8 +25,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.util.DateTimeConstants.SECONDS_PER_DAY import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.{withDefaultTimeZone, LA} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy._ -import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType +import org.apache.spark.sql.internal.SQLConf.{LegacyBehaviorPolicy, ParquetOutputTimestampType} object DateTime extends Enumeration { type DateTime = Value @@ -118,12 +117,12 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { private def caseName( modernDates: Boolean, dateTime: DateTime, - rebase: Option[Boolean] = None, + mode: Option[LegacyBehaviorPolicy.Value] = None, vec: Option[Boolean] = None): String = { val period = if (modernDates) "after" else "before" val year = if (dateTime == DATE) 1582 else 1900 val vecFlag = vec.map(flagToStr).map(flag => s", vec $flag").getOrElse("") - val rebaseFlag = rebase.map(flagToStr).map(flag => s", rebase $flag").getOrElse("") + val rebaseFlag = mode.map(_.toString).map(m => s", rebase $m").getOrElse("") s"$period $year$vecFlag$rebaseFlag" } @@ -131,10 +130,10 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { basePath: File, dateTime: DateTime, modernDates: Boolean, - rebase: Option[Boolean] = None): String = { + mode: Option[LegacyBehaviorPolicy.Value] = None): String = { val period = if (modernDates) "after" else "before" val year = if (dateTime == DATE) 1582 else 1900 - val rebaseFlag = rebase.map(flagToStr).map(flag => s"_$flag").getOrElse("") + val rebaseFlag = mode.map(_.toString).map(m => s"_$m").getOrElse("") basePath.getAbsolutePath + s"/${dateTime}_${period}_$year$rebaseFlag" } @@ -160,9 +159,10 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { output = output) benchmarkInputs(benchmark, rowsNum, dateTime) Seq(true, false).foreach { modernDates => - Seq(false, true).foreach { rebase => - benchmark.addCase(caseName(modernDates, dateTime, Some(rebase)), 1) { _ => - val mode = if (rebase) LEGACY else CORRECTED + LegacyBehaviorPolicy.values + .filterNot(v => !modernDates && v == LegacyBehaviorPolicy.EXCEPTION) + .foreach { mode => + benchmark.addCase(caseName(modernDates, dateTime, Some(mode)), 1) { _ => withSQLConf( SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> getOutputType(dateTime), SQLConf.LEGACY_PARQUET_REBASE_MODE_IN_WRITE.key -> mode.toString) { @@ -170,7 +170,7 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { .write .mode("overwrite") .format("parquet") - .save(getPath(path, dateTime, modernDates, Some(rebase))) + .save(getPath(path, dateTime, modernDates, Some(mode))) } } } @@ -181,13 +181,15 @@ object DateTimeRebaseBenchmark extends SqlBasedBenchmark { s"Load $dateTime from parquet", rowsNum, output = output) Seq(true, false).foreach { modernDates => Seq(false, true).foreach { vec => - Seq(false, true).foreach { rebase => - val name = caseName(modernDates, dateTime, Some(rebase), Some(vec)) + LegacyBehaviorPolicy.values + .filterNot(v => !modernDates && v == LegacyBehaviorPolicy.EXCEPTION) + .foreach { mode => + val name = caseName(modernDates, dateTime, Some(mode), Some(vec)) benchmark2.addCase(name, 3) { _ => withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vec.toString) { spark.read .format("parquet") - .load(getPath(path, dateTime, modernDates, Some(rebase))) + .load(getPath(path, dateTime, modernDates, Some(mode))) .noop() } } From f83cb3cbb3ce3f22fd122bce620917bfd0699ce7 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 15 Jun 2020 08:49:03 -0500 Subject: [PATCH 06/28] [SPARK-31925][ML] Summary.totalIterations greater than maxIters ### What changes were proposed in this pull request? In LogisticRegression and LinearRegression, if set maxIter=n, the model.summary.totalIterations returns n+1 if the training procedure does not drop out. This is because we use ```objectiveHistory.length``` as totalIterations, but ```objectiveHistory``` contains init sate, thus ```objectiveHistory.length``` is 1 larger than number of training iterations. ### Why are the changes needed? correctness ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? add new tests and also modify existing tests Closes #28786 from huaxingao/summary_iter. Authored-by: Huaxin Gao Signed-off-by: Sean Owen --- .../ml/classification/LogisticRegression.scala | 12 +++++++++--- .../spark/ml/regression/LinearRegression.scala | 8 +++++--- .../JavaLogisticRegressionSuite.java | 2 +- .../classification/LogisticRegressionSuite.scala | 13 ++++++++++--- .../ml/regression/LinearRegressionSuite.scala | 15 +++++++++++++++ python/pyspark/ml/classification.py | 3 ++- python/pyspark/ml/tests/test_training_summary.py | 2 +- 7 files changed, 43 insertions(+), 12 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 0d1350640c74a..1f5976c59235b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -594,7 +594,7 @@ class LogisticRegression @Since("1.2.0") ( Vectors.dense(if (numClasses == 2) Double.PositiveInfinity else Double.NegativeInfinity) } if (instances.getStorageLevel != StorageLevel.NONE) instances.unpersist() - return createModel(dataset, numClasses, coefMatrix, interceptVec, Array.empty) + return createModel(dataset, numClasses, coefMatrix, interceptVec, Array(0.0)) } if (!$(fitIntercept) && isConstantLabel) { @@ -1545,13 +1545,19 @@ sealed trait LogisticRegressionSummary extends Serializable { */ sealed trait LogisticRegressionTrainingSummary extends LogisticRegressionSummary { - /** objective function (scaled loss + regularization) at each iteration. */ + /** + * objective function (scaled loss + regularization) at each iteration. + * It contains one more element, the initial state, than number of iterations. + */ @Since("1.5.0") def objectiveHistory: Array[Double] /** Number of training iterations. */ @Since("1.5.0") - def totalIterations: Int = objectiveHistory.length + def totalIterations: Int = { + assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") + objectiveHistory.length - 1 + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 8b6ede3bb362c..d9f09c097292a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -433,7 +433,7 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String Vectors.dense(Array.fill(dim)(1.0)) } - val (parameters, objectiveHistory) = if ($(blockSize) == 1) { + val (parameters, objectiveHistory) = if ($(blockSize) == 1) { trainOnRows(instances, yMean, yStd, featuresMean, featuresStd, initialValues, regularization, optimizer) } else { @@ -939,8 +939,10 @@ class LinearRegressionTrainingSummary private[regression] ( * @see `LinearRegression.solver` */ @Since("1.5.0") - val totalIterations = objectiveHistory.length - + val totalIterations = { + assert(objectiveHistory.length > 0, s"objectiveHistory length should be greater than 1.") + objectiveHistory.length - 1 + } } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java index 49ac49339415a..7c63a8755b4f3 100644 --- a/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/classification/JavaLogisticRegressionSuite.java @@ -143,6 +143,6 @@ public void logisticRegressionTrainingSummary() { LogisticRegressionModel model = lr.fit(dataset); LogisticRegressionTrainingSummary summary = model.summary(); - Assert.assertEquals(summary.totalIterations(), summary.objectiveHistory().length); + Assert.assertEquals(summary.totalIterations(), summary.objectiveHistory().length - 1); } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 25e9697d64855..30c21d8b06670 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -266,6 +266,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(blorModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) assert(blorModel.summary.asBinary.isInstanceOf[BinaryLogisticRegressionSummary]) assert(blorModel.binarySummary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) + assert(blorModel.summary.totalIterations == 1) + assert(blorModel.binarySummary.totalIterations == 1) val mlorModel = lr.setFamily("multinomial").fit(smallMultinomialDataset) assert(mlorModel.summary.isInstanceOf[LogisticRegressionTrainingSummary]) @@ -279,6 +281,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { mlorModel.summary.asBinary } } + assert(mlorModel.summary.totalIterations == 1) val mlorBinaryModel = lr.setFamily("multinomial").fit(smallBinaryDataset) assert(mlorBinaryModel.summary.isInstanceOf[BinaryLogisticRegressionTrainingSummary]) @@ -2570,7 +2573,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { rows.map(_.getDouble(0)).toArray === binaryExpected } } - assert(model2.summary.totalIterations === 1) + assert(model2.summary.totalIterations === 0) val lr3 = new LogisticRegression().setFamily("multinomial") val model3 = lr3.fit(smallMultinomialDataset) @@ -2585,7 +2588,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { rows.map(_.getDouble(0)).toArray === multinomialExpected } } - assert(model4.summary.totalIterations === 1) + assert(model4.summary.totalIterations === 0) } test("binary logistic regression with all labels the same") { @@ -2605,6 +2608,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(allZeroInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) assert(allZeroInterceptModel.intercept === Double.NegativeInfinity) assert(allZeroInterceptModel.summary.totalIterations === 0) + assert(allZeroInterceptModel.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) val allOneInterceptModel = lrIntercept .setLabelCol("oneLabel") @@ -2612,6 +2616,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(allOneInterceptModel.coefficients ~== Vectors.dense(0.0) absTol 1E-3) assert(allOneInterceptModel.intercept === Double.PositiveInfinity) assert(allOneInterceptModel.summary.totalIterations === 0) + assert(allOneInterceptModel.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) // fitIntercept=false val lrNoIntercept = new LogisticRegression() @@ -2647,6 +2652,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(pred === 4.0) } assert(model.summary.totalIterations === 0) + assert(model.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) // force the model to be trained with only one class val constantZeroData = Seq( @@ -2660,7 +2666,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(prob === Vectors.dense(Array(1.0))) assert(pred === 0.0) } - assert(modelZeroLabel.summary.totalIterations > 0) + assert(modelZeroLabel.summary.totalIterations === 0) // ensure that the correct value is predicted when numClasses passed through metadata val labelMeta = NominalAttribute.defaultAttr.withName("label").withNumValues(6).toMetadata() @@ -2675,6 +2681,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { assert(pred === 4.0) } require(modelWithMetadata.summary.totalIterations === 0) + assert(model.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) } test("compressed storage for constant label") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index c4a94ff2d6f44..fb70883bffc5f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -761,6 +761,7 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .fit(datasetWithWeightConstantLabel) if (fitIntercept) { assert(model1.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) + assert(model1.summary.totalIterations === 0) } val model2 = new LinearRegression() .setFitIntercept(fitIntercept) @@ -768,6 +769,7 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .setSolver("l-bfgs") .fit(datasetWithWeightZeroLabel) assert(model2.summary.objectiveHistory(0) ~== 0.0 absTol 1e-4) + assert(model2.summary.totalIterations === 0) } } } @@ -940,6 +942,19 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe } } + test("linear regression training summary totalIterations") { + Seq(1, 5, 10, 20).foreach { maxIter => + val trainer = new LinearRegression().setSolver("l-bfgs").setMaxIter(maxIter) + val model = trainer.fit(datasetWithDenseFeature) + assert(model.summary.totalIterations <= maxIter) + } + Seq("auto", "normal").foreach { solver => + val trainer = new LinearRegression().setSolver(solver) + val model = trainer.fit(datasetWithDenseFeature) + assert(model.summary.totalIterations === 0) + } + } + test("linear regression with weighted samples") { val sqlContext = spark.sqlContext import sqlContext.implicits._ diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 734c393db2a26..3f3699ce53b51 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -1119,7 +1119,8 @@ class LogisticRegressionTrainingSummary(LogisticRegressionSummary): def objectiveHistory(self): """ Objective function (scaled loss + regularization) at each - iteration. + iteration. It contains one more element, the initial state, + than number of iterations. """ return self._call_java("objectiveHistory") diff --git a/python/pyspark/ml/tests/test_training_summary.py b/python/pyspark/ml/tests/test_training_summary.py index b5054095d190b..ac944d8397a86 100644 --- a/python/pyspark/ml/tests/test_training_summary.py +++ b/python/pyspark/ml/tests/test_training_summary.py @@ -42,7 +42,7 @@ def test_linear_regression_summary(self): self.assertTrue(model.hasSummary) s = model.summary # test that api is callable and returns expected types - self.assertGreater(s.totalIterations, 0) + self.assertEqual(s.totalIterations, 0) self.assertTrue(isinstance(s.predictions, DataFrame)) self.assertEqual(s.predictionCol, "prediction") self.assertEqual(s.labelCol, "label") From 7f7b4dd5199e7c185aedf51fccc400c7072bed05 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 15 Jun 2020 07:48:48 -0700 Subject: [PATCH 07/28] [SPARK-31990][SS] Use toSet.toSeq in Dataset.dropDuplicates ### What changes were proposed in this pull request? This PR partially revert SPARK-31292 in order to provide a hot-fix for a bug in `Dataset.dropDuplicates`; we must preserve the input order of `colNames` for `groupCols` because the Streaming's state store depends on the `groupCols` order. ### Why are the changes needed? Bug fix. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests in `DataFrameSuite`. Closes #28830 from maropu/SPARK-31990. Authored-by: Takeshi Yamamuro Signed-off-by: Dongjoon Hyun --- sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index ae8d33d8558ba..524e231eb7eb9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -2541,7 +2541,9 @@ class Dataset[T] private[sql]( def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { val resolver = sparkSession.sessionState.analyzer.resolver val allColumns = queryExecution.analyzed.output - val groupCols = colNames.distinct.flatMap { (colName: String) => + // SPARK-31990: We must keep `toSet.toSeq` here because of the backward compatibility issue + // (the Streaming's state store depends on the `groupCols` order). + val groupCols = colNames.toSet.toSeq.flatMap { (colName: String) => // It is possibly there are more than one columns with the same name, // so we call filter instead of find. val cols = allColumns.filter(col => resolver(col.name, colName)) From eae1747b663adaa612681d799dcaa11915cd1692 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Mon, 15 Jun 2020 08:09:07 -0700 Subject: [PATCH 08/28] [SPARK-31959][SQL][TESTS][FOLLOWUP] Adopt the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" to outdated tzdb ### What changes were proposed in this pull request? Old JDK can have outdated time zone database in which `Asia/Hong_Kong` doesn't have timestamp overlapping in 1946 at all. This PR changes the test "SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945" in `RebaseDateTimeSuite`, and makes it tolerant to the case. ### Why are the changes needed? To fix the test failures on old JDK w/ outdated tzdb like on Jenkins machine `research-jenkins-worker-09`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the test on old JDK Closes #28832 from MaxGekk/HongKong-tz-1945-followup. Authored-by: Max Gekk Signed-off-by: Dongjoon Hyun --- .../spark/sql/catalyst/util/RebaseDateTimeSuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index e3af64a562e23..8a51f158a9429 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -421,7 +421,8 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - if (earlierMicros + MICROS_PER_HOUR != laterMicros) { + var overlapInterval = MICROS_PER_HOUR + if (earlierMicros + overlapInterval != laterMicros) { // Old JDK might have an outdated time zone database. // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" @@ -429,14 +430,15 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - assert(earlierMicros + MICROS_PER_HOUR === laterMicros) + // If time zone db doesn't have overlapping at all, set the overlap interval to zero. + overlapInterval = laterMicros - earlierMicros } val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkZid, earlierMicros) val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkZid, laterMicros) def toTsStr(micros: Long): String = toJavaTimestamp(micros).toString assert(toTsStr(rebasedEarlierMicros) === expected) assert(toTsStr(rebasedLaterMicros) === expected) - assert(rebasedEarlierMicros + MICROS_PER_HOUR === rebasedLaterMicros) + assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) // Check optimized rebasing assert(rebaseGregorianToJulianMicros(earlierMicros) === rebasedEarlierMicros) assert(rebaseGregorianToJulianMicros(laterMicros) === rebasedLaterMicros) From 3698a14204dd861ea3ee3c14aa923123b52caba1 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 16 Jun 2020 00:27:45 +0900 Subject: [PATCH 09/28] [SPARK-26905][SQL] Follow the SQL:2016 reserved keywords ### What changes were proposed in this pull request? This PR intends to move keywords `ANTI`, `SEMI`, and `MINUS` from reserved to non-reserved. ### Why are the changes needed? To comply with the ANSI/SQL standard. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added tests. Closes #28807 from maropu/SPARK-26905-2. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro --- docs/sql-ref-ansi-compliance.md | 6 +- .../spark/sql/catalyst/parser/SqlBase.g4 | 3 + .../ansi-sql-2016-reserved-keywords.txt | 401 ++++++++++++++++++ .../parser/TableIdentifierParserSuite.scala | 24 +- 4 files changed, 429 insertions(+), 5 deletions(-) create mode 100644 sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index eab194c71ec79..e5ca7e9d10d59 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -135,7 +135,7 @@ Below is a list of all the keywords in Spark SQL. |ALTER|non-reserved|non-reserved|reserved| |ANALYZE|non-reserved|non-reserved|non-reserved| |AND|reserved|non-reserved|reserved| -|ANTI|reserved|strict-non-reserved|non-reserved| +|ANTI|non-reserved|strict-non-reserved|non-reserved| |ANY|reserved|non-reserved|reserved| |ARCHIVE|non-reserved|non-reserved|non-reserved| |ARRAY|non-reserved|non-reserved|reserved| @@ -264,7 +264,7 @@ Below is a list of all the keywords in Spark SQL. |MAP|non-reserved|non-reserved|non-reserved| |MATCHED|non-reserved|non-reserved|non-reserved| |MERGE|non-reserved|non-reserved|non-reserved| -|MINUS|reserved|strict-non-reserved|non-reserved| +|MINUS|not-reserved|strict-non-reserved|non-reserved| |MINUTE|reserved|non-reserved|reserved| |MONTH|reserved|non-reserved|reserved| |MSCK|non-reserved|non-reserved|non-reserved| @@ -325,7 +325,7 @@ Below is a list of all the keywords in Spark SQL. |SCHEMA|non-reserved|non-reserved|non-reserved| |SECOND|reserved|non-reserved|reserved| |SELECT|reserved|non-reserved|reserved| -|SEMI|reserved|strict-non-reserved|non-reserved| +|SEMI|non-reserved|strict-non-reserved|non-reserved| |SEPARATED|non-reserved|non-reserved|non-reserved| |SERDE|non-reserved|non-reserved|non-reserved| |SERDEPROPERTIES|non-reserved|non-reserved|non-reserved| diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 61aeafdc27f11..691fde8d48f94 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -1014,6 +1014,7 @@ ansiNonReserved | AFTER | ALTER | ANALYZE + | ANTI | ARCHIVE | ARRAY | ASC @@ -1146,10 +1147,12 @@ ansiNonReserved | ROW | ROWS | SCHEMA + | SEMI | SEPARATED | SERDE | SERDEPROPERTIES | SET + | SETMINUS | SETS | SHOW | SKEWED diff --git a/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt b/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt new file mode 100644 index 0000000000000..921491a4a4761 --- /dev/null +++ b/sql/catalyst/src/test/resources/ansi-sql-2016-reserved-keywords.txt @@ -0,0 +1,401 @@ +-- This file comes from: https://github.com/postgres/postgres/tree/master/doc/src/sgml/keywords +ABS +ACOS +ALL +ALLOCATE +ALTER +AND +ANY +ARE +ARRAY +ARRAY_AGG +ARRAY_MAX_CARDINALITY +AS +ASENSITIVE +ASIN +ASYMMETRIC +AT +ATAN +ATOMIC +AUTHORIZATION +AVG +BEGIN +BEGIN_FRAME +BEGIN_PARTITION +BETWEEN +BIGINT +BINARY +BLOB +BOOLEAN +BOTH +BY +CALL +CALLED +CARDINALITY +CASCADED +CASE +CAST +CEIL +CEILING +CHAR +CHAR_LENGTH +CHARACTER +CHARACTER_LENGTH +CHECK +CLASSIFIER +CLOB +CLOSE +COALESCE +COLLATE +COLLECT +COLUMN +COMMIT +CONDITION +CONNECT +CONSTRAINT +CONTAINS +CONVERT +COPY +CORR +CORRESPONDING +COS +COSH +COUNT +COVAR_POP +COVAR_SAMP +CREATE +CROSS +CUBE +CUME_DIST +CURRENT +CURRENT_CATALOG +CURRENT_DATE +CURRENT_DEFAULT_TRANSFORM_GROUP +CURRENT_PATH +CURRENT_ROLE +CURRENT_ROW +CURRENT_SCHEMA +CURRENT_TIME +CURRENT_TIMESTAMP +CURRENT_TRANSFORM_GROUP_FOR_TYPE +CURRENT_USER +CURSOR +CYCLE +DATE +DAY +DEALLOCATE +DEC +DECIMAL +DECFLOAT +DECLARE +DEFAULT +DEFINE +DELETE +DENSE_RANK +DEREF +DESCRIBE +DETERMINISTIC +DISCONNECT +DISTINCT +DOUBLE +DROP +DYNAMIC +EACH +ELEMENT +ELSE +EMPTY +END +END_FRAME +END_PARTITION +END-EXEC +EQUALS +ESCAPE +EVERY +EXCEPT +EXEC +EXECUTE +EXISTS +EXP +EXTERNAL +EXTRACT +FALSE +FETCH +FILTER +FIRST_VALUE +FLOAT +FLOOR +FOR +FOREIGN +FRAME_ROW +FREE +FROM +FULL +FUNCTION +FUSION +GET +GLOBAL +GRANT +GROUP +GROUPING +GROUPS +HAVING +HOLD +HOUR +IDENTITY +IN +INDICATOR +INITIAL +INNER +INOUT +INSENSITIVE +INSERT +INT +INTEGER +INTERSECT +INTERSECTION +INTERVAL +INTO +IS +JOIN +JSON_ARRAY +JSON_ARRAYAGG +JSON_EXISTS +JSON_OBJECT +JSON_OBJECTAGG +JSON_QUERY +JSON_TABLE +JSON_TABLE_PRIMITIVE +JSON_VALUE +LAG +LANGUAGE +LARGE +LAST_VALUE +LATERAL +LEAD +LEADING +LEFT +LIKE +LIKE_REGEX +LISTAGG +LN +LOCAL +LOCALTIME +LOCALTIMESTAMP +LOG +LOG10 +LOWER +MATCH +MATCH_NUMBER +MATCH_RECOGNIZE +MATCHES +MAX +MEASURES +MEMBER +MERGE +METHOD +MIN +MINUTE +MOD +MODIFIES +MODULE +MONTH +MULTISET +NATIONAL +NATURAL +NCHAR +NCLOB +NEW +NO +NONE +NORMALIZE +NOT +NTH_VALUE +NTILE +NULL +NULLIF +NUMERIC +OCTET_LENGTH +OCCURRENCES_REGEX +OF +OFFSET +OLD +OMIT +ON +ONE +ONLY +OPEN +OR +ORDER +OUT +OUTER +OVER +OVERLAPS +OVERLAY +PARAMETER +PARTITION +PATTERN +PER +PERCENT +PERCENT_RANK +PERCENTILE_CONT +PERCENTILE_DISC +PERIOD +PERMUTE +PORTION +POSITION +POSITION_REGEX +POWER +PRECEDES +PRECISION +PREPARE +PRIMARY +PROCEDURE +PTF +RANGE +RANK +READS +REAL +RECURSIVE +REF +REFERENCES +REFERENCING +REGR_AVGX +REGR_AVGY +REGR_COUNT +REGR_INTERCEPT +REGR_R2 +REGR_SLOPE +REGR_SXX +REGR_SXY +REGR_SYY +RELEASE +RESULT +RETURN +RETURNS +REVOKE +RIGHT +ROLLBACK +ROLLUP +ROW +ROW_NUMBER +ROWS +RUNNING +SAVEPOINT +SCOPE +SCROLL +SEARCH +SECOND +SEEK +SELECT +SENSITIVE +SESSION_USER +SET +SHOW +SIMILAR +SIN +SINH +SKIP +SMALLINT +SOME +SPECIFIC +SPECIFICTYPE +SQL +SQLEXCEPTION +SQLSTATE +SQLWARNING +SQRT +START +STATIC +STDDEV_POP +STDDEV_SAMP +SUBMULTISET +SUBSET +SUBSTRING +SUBSTRING_REGEX +SUCCEEDS +SUM +SYMMETRIC +SYSTEM +SYSTEM_TIME +SYSTEM_USER +TABLE +TABLESAMPLE +TAN +TANH +THEN +TIME +TIMESTAMP +TIMEZONE_HOUR +TIMEZONE_MINUTE +TO +TRAILING +TRANSLATE +TRANSLATE_REGEX +TRANSLATION +TREAT +TRIGGER +TRIM +TRIM_ARRAY +TRUE +TRUNCATE +UESCAPE +UNION +UNIQUE +UNKNOWN +UNMATCHED +UNNEST +UPDATE +UPPER +USER +USING +VALUE +VALUES +VALUE_OF +VAR_POP +VAR_SAMP +VARBINARY +VARCHAR +VARYING +VERSIONING +WHEN +WHENEVER +WHERE +WIDTH_BUCKET +WINDOW +WITH +WITHIN +WITHOUT +YEAR +DATALINK +DLNEWCOPY +DLPREVIOUSCOPY +DLURLCOMPLETE +DLURLCOMPLETEWRITE +DLURLCOMPLETEONLY +DLURLPATH +DLURLPATHWRITE +DLURLPATHONLY +DLURLSCHEME +DLURLSERVER +DLVALUE +IMPORT +XML +XMLAGG +XMLATTRIBUTES +XMLBINARY +XMLCAST +XMLCOMMENT +XMLCONCAT +XMLDOCUMENT +XMLELEMENT +XMLEXISTS +XMLFOREST +XMLITERATE +XMLNAMESPACES +XMLPARSE +XMLPI +XMLQUERY +XMLSERIALIZE +XMLTABLE +XMLTEXT +XMLVALIDATE diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala index 04969e34fb841..04c427da3ec87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala @@ -16,8 +16,11 @@ */ package org.apache.spark.sql.catalyst.parser +import java.io.File +import java.nio.file.Files import java.util.Locale +import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.SparkFunSuite @@ -340,7 +343,12 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { // The case where a symbol has multiple literal definitions, // e.g., `DATABASES: 'DATABASES' | 'SCHEMAS';`. if (hasMultipleLiterals) { - val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq + // Filters out inappropriate entries, e.g., `!` in `NOT: 'NOT' | '!';` + val litDef = """([A-Z_]+)""".r + val literals = splitDefs.map(_.replaceAll("'", "").trim).toSeq.flatMap { + case litDef(lit) => Some(lit) + case _ => None + } (symbol, literals) :: Nil } else { val literal = literalDef.replaceAll("'", "").trim @@ -388,12 +396,24 @@ class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { val reservedKeywordsInAnsiMode = allCandidateKeywords -- nonReservedKeywordsInAnsiMode test("check # of reserved keywords") { - val numReservedKeywords = 78 + val numReservedKeywords = 74 assert(reservedKeywordsInAnsiMode.size == numReservedKeywords, s"The expected number of reserved keywords is $numReservedKeywords, but " + s"${reservedKeywordsInAnsiMode.size} found.") } + test("reserved keywords in Spark are also reserved in SQL 2016") { + withTempDir { dir => + val tmpFile = new File(dir, "tmp") + val is = Thread.currentThread().getContextClassLoader + .getResourceAsStream("ansi-sql-2016-reserved-keywords.txt") + Files.copy(is, tmpFile.toPath) + val reservedKeywordsInSql2016 = Files.readAllLines(tmpFile.toPath) + .asScala.filterNot(_.startsWith("--")).map(_.trim).toSet + assert((reservedKeywordsInAnsiMode -- reservedKeywordsInSql2016).isEmpty) + } + } + test("table identifier") { // Regular names. assert(TableIdentifier("q") === parseTableIdentifier("q")) From a7d0d353cd745391cf285039a4630b37754a56a0 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 15 Jun 2020 11:26:03 -0700 Subject: [PATCH 10/28] [SPARK-31994][K8S] Docker image should use `https` urls for only deb.debian.org mirrors ### What changes were proposed in this pull request? At the moment, we switch to `https` urls for all the debian mirrors, but turns out some of the mirrors do not support. In this patch, we turn on https mode only for `deb.debian.org` mirror (as it supports SSL). ### Why are the changes needed? It appears, that security.debian.org does not support https. ``` curl https://security.debian.org curl: (35) LibreSSL SSL_connect: SSL_ERROR_SYSCALL in connection to security.debian.org:443 ``` While building the image, it fails in the following way. ``` MacBook-Pro:spark prashantsharma$ bin/docker-image-tool.sh -r scrapcodes -t v3.1.0-1 build Sending build context to Docker daemon 222.1MB Step 1/18 : ARG java_image_tag=8-jre-slim Step 2/18 : FROM openjdk:${java_image_tag} ---> 381b20190cf7 Step 3/18 : ARG spark_uid=185 ---> Using cache ---> 65c06f86753c Step 4/18 : RUN set -ex && sed -i 's/http:/https:/g' /etc/apt/sources.list && apt-get update && ln -s /lib /lib64 && apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && mkdir -p /opt/spark && mkdir -p /opt/spark/examples && mkdir -p /opt/spark/work-dir && touch /opt/spark/RELEASE && rm /bin/sh && ln -sv /bin/bash /bin/sh && echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && chgrp root /etc/passwd && chmod ug+rw /etc/passwd && rm -rf /var/cache/apt/* ---> Running in a3461dadd6eb + sed -i s/http:/https:/g /etc/apt/sources.list + apt-get update Ign:1 https://security.debian.org/debian-security buster/updates InRelease Err:2 https://security.debian.org/debian-security buster/updates Release Could not handshake: The TLS connection was non-properly terminated. [IP: 151.101.0.204 443] Get:3 https://deb.debian.org/debian buster InRelease [121 kB] Get:4 https://deb.debian.org/debian buster-updates InRelease [51.9 kB] Get:5 https://deb.debian.org/debian buster/main amd64 Packages [7905 kB] Get:6 https://deb.debian.org/debian buster-updates/main amd64 Packages [7868 B] Reading package lists... E: The repository 'https://security.debian.org/debian-security buster/updates Release' does not have a Release file. The command '/bin/sh -c set -ex && sed -i 's/http:/https:/g' /etc/apt/sources.list && apt-get update && ln -s /lib /lib64 && apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && mkdir -p /opt/spark && mkdir -p /opt/spark/examples && mkdir -p /opt/spark/work-dir && touch /opt/spark/RELEASE && rm /bin/sh && ln -sv /bin/bash /bin/sh && echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && chgrp root /etc/passwd && chmod ug+rw /etc/passwd && rm -rf /var/cache/apt/*' returned a non-zero code: 100 Failed to build Spark JVM Docker image, please refer to Docker build output for details. ``` So, if we limit the `https` support to only deb.debian.org, does the trick. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually, by building an image and testing it by running spark shell against it locally using kubernetes. Closes #28834 from ScrapCodes/spark-31994/debian_mirror_fix. Authored-by: Prashant Sharma Signed-off-by: Dongjoon Hyun --- .../kubernetes/docker/src/main/dockerfiles/spark/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index b722123195dc2..a13fe67c9eb72 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -28,7 +28,7 @@ ARG spark_uid=185 # docker build -t spark:latest -f kubernetes/dockerfiles/spark/Dockerfile . RUN set -ex && \ - sed -i 's/http:/https:/g' /etc/apt/sources.list && \ + sed -i 's/http:\/\/deb.\(.*\)/https:\/\/deb.\1/g' /etc/apt/sources.list && \ apt-get update && \ ln -s /lib /lib64 && \ apt install -y bash tini libc6 libpam-modules krb5-user libnss3 procps && \ From 5e89fbe44e945ceeb674c5b9a5b536da92866a56 Mon Sep 17 00:00:00 2001 From: gengjiaan Date: Mon, 15 Jun 2020 14:15:54 -0700 Subject: [PATCH 11/28] [SPARK-31824][CORE][TESTS] DAGSchedulerSuite: Improve and reuse completeShuffleMapStageSuccessfully ### What changes were proposed in this pull request? `DAGSchedulerSuite `provides `completeShuffleMapStageSuccessfully `to make `ShuffleMapStage `successfully. But many test case uses complete directly as follows: `complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1))))` We need to improve `completeShuffleMapStageSuccessfully `and reuse it. `completeShuffleMapStageSuccessfully(0, 0, 1, Some(0), Seq("hostA"))` ### Why are the changes needed? Improve and reuse completeShuffleMapStageSuccessfully ### Does this PR introduce _any_ user-facing change? 'No'. ### How was this patch tested? Jenkins test Closes #28641 from beliefer/improve-and-reuse-method. Authored-by: gengjiaan Signed-off-by: Xingbo Jiang --- .../spark/scheduler/DAGSchedulerSuite.scala | 197 +++++++----------- 1 file changed, 77 insertions(+), 120 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 4c6033edff247..9d412f2dba3ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -458,9 +458,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(mapStageC.parents === List(mapStageA, mapStageB)) assert(finalStage.parents === List(mapStageC)) - complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) - complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) + completeShuffleMapStageSuccessfully(1, 0, 1) + completeShuffleMapStageSuccessfully(2, 0, 1) complete(taskSets(3), Seq((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty() @@ -473,9 +473,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") conf.set("spark.files.fetchFailure.unRegisterOutputOnHost", "true") init(conf) - runEvent(ExecutorAdded("exec-hostA1", "hostA")) - runEvent(ExecutorAdded("exec-hostA2", "hostA")) - runEvent(ExecutorAdded("exec-hostB", "hostB")) + runEvent(ExecutorAdded("hostA-exec1", "hostA")) + runEvent(ExecutorAdded("hostA-exec2", "hostA")) + runEvent(ExecutorAdded("hostB-exec", "hostB")) val firstRDD = new MyRDD(sc, 3, Nil) val firstShuffleDep = new ShuffleDependency(firstRDD, new HashPartitioner(3)) val firstShuffleId = firstShuffleDep.shuffleId @@ -488,20 +488,20 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi complete(taskSets(0), Seq( (Success, MapStatus( - BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), + BlockManagerId("hostA-exec1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 5)), (Success, MapStatus( - BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), + BlockManagerId("hostA-exec2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 6)), (Success, makeMapStatus("hostB", 1, mapTaskId = 7)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, MapStatus( - BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), + BlockManagerId("hostA-exec1", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 8)), (Success, MapStatus( - BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), + BlockManagerId("hostA-exec2", "hostA", 12345), Array.fill[Long](1)(2), mapTaskId = 9)), (Success, makeMapStatus("hostB", 1, mapTaskId = 10)) )) // make sure our test setup is correct @@ -509,19 +509,19 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus1.count(_ != null) === 3) assert(initialMapStatus1.map{_.location.executorId}.toSet === - Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + Set("hostA-exec1", "hostA-exec2", "hostB-exec")) assert(initialMapStatus1.map{_.mapId}.toSet === Set(5, 6, 7)) val initialMapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses // val initialMapStatus1 = mapOutputTracker.mapStatuses.get(0).get assert(initialMapStatus2.count(_ != null) === 3) assert(initialMapStatus2.map{_.location.executorId}.toSet === - Set("exec-hostA1", "exec-hostA2", "exec-hostB")) + Set("hostA-exec1", "hostA-exec2", "hostB-exec")) assert(initialMapStatus2.map{_.mapId}.toSet === Set(8, 9, 10)) // reduce stage fails with a fetch failure from one host complete(taskSets(2), Seq( - (FetchFailed(BlockManagerId("exec-hostA2", "hostA", 12345), + (FetchFailed(BlockManagerId("hostA-exec2", "hostA", 12345), firstShuffleId, 0L, 0, 0, "ignored"), null) )) @@ -531,12 +531,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStatus1 = mapOutputTracker.shuffleStatuses(firstShuffleId).mapStatuses assert(mapStatus1.count(_ != null) === 1) - assert(mapStatus1(2).location.executorId === "exec-hostB") + assert(mapStatus1(2).location.executorId === "hostB-exec") assert(mapStatus1(2).location.host === "hostB") val mapStatus2 = mapOutputTracker.shuffleStatuses(secondShuffleId).mapStatuses assert(mapStatus2.count(_ != null) === 1) - assert(mapStatus2(2).location.executorId === "exec-hostB") + assert(mapStatus2(2).location.executorId === "hostB-exec") assert(mapStatus2(2).location.host === "hostB") } @@ -743,9 +743,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) @@ -759,15 +757,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) // the 2nd ResultTask failed complete(taskSets(1), Seq( (Success, 42), (FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0L, 0, 0, "ignored"), null))) // this will get called - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // ask the scheduler to try it again scheduler.resubmitFailedStages() // have the 2nd attempt pass @@ -806,10 +802,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) - runEvent(ExecutorLost("exec-hostA", event)) + completeShuffleMapStageSuccessfully(0, 0, 1) + runEvent(ExecutorLost("hostA-exec", event)) if (expectFileLoss) { intercept[MetadataFetchFailedException] { mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0) @@ -889,16 +883,28 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi * @param stageId - The current stageId * @param attemptIdx - The current attempt count * @param numShufflePartitions - The number of partitions in the next stage + * @param hostNames - Host on which each task in the task set is executed */ private def completeShuffleMapStageSuccessfully( stageId: Int, attemptIdx: Int, - numShufflePartitions: Int): Unit = { - val stageAttempt = taskSets.last - checkStageId(stageId, attemptIdx, stageAttempt) + numShufflePartitions: Int, + hostNames: Seq[String] = Seq.empty[String]): Unit = { + def compareStageAttempt(taskSet: TaskSet): Boolean = { + taskSet.stageId == stageId && taskSet.stageAttemptId == attemptIdx + } + + val stageAttemptOpt = taskSets.find(compareStageAttempt(_)) + assert(stageAttemptOpt.isDefined) + val stageAttempt = stageAttemptOpt.get complete(stageAttempt, stageAttempt.tasks.zipWithIndex.map { case (task, idx) => - (Success, makeMapStatus("host" + ('A' + idx).toChar, numShufflePartitions)) + val hostName = if (idx < hostNames.size) { + hostNames(idx) + } else { + s"host${('A' + idx).toChar}" + } + (Success, makeMapStatus(hostName, numShufflePartitions)) }.toSeq) } @@ -1140,9 +1146,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === HashSet("hostA", "hostB")) @@ -1169,9 +1173,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) // The first result task fails, with a fetch failure for the output from the first mapper. @@ -1231,9 +1233,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) // Complete the map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2, hostNames = Seq("hostA", "hostA")) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) // The first ResultTask fails @@ -1267,9 +1267,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // The map stage should have been submitted. assert(countSubmittedMapStageAttempts() === 1) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // The MapOutputTracker should know about both map output locations. assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1.host).toSet === HashSet("hostA", "hostB")) @@ -1328,9 +1326,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(countSubmittedMapStageAttempts() === 1) // Complete the map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // The reduce stage should have been submitted. assert(countSubmittedReduceStageAttempts() === 1) @@ -1415,7 +1411,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // pretend we were told hostA went away val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -1538,14 +1534,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0)) // things start out smoothly, stage 0 completes with no issues - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostB", shuffleMapRdd.partitions.length)), - (Success, makeMapStatus("hostB", shuffleMapRdd.partitions.length)), - (Success, makeMapStatus("hostA", shuffleMapRdd.partitions.length)) - )) + completeShuffleMapStageSuccessfully( + 0, 0, shuffleMapRdd.partitions.length, Seq("hostB", "hostB", "hostA")) // then one executor dies, and a task fails in stage 1 - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) runEvent(makeCompletionEvent( taskSets(1).tasks(0), FetchFailed(null, firstShuffleId, 2L, 2, 0, "Fetch failed"), @@ -1643,18 +1636,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi makeMapStatus("hostA", reduceRdd.partitions.length))) // now that host goes down - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) // so we resubmit those tasks runEvent(makeCompletionEvent(taskSets(0).tasks(0), Resubmitted, null)) runEvent(makeCompletionEvent(taskSets(0).tasks(1), Resubmitted, null)) // now complete everything on a different host - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)) - )) + completeShuffleMapStageSuccessfully( + 0, 0, reduceRdd.partitions.length, Seq("hostB", "hostB", "hostB")) // now we should submit stage 1, and the map output from stage 0 should be registered @@ -1768,7 +1758,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // correct behavior. val job1Id = 0 // TaskSet priority for Stages run with "job1" as the ActiveJob checkJobPropertiesAndPriority(taskSets(0), "job1", job1Id) - complete(taskSets(0), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) shuffleDep1 } @@ -1785,7 +1775,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // the stage. checkJobPropertiesAndPriority(taskSets(1), "job2", 1) - complete(taskSets(1), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1) assert(taskSets(2).properties != null) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -1817,9 +1807,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi checkJobPropertiesAndPriority(taskSets(2), "job2", job2Id) // run the rest of the stages normally, checking that they have the correct properties - complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 1, 1) checkJobPropertiesAndPriority(taskSets(3), "job2", job2Id) - complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(1, 1, 1) checkJobPropertiesAndPriority(taskSets(4), "job2", job2Id) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) @@ -1841,10 +1831,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) // Tell the DAGScheduler that hostA was lost. - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) + completeShuffleMapStageSuccessfully(0, 0, 1) // At this point, no more tasks are running for the stage (and the TaskSetManager considers the // stage complete), but the tasks that ran on HostA need to be re-run, so the DAGScheduler @@ -1880,19 +1868,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val finalRdd = new MyRDD(sc, 1, List(shuffleDepTwo), tracker = mapOutputTracker) submit(finalRdd, Array(0)) // have the first stage complete normally - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // have the second stage complete normally - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostC", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1, Seq("hostA", "hostC")) // fail the third stage because hostA went down complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // have DAGScheduler try again scheduler.resubmitFailedStages() complete(taskSets(3), Seq((Success, makeMapStatus("hostA", 2)))) @@ -1912,19 +1896,15 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi cacheLocations(shuffleTwoRdd.id -> 0) = Seq(makeBlockManagerId("hostD")) cacheLocations(shuffleTwoRdd.id -> 1) = Seq(makeBlockManagerId("hostC")) // complete stage 0 - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) // complete stage 1 - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostA", 1)), - (Success, makeMapStatus("hostB", 1)))) + completeShuffleMapStageSuccessfully(1, 0, 1) // pretend stage 2 failed because hostA went down complete(taskSets(2), Seq( (FetchFailed(makeBlockManagerId("hostA"), shuffleDepTwo.shuffleId, 0L, 0, 0, "ignored"), null))) // TODO assert this: - // blockManagerMaster.removeExecutor("exec-hostA") + // blockManagerMaster.removeExecutor("hostA-exec") // DAGScheduler should notice the cached copy of the second shuffle and try to get it rerun. scheduler.resubmitFailedStages() assertLocations(taskSets(3), Seq(Seq("hostD"))) @@ -2169,8 +2149,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"))) @@ -2215,8 +2194,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 1, List(shuffleDep, narrowDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0)) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 1)))) + completeShuffleMapStageSuccessfully(0, 0, 1) assert(mapOutputTracker.getMapSizesByExecutorId(shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"))) @@ -2319,9 +2297,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Submit a map stage by itself submitMapStage(shuffleDep) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", reduceRdd.partitions.length)), - (Success, makeMapStatus("hostB", reduceRdd.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, reduceRdd.partitions.length) assert(results.size === 1) results.clear() assertDataStructuresEmpty() @@ -2378,9 +2354,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the first stage assert(taskSets(0).stageId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", rdd1.partitions.length)), - (Success, makeMapStatus("hostB", rdd1.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, rdd1.partitions.length) assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) @@ -2442,9 +2416,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete the stage0. assert(taskSets(0).stageId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", rdd1.partitions.length)), - (Success, makeMapStatus("hostB", rdd1.partitions.length)))) + completeShuffleMapStageSuccessfully(0, 0, rdd1.partitions.length) assert(mapOutputTracker.getMapSizesByExecutorId(dep1.shuffleId, 0).map(_._1).toSet === HashSet(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) assert(listener1.results.size === 1) @@ -2503,7 +2475,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Pretend host A was lost. This will cause the TaskSetManager to resubmit task 0, because it // completed on hostA. val oldEpoch = mapOutputTracker.getEpoch - runEvent(ExecutorLost("exec-hostA", ExecutorKilled)) + runEvent(ExecutorLost("hostA-exec", ExecutorKilled)) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) @@ -2651,9 +2623,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Complete both tasks in rddA. assert(taskSets(0).stageId === 0 && taskSets(0).stageAttemptId === 0) - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostA", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2, Seq("hostA", "hostA")) // Fetch failed for task(stageId=1, stageAttemptId=0, partitionId=0) running on hostA // and task(stageId=1, stageAttemptId=0, partitionId=1) is still running. @@ -2827,15 +2797,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(finalRdd, Array(0, 1)) // Finish the first shuffle map stage. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) // Finish the second shuffle map stage. - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostC", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostC", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) // The first task of the final stage failed with fetch failure @@ -2894,14 +2860,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(4).tasks.length == 2) // Finish all stage. - complete(taskSets(4), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 1, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) - complete(taskSets(5), Seq( - (Success, makeMapStatus("hostC", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 2, 2, Seq("hostC", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) complete(taskSets(6), Seq((Success, 11), (Success, 12))) @@ -2932,14 +2894,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(finalRdd, Array(0, 1), properties = new Properties()) // Finish the first 2 shuffle map stages. - complete(taskSets(0), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(0, 0, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId1) === Some(Seq.empty)) - complete(taskSets(1), Seq( - (Success, makeMapStatus("hostB", 2)), - (Success, makeMapStatus("hostD", 2)))) + completeShuffleMapStageSuccessfully(1, 0, 2, Seq("hostB", "hostD")) assert(mapOutputTracker.findMissingPartitions(shuffleId2) === Some(Seq.empty)) // Executor lost on hostB, both of stage 0 and 1 should be reran. @@ -2959,9 +2917,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(taskSets(taskSetIndex).stageId == stageId) assert(taskSets(taskSetIndex).stageAttemptId == 1) assert(taskSets(taskSetIndex).tasks.length == 2) - complete(taskSets(taskSetIndex), Seq( - (Success, makeMapStatus("hostA", 2)), - (Success, makeMapStatus("hostB", 2)))) + completeShuffleMapStageSuccessfully(stageId, 1, 2) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq.empty)) } @@ -3434,8 +3390,9 @@ object DAGSchedulerSuite { def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2, mapTaskId: Long = -1): MapStatus = MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), mapTaskId) - def makeBlockManagerId(host: String): BlockManagerId = - BlockManagerId("exec-" + host, host, 12345) + def makeBlockManagerId(host: String): BlockManagerId = { + BlockManagerId(host + "-exec", host, 12345) + } } object FailThisAttempt { From 75afd889044c789b259b9e609aea34b542490f53 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 15 Jun 2020 19:04:23 -0700 Subject: [PATCH 12/28] Revert "[SPARK-31926][SQL][TEST-HIVE1.2][TEST-MAVEN] Fix concurrency issue for ThriftCLIService to getPortNumber" This reverts commit a0187cd6b59a6b6bb2cadc6711bb663d4d35a844. --- project/SparkBuild.scala | 1 + .../src/test/resources/log4j.properties | 65 ------------------- .../thriftserver/SharedThriftServer.scala | 50 ++++---------- .../ThriftServerQueryTestSuite.scala | 3 - .../ThriftServerWithSparkContextSuite.scala | 11 +--- .../cli/thrift/ThriftBinaryCLIService.java | 11 +--- .../service/cli/thrift/ThriftCLIService.java | 3 - .../cli/thrift/ThriftHttpCLIService.java | 21 ++---- .../cli/thrift/ThriftBinaryCLIService.java | 11 +--- .../service/cli/thrift/ThriftCLIService.java | 3 - .../cli/thrift/ThriftHttpCLIService.java | 21 ++---- 11 files changed, 30 insertions(+), 170 deletions(-) delete mode 100644 sql/hive-thriftserver/src/test/resources/log4j.properties diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 04a3fc4b63050..0035f1d95a90d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -480,6 +480,7 @@ object SparkParallelTestGrouping { "org.apache.spark.sql.hive.thriftserver.SparkSQLEnvSuite", "org.apache.spark.sql.hive.thriftserver.ui.ThriftServerPageSuite", "org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2ListenerSuite", + "org.apache.spark.sql.hive.thriftserver.ThriftServerWithSparkContextSuite", "org.apache.spark.sql.kafka010.KafkaDelegationTokenSuite" ) diff --git a/sql/hive-thriftserver/src/test/resources/log4j.properties b/sql/hive-thriftserver/src/test/resources/log4j.properties deleted file mode 100644 index 21975ba818142..0000000000000 --- a/sql/hive-thriftserver/src/test/resources/log4j.properties +++ /dev/null @@ -1,65 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -# Set everything to be logged to the file hive-thriftserver/target/unit-tests.log -log4j.rootLogger=DEBUG, CA, FA - -#Console Appender -log4j.appender.CA=org.apache.log4j.ConsoleAppender -log4j.appender.CA.layout=org.apache.log4j.PatternLayout -log4j.appender.CA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %p %c: %m%n -log4j.appender.CA.Threshold = WARN - - -#File Appender -log4j.appender.FA=org.apache.log4j.FileAppender -log4j.appender.FA.append=false -log4j.appender.FA.file=target/unit-tests.log -log4j.appender.FA.layout=org.apache.log4j.PatternLayout -log4j.appender.FA.layout.ConversionPattern=%d{HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Set the logger level of File Appender to WARN -log4j.appender.FA.Threshold = DEBUG - -# Some packages are noisy for no good reason. -log4j.additivity.org.apache.hadoop.hive.serde2.lazy.LazyStruct=false -log4j.logger.org.apache.hadoop.hive.serde2.lazy.LazyStruct=OFF - -log4j.additivity.org.apache.hadoop.hive.metastore.RetryingHMSHandler=false -log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF - -log4j.additivity.hive.log=false -log4j.logger.hive.log=OFF - -log4j.additivity.parquet.hadoop.ParquetRecordReader=false -log4j.logger.parquet.hadoop.ParquetRecordReader=OFF - -log4j.additivity.org.apache.parquet.hadoop.ParquetRecordReader=false -log4j.logger.org.apache.parquet.hadoop.ParquetRecordReader=OFF - -log4j.additivity.org.apache.parquet.hadoop.ParquetOutputCommitter=false -log4j.logger.org.apache.parquet.hadoop.ParquetOutputCommitter=OFF - -log4j.additivity.hive.ql.metadata.Hive=false -log4j.logger.hive.ql.metadata.Hive=OFF - -log4j.additivity.org.apache.hadoop.hive.ql.io.RCFile=false -log4j.logger.org.apache.hadoop.hive.ql.io.RCFile=ERROR - -# Parquet related logging -log4j.logger.org.apache.parquet.CorruptStatistics=ERROR -log4j.logger.parquet.CorruptStatistics=ERROR diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index 1c33abff0780c..e002bc0117c8b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -24,7 +24,6 @@ import scala.concurrent.duration._ import scala.util.Try import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hive.service.cli.thrift.ThriftCLIService import org.apache.spark.sql.test.SharedSparkSession @@ -34,8 +33,6 @@ trait SharedThriftServer extends SharedSparkSession { private var hiveServer2: HiveThriftServer2 = _ private var serverPort: Int = 0 - def mode: ServerMode.Value - override def beforeAll(): Unit = { super.beforeAll() // Retries up to 3 times with different port numbers if the server fails to start @@ -53,21 +50,14 @@ trait SharedThriftServer extends SharedSparkSession { hiveServer2.stop() } finally { super.afterAll() - SessionState.detachSession() } } - protected def jdbcUri: String = if (mode == ServerMode.http) { - s"jdbc:hive2://localhost:$serverPort/default;transportMode=http;httpPath=cliservice" - } else { - s"jdbc:hive2://localhost:$serverPort/" - } - protected def withJdbcStatement(fs: (Statement => Unit)*): Unit = { val user = System.getProperty("user.name") require(serverPort != 0, "Failed to bind an actual port for HiveThriftServer2") val connections = - fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + fs.map { _ => DriverManager.getConnection(s"jdbc:hive2://localhost:$serverPort", user, "") } val statements = connections.map(_.createStatement()) try { @@ -79,35 +69,23 @@ trait SharedThriftServer extends SharedSparkSession { } private def startThriftServer(attempt: Int): Unit = { - logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") + logInfo(s"Trying to start HiveThriftServer2:, attempt=$attempt") val sqlContext = spark.newSession().sqlContext - // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could - // randomly pick any free port to use. + // Set the HIVE_SERVER2_THRIFT_PORT to 0, so it could randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) - - try { - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) - hiveServer2.getServices.asScala.foreach { - case t: ThriftCLIService => - serverPort = t.getPortNumber - logInfo(s"Started HiveThriftServer2: mode=$mode, port=$serverPort, attempt=$attempt") - case _ => - } + hiveServer2 = HiveThriftServer2.startWithContext(sqlContext) + hiveServer2.getServices.asScala.foreach { + case t: ThriftCLIService if t.getPortNumber != 0 => + serverPort = t.getPortNumber + logInfo(s"Started HiveThriftServer2: port=$serverPort, attempt=$attempt") + case _ => + } - // Wait for thrift server to be ready to serve the query, via executing simple query - // till the query succeeds. See SPARK-30345 for more details. - eventually(timeout(30.seconds), interval(1.seconds)) { - withJdbcStatement { _.execute("SELECT 1") } - } - } catch { - case e: Exception => - logError("Error start hive server with Context ", e) - if (hiveServer2 != null) { - hiveServer2.stop() - } + // Wait for thrift server to be ready to serve the query, via executing simple query + // till the query succeeds. See SPARK-30345 for more details. + eventually(timeout(30.seconds), interval(1.seconds)) { + withJdbcStatement { _.execute("SELECT 1") } } } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 553f10a275bce..15cc3109da3f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -54,9 +54,6 @@ import org.apache.spark.sql.types._ */ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServer { - - override def mode: ServerMode.Value = ServerMode.binary - override protected def testFile(fileName: String): String = { val url = Thread.currentThread().getContextClassLoader.getResource(fileName) // Copy to avoid URISyntaxException during accessing the resources in `sql/core` diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index d6420dee41adb..3e1fce78ae71c 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.thriftserver -trait ThriftServerWithSparkContextSuite extends SharedThriftServer { +class ThriftServerWithSparkContextSuite extends SharedThriftServer { test("SPARK-29911: Uncache cached tables when session closed") { val cacheManager = spark.sharedState.cacheManager @@ -42,12 +42,3 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { } } } - - -class ThriftServerWithSparkContextInBinarySuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.binary -} - -class ThriftServerWithSparkContextInHttpSuite extends ThriftServerWithSparkContextSuite { - override def mode: ServerMode.Value = ServerMode.http -} diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index 00bdf7e19126e..e1ee503b81209 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -46,7 +45,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -101,14 +100,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index 783e5795aca76..8fce9d9383438 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -175,7 +175,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -634,8 +633,6 @@ public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index bd64c777c1d76..1099a00b67eb7 100644 --- a/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.cli.thrift.TCLIService.Iface; @@ -54,8 +53,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -146,19 +150,6 @@ protected void initializeServer() { + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.fatal( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index ce79e3c8228a6..a7de9c0f3d0d2 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.conf.HiveConf.ConfVars; import org.apache.hadoop.hive.shims.ShimLoader; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; @@ -47,7 +46,7 @@ public ThriftBinaryCLIService(CLIService cliService) { } @Override - protected void initializeServer() { + public void run() { try { // Server thread pool String threadPoolName = "HiveServer2-Handler-Pool"; @@ -102,14 +101,6 @@ protected void initializeServer() { String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - @Override - public void run() { - try { server.serve(); } catch (Throwable t) { LOG.error( diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index e46799a1c427d..d41c3b493bb47 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -176,7 +176,6 @@ public synchronized void init(HiveConf hiveConf) { public synchronized void start() { super.start(); if (!isStarted && !isEmbedded) { - initializeServer(); new Thread(this).start(); isStarted = true; } @@ -671,8 +670,6 @@ public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) return resp; } - protected abstract void initializeServer(); - @Override public abstract void run(); diff --git a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index ab9ed5b1f371e..73d5f84476af0 100644 --- a/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v2.3/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -28,7 +28,6 @@ import org.apache.hadoop.hive.shims.ShimLoader; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.Shell; -import org.apache.hive.service.ServiceException; import org.apache.hive.service.auth.HiveAuthFactory; import org.apache.hive.service.cli.CLIService; import org.apache.hive.service.rpc.thrift.TCLIService; @@ -55,8 +54,13 @@ public ThriftHttpCLIService(CLIService cliService) { super(cliService, ThriftHttpCLIService.class.getSimpleName()); } + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ @Override - protected void initializeServer() { + public void run() { try { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests @@ -147,19 +151,6 @@ protected void initializeServer() { + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); - } catch (Exception t) { - throw new ServiceException("Error initializing " + getName(), t); - } - } - - /** - * Configure Jetty to serve http requests. Example of a client connection URL: - * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, - * e.g. http://gateway:port/hive2/servlets/thrifths2/ - */ - @Override - public void run() { - try { httpServer.join(); } catch (Throwable t) { LOG.error( From e9145d41f3eae53dcee7d298ee1ae9d065645019 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 06:00:05 +0000 Subject: [PATCH 13/28] [SPARK-31986][SQL] Fix Julian-Gregorian micros rebasing of overlapping local timestamps ### What changes were proposed in this pull request? It fixes microseconds rebasing from the hybrid calendar (Julian + Gregorian) to Proleptic Gregorian calendar in the function `RebaseDateTime`.`rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long` in the case of local timestamp overlapping. In the case of overlapping, we look ahead of 1 day to determinate which instant we should take - earlier or later zoned timestamp. If our current standard zone and DST offsets are equal to zone offset of the next date, we choose the later timestamp otherwise the earlier one. For example, the local timestamp **1945-11-18 01:30:00.0** can be mapped to two instants (microseconds since 1970-01-01 00:00:00Z): -761211000000000 or -761207400000000. If the first one is passed to `rebaseJulianToGregorianMicros()`, we take the earlier instant in Proleptic Gregorian calendar while rebasing **1945-11-18T01:30+09:00[Asia/Hong_Kong]** otherwise the later one **1945-11-18T01:30+08:00[Asia/Hong_Kong]**. Note: The fix assumes that only one transition of standard or DST offsets can occur during a day. ### Why are the changes needed? Current implementation of `rebaseJulianToGregorianMicros()` handles timestamps overlapping only during daylight saving time but overlapping can happen also during transition from one standard time zone to another one. For example in the case of `Asia/Hong_Kong`, the time zone switched from `Japan Standard Time` (UTC+9) to `Hong Kong Time` (UTC+8) on _Sunday, 18 November, 1945 01:59:59 AM_. The changes allow to handle the special case as well. ### Does this PR introduce _any_ user-facing change? There is no behaviour change for timestamps of CE after 0001-01-01. The PR might affects timestamps of BCE for which the modified `rebaseJulianToGregorianMicros()` is called directly. ### How was this patch tested? 1. By existing tests in `DateTimeUtilsSuite`, `RebaseDateTimeSuite`, `DateFunctionsSuite`, `DateExpressionsSuite` and `TimestampFormatterSuite`. 2. Added new checks to `RebaseDateTimeSuite`.`SPARK-31959: JST -> HKT at Asia/Hong_Kong in 1945`: ```scala assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) ``` 3. Regenerated `julian-gregorian-rebase-micros.json` with the step of 30 minutes, and got the same JSON file. The JSON file isn't affected because previously it was generated with the step of 1 week. And the spike in diffs/switch points during 1 hour of timestamp overlapping wasn't detected. Closes #28816 from MaxGekk/fix-overlap-julian-2-grep. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/RebaseDateTime.scala | 57 ++++++++++--------- .../catalyst/util/RebaseDateTimeSuite.scala | 3 + 2 files changed, 34 insertions(+), 26 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index c31dc624b0611..8b23bed6d2788 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.util import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId} import java.time.temporal.ChronoField import java.util.{Calendar, TimeZone} +import java.util.Calendar.{DAY_OF_MONTH, DST_OFFSET, ERA, HOUR_OF_DAY, MINUTE, MONTH, SECOND, YEAR, ZONE_OFFSET} import scala.collection.mutable.AnyRefMap @@ -102,15 +103,15 @@ object RebaseDateTime { .setInstant(Math.multiplyExact(days, MILLIS_PER_DAY)) .build() val localDate = LocalDate.of( - utcCal.get(Calendar.YEAR), - utcCal.get(Calendar.MONTH) + 1, + utcCal.get(YEAR), + utcCal.get(MONTH) + 1, // The number of days will be added later to handle non-existing // Julian dates in Proleptic Gregorian calendar. // For example, 1000-02-29 exists in Julian calendar because 1000 // is a leap year but it is not a leap year in Gregorian calendar. 1) - .`with`(ChronoField.ERA, utcCal.get(Calendar.ERA)) - .plusDays(utcCal.get(Calendar.DAY_OF_MONTH) - 1) + .`with`(ChronoField.ERA, utcCal.get(ERA)) + .plusDays(utcCal.get(DAY_OF_MONTH) - 1) Math.toIntExact(localDate.toEpochDay) } @@ -350,9 +351,9 @@ object RebaseDateTime { // If so, we will take zone offsets from the previous day otherwise from the next day. // This assumes that transitions cannot happen often than once per 2 days. val shift = if (trans.getOffsetBefore == zonedDateTime.getOffset) -1 else 1 - cloned.add(Calendar.DAY_OF_MONTH, shift) - cal.set(Calendar.ZONE_OFFSET, cloned.get(Calendar.ZONE_OFFSET)) - cal.set(Calendar.DST_OFFSET, cloned.get(Calendar.DST_OFFSET)) + cloned.add(DAY_OF_MONTH, shift) + cal.set(ZONE_OFFSET, cloned.get(ZONE_OFFSET)) + cal.set(DST_OFFSET, cloned.get(DST_OFFSET)) } millisToMicros(cal.getTimeInMillis) + ldt.get(ChronoField.MICRO_OF_SECOND) } @@ -413,32 +414,36 @@ object RebaseDateTime { .setInstant(microsToMillis(micros)) .build() val localDateTime = LocalDateTime.of( - cal.get(Calendar.YEAR), - cal.get(Calendar.MONTH) + 1, + cal.get(YEAR), + cal.get(MONTH) + 1, // The number of days will be added later to handle non-existing // Julian dates in Proleptic Gregorian calendar. // For example, 1000-02-29 exists in Julian calendar because 1000 // is a leap year but it is not a leap year in Gregorian calendar. 1, - cal.get(Calendar.HOUR_OF_DAY), - cal.get(Calendar.MINUTE), - cal.get(Calendar.SECOND), + cal.get(HOUR_OF_DAY), + cal.get(MINUTE), + cal.get(SECOND), (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) - .`with`(ChronoField.ERA, cal.get(Calendar.ERA)) - .plusDays(cal.get(Calendar.DAY_OF_MONTH) - 1) + .`with`(ChronoField.ERA, cal.get(ERA)) + .plusDays(cal.get(DAY_OF_MONTH) - 1) val zonedDateTime = localDateTime.atZone(zoneId) - // Assuming the daylight saving switchover time is 2:00, the local clock will go back to - // 2:00 after hitting 2:59. This means the local time between [2:00, 3:00) appears twice, and - // can map to two different physical times (seconds from the UTC epoch). - // Java 8 time API resolves the ambiguity by picking the earlier physical time. This is the same - // as Java 7 time API, except for 2:00 where Java 7 picks the later physical time. - // Here we detect the "2:00" case and pick the latter physical time, to be compatible with the - // Java 7 date-time. - val adjustedZdt = if (cal.get(Calendar.DST_OFFSET) == 0) { - zonedDateTime.withLaterOffsetAtOverlap() - } else { - zonedDateTime - } + // In the case of local timestamp overlapping, we need to choose the correct time instant + // which is related to the original local timestamp. We look ahead of 1 day, and if the next + // date has the same standard zone and DST offsets, the current local timestamp belongs to + // the period after the transition. In that case, we take the later zoned date time otherwise + // earlier one. Here, we assume that transitions happen not often than once per day. + val trans = zoneId.getRules.getTransition(localDateTime) + val adjustedZdt = if (trans != null && trans.isOverlap) { + val dstOffset = cal.get(DST_OFFSET) + val zoneOffset = cal.get(ZONE_OFFSET) + cal.add(DAY_OF_MONTH, 1) + if (zoneOffset == cal.get(ZONE_OFFSET) && dstOffset == cal.get(DST_OFFSET)) { + zonedDateTime.withLaterOffsetAtOverlap() + } else { + zonedDateTime.withEarlierOffsetAtOverlap() + } + } else zonedDateTime instantToMicros(adjustedZdt.toInstant) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index 8a51f158a9429..b3363169b391e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -445,6 +445,9 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { // Check reverse rebasing assert(rebaseJulianToGregorianMicros(rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(rebasedLaterMicros) === laterMicros) + // Check reverse not-optimized rebasing + assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) + assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) } } } From fe68e95a5a5a2358c4e66a4f5b7b5cc6d3f5aaba Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Tue, 16 Jun 2020 16:41:08 +0900 Subject: [PATCH 14/28] [SPARK-24634][SS][FOLLOWUP] Rename the variable from "numLateInputs" to "numRowsDroppedByWatermark" ### What changes were proposed in this pull request? This PR renames the variable from "numLateInputs" to "numRowsDroppedByWatermark" so that it becomes self-explanation. ### Why are the changes needed? This is originated from post-review, see https://github.com/apache/spark/pull/28607#discussion_r439853232 ### Does this PR introduce _any_ user-facing change? No, as SPARK-24634 is not introduced in any release yet. ### How was this patch tested? Existing UTs. Closes #28828 from HeartSaVioR/SPARK-24634-v3-followup. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: HyukjinKwon --- .../structured-streaming-programming-guide.md | 9 ++++---- .../streaming/ProgressReporter.scala | 6 ++++- .../streaming/statefulOperators.scala | 14 +++++------ .../apache/spark/sql/streaming/progress.scala | 10 ++++---- .../streaming/EventTimeWatermarkSuite.scala | 16 +++++++------ .../sql/streaming/StateStoreMetricsTest.scala | 23 +++++++++++-------- .../StreamingDeduplicationSuite.scala | 11 +++++---- .../sql/streaming/StreamingJoinSuite.scala | 10 ++++---- ...StreamingQueryStatusAndProgressSuite.scala | 8 +++---- 9 files changed, 59 insertions(+), 48 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 53b7a3a8e46f6..a371f4f50f9f0 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1678,12 +1678,11 @@ emits late rows if the operator uses Append mode. Spark provides two ways to check the number of late rows on stateful operators which would help you identify the issue: 1. On Spark UI: check the metrics in stateful operator nodes in query execution details page in SQL tab -2. On Streaming Query Listener: check "numLateInputs" in "stateOperators" in QueryProcessEvent. +2. On Streaming Query Listener: check "numRowsDroppedByWatermark" in "stateOperators" in QueryProcessEvent. -Please note that the definition of "input" is relative: it doesn't always mean "input rows" for the operator. -Streaming aggregation does pre-aggregate input rows and checks the late inputs against pre-aggregated inputs, -hence the number is not same as the number of original input rows. You'd like to check the fact whether the value is zero -or non-zero. +Please note that "numRowsDroppedByWatermark" represents the number of "dropped" rows by watermark, which is not always same as the count of "late input rows" for the operator. +It depends on the implementation of the operator - e.g. streaming aggregation does pre-aggregate input rows and checks the late inputs against pre-aggregated inputs, +hence the number is not same as the number of original input rows. You'd like to just check the fact whether the value is zero or non-zero. There's a known workaround: split your streaming query into multiple queries per stateful operator, and ensure end-to-end exactly once per query. Ensuring end-to-end exactly once for the last query is optional. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index 55491d96f9b1a..2c737206dd2d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -222,7 +222,11 @@ trait ProgressReporter extends Logging { lastExecution.executedPlan.collect { case p if p.isInstanceOf[StateStoreWriter] => val progress = p.asInstanceOf[StateStoreWriter].getProgress() - if (hasExecuted) progress else progress.copy(newNumRowsUpdated = 0, newNumLateInputs = 0) + if (hasExecuted) { + progress + } else { + progress.copy(newNumRowsUpdated = 0, newNumRowsDroppedByWatermark = 0) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 073266bd621d2..74daaf80b10e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -77,8 +77,8 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numLateInputs" -> SQLMetrics.createMetric(sparkContext, - "number of inputs which are later than watermark ('inputs' are relative to operators)"), + "numRowsDroppedByWatermark" -> SQLMetrics.createMetric(sparkContext, + "number of rows which are dropped by watermark"), "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows"), "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to update"), @@ -102,7 +102,7 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => numRowsTotal = longMetric("numTotalStateRows").value, numRowsUpdated = longMetric("numUpdatedStateRows").value, memoryUsedBytes = longMetric("stateMemory").value, - numLateInputs = longMetric("numLateInputs").value, + numRowsDroppedByWatermark = longMetric("numRowsDroppedByWatermark").value, javaConvertedCustomMetrics ) } @@ -137,11 +137,11 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => protected def applyRemovingRowsOlderThanWatermark( iter: Iterator[InternalRow], - predicateFilterOutLateInput: BasePredicate): Iterator[InternalRow] = { + predicateDropRowByWatermark: BasePredicate): Iterator[InternalRow] = { iter.filterNot { row => - val lateInput = predicateFilterOutLateInput.eval(row) - if (lateInput) longMetric("numLateInputs") += 1 - lateInput + val shouldDrop = predicateDropRowByWatermark.eval(row) + if (shouldDrop) longMetric("numRowsDroppedByWatermark") += 1 + shouldDrop } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index 22bae76ef4222..482f2b4bf4ed7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -43,7 +43,7 @@ class StateOperatorProgress private[sql]( val numRowsTotal: Long, val numRowsUpdated: Long, val memoryUsedBytes: Long, - val numLateInputs: Long, + val numRowsDroppedByWatermark: Long, val customMetrics: ju.Map[String, JLong] = new ju.HashMap() ) extends Serializable { @@ -55,15 +55,15 @@ class StateOperatorProgress private[sql]( private[sql] def copy( newNumRowsUpdated: Long, - newNumLateInputs: Long): StateOperatorProgress = - new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, newNumLateInputs, - customMetrics) + newNumRowsDroppedByWatermark: Long): StateOperatorProgress = + new StateOperatorProgress(numRowsTotal, newNumRowsUpdated, memoryUsedBytes, + newNumRowsDroppedByWatermark, customMetrics) private[sql] def jsonValue: JValue = { ("numRowsTotal" -> JInt(numRowsTotal)) ~ ("numRowsUpdated" -> JInt(numRowsUpdated)) ~ ("memoryUsedBytes" -> JInt(memoryUsedBytes)) ~ - ("numLateInputs" -> JInt(numLateInputs)) ~ + ("numRowsDroppedByWatermark" -> JInt(numRowsDroppedByWatermark)) ~ ("customMetrics" -> { if (!customMetrics.isEmpty) { val keys = customMetrics.keySet.asScala.toSeq.sorted diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 329196a5cfef7..b5e313d2e107c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -298,11 +298,11 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((10, 5)), assertNumStateRows(2), - assertNumLateInputs(0), + assertNumRowsDroppedByWatermark(0), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), assertNumStateRows(2), - assertNumLateInputs(1) + assertNumRowsDroppedByWatermark(1) ) } @@ -323,15 +323,15 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche AddData(inputData, 25), // Advance watermark to 15 seconds CheckNewAnswer((25, 1)), assertNumStateRows(2), - assertNumLateInputs(0), + assertNumRowsDroppedByWatermark(0), AddData(inputData, 10, 25), // Ignore 10 as its less than watermark CheckNewAnswer((25, 2)), assertNumStateRows(2), - assertNumLateInputs(1), + assertNumRowsDroppedByWatermark(1), AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), assertNumStateRows(2), - assertNumLateInputs(1) + assertNumRowsDroppedByWatermark(1) ) } @@ -788,7 +788,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche true } - private def assertNumLateInputs(numLateInputs: Long): AssertOnQuery = AssertOnQuery { q => + private def assertNumRowsDroppedByWatermark( + numRowsDroppedByWatermark: Long): AssertOnQuery = AssertOnQuery { q => q.processAllAvailable() val progressWithData = q.recentProgress.filterNot { p => // filter out batches which are falling into one of types: @@ -796,7 +797,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // 2) empty input batch p.inputRowsPerSecond == 0 }.lastOption.get - assert(progressWithData.stateOperators(0).numLateInputs === numLateInputs) + assert(progressWithData.stateOperators(0).numRowsDroppedByWatermark + === numRowsDroppedByWatermark) true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala index 640f5181aa526..be83f0ee776da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StateStoreMetricsTest.scala @@ -32,9 +32,9 @@ trait StateStoreMetricsTest extends StreamTest { def assertNumStateRows( total: Seq[Long], updated: Seq[Long], - lateInputs: Seq[Long]): AssertOnQuery = + droppedByWatermark: Seq[Long]): AssertOnQuery = AssertOnQuery(s"Check total state rows = $total, updated state rows = $updated" + - s", late inputs = $lateInputs") { q => + s", rows dropped by watermark = $droppedByWatermark") { q => // This assumes that the streaming query will not make any progress while the eventually // is being executed. eventually(timeout(streamingTimeout)) { @@ -55,8 +55,8 @@ trait StateStoreMetricsTest extends StreamTest { val allNumUpdatedRowsSinceLastCheck = progressesSinceLastCheck.map(_.stateOperators.map(_.numRowsUpdated)) - val allNumLateInputsSinceLastCheck = - progressesSinceLastCheck.map(_.stateOperators.map(_.numLateInputs)) + val allNumRowsDroppedByWatermarkSinceLastCheck = + progressesSinceLastCheck.map(_.stateOperators.map(_.numRowsDroppedByWatermark)) lazy val debugString = "recent progresses:\n" + progressesSinceLastCheck.map(_.prettyJson).mkString("\n\n") @@ -67,8 +67,10 @@ trait StateStoreMetricsTest extends StreamTest { val numUpdatedRows = arraySum(allNumUpdatedRowsSinceLastCheck, numStateOperators) assert(numUpdatedRows === updated, s"incorrect updates rows, $debugString") - val numLateInputs = arraySum(allNumLateInputsSinceLastCheck, numStateOperators) - assert(numLateInputs === lateInputs, s"incorrect late inputs, $debugString") + val numRowsDroppedByWatermark = arraySum(allNumRowsDroppedByWatermarkSinceLastCheck, + numStateOperators) + assert(numRowsDroppedByWatermark === droppedByWatermark, + s"incorrect dropped rows by watermark, $debugString") lastCheckedRecentProgressIndex = recentProgress.length - 1 } @@ -77,11 +79,14 @@ trait StateStoreMetricsTest extends StreamTest { def assertNumStateRows(total: Seq[Long], updated: Seq[Long]): AssertOnQuery = { assert(total.length === updated.length) - assertNumStateRows(total, updated, lateInputs = (0 until total.length).map(_ => 0L)) + assertNumStateRows(total, updated, droppedByWatermark = (0 until total.length).map(_ => 0L)) } - def assertNumStateRows(total: Long, updated: Long, lateInput: Long = 0): AssertOnQuery = { - assertNumStateRows(Seq(total), Seq(updated), Seq(lateInput)) + def assertNumStateRows( + total: Long, + updated: Long, + droppedByWatermark: Long = 0): AssertOnQuery = { + assertNumStateRows(Seq(total), Seq(updated), Seq(droppedByWatermark)) } def arraySum(arraySeq: Seq[Array[Long]], arrayLength: Int): Seq[Long] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index ee0b980a9d87b..f6f150e7bbbb1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -54,13 +54,13 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { testStream(result, Append)( AddData(inputData, "a" -> 1), CheckLastBatch("a" -> 1), - assertNumStateRows(total = 1, updated = 1, lateInput = 0), + assertNumStateRows(total = 1, updated = 1, droppedByWatermark = 0), AddData(inputData, "a" -> 2), // Dropped CheckLastBatch(), - assertNumStateRows(total = 1, updated = 0, lateInput = 0), + assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 0), AddData(inputData, "b" -> 1), CheckLastBatch("b" -> 1), - assertNumStateRows(total = 2, updated = 1, lateInput = 0) + assertNumStateRows(total = 2, updated = 1, droppedByWatermark = 0) ) } @@ -102,7 +102,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckNewAnswer(), - assertNumStateRows(total = 1, updated = 0, lateInput = 1), + assertNumStateRows(total = 1, updated = 0, droppedByWatermark = 1), AddData(inputData, 45), // Advance watermark to 35 seconds, no-data-batch drops row 25 CheckNewAnswer(45), @@ -136,7 +136,8 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { AddData(inputData, 10), // Should not emit anything as data less than watermark CheckLastBatch(), - assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L), lateInputs = Seq(0L, 1L)), + assertNumStateRows(total = Seq(2L, 1L), updated = Seq(0L, 0L), + droppedByWatermark = Seq(0L, 1L)), AddData(inputData, 40), // Advance watermark to 30 seconds CheckLastBatch((15 -> 1), (25 -> 1)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 1f6d0a9945682..833fc24e8b66d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -166,7 +166,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(input1, 5), CheckNewAnswer(), // Same reason as above - assertNumStateRows(total = 2, updated = 0, lateInput = 1) + assertNumStateRows(total = 2, updated = 0, droppedByWatermark = 1) ) } @@ -219,12 +219,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with // (1, 28) ==> passed filter, matched with left (1, 3) and (1, 5), added to state AddData(rightInput, (1, 20), (1, 21), (1, 28)), CheckNewAnswer((1, 3, 21), (1, 5, 21), (1, 3, 28), (1, 5, 28)), - assertNumStateRows(total = 5, updated = 1, lateInput = 1), + assertNumStateRows(total = 5, updated = 1, droppedByWatermark = 1), // New data to left input with leftTime <= 20 should be filtered due to event time watermark AddData(leftInput, (1, 20), (1, 21)), CheckNewAnswer((1, 21, 28)), - assertNumStateRows(total = 6, updated = 1, lateInput = 1) + assertNumStateRows(total = 6, updated = 1, droppedByWatermark = 1) ) } @@ -293,7 +293,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(leftInput, (1, 30), (1, 31)), // 30 should not be processed or added to state CheckNewAnswer((1, 31, 26), (1, 31, 30), (1, 31, 31)), - assertNumStateRows(total = 11, updated = 1, lateInput = 1), // only 31 added + assertNumStateRows(total = 11, updated = 1, droppedByWatermark = 1), // only 31 added // Advance the watermark AddData(rightInput, (1, 80)), @@ -307,7 +307,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with AddData(rightInput, (1, 46), (1, 50)), // 46 should not be processed or added to state CheckNewAnswer((1, 49, 50), (1, 50, 50)), - assertNumStateRows(total = 7, updated = 1, lateInput = 1) // 50 added + assertNumStateRows(total = 7, updated = 1, droppedByWatermark = 1) // 50 added ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala index 79028a6c442d0..98e2342c78e56 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryStatusAndProgressSuite.scala @@ -64,7 +64,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, | "memoryUsedBytes" : 3, - | "numLateInputs" : 0, + | "numRowsDroppedByWatermark" : 0, | "customMetrics" : { | "loadedMapCacheHitCount" : 1, | "loadedMapCacheMissCount" : 0, @@ -115,7 +115,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numRowsTotal" : 0, | "numRowsUpdated" : 1, | "memoryUsedBytes" : 2, - | "numLateInputs" : 0 + | "numRowsDroppedByWatermark" : 0 | } ], | "sources" : [ { | "description" : "source", @@ -323,7 +323,7 @@ object StreamingQueryStatusAndProgressSuite { "avg" -> "2016-12-05T20:54:20.827Z", "watermark" -> "2016-12-05T20:54:20.827Z").asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numLateInputs = 0, + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 3, numRowsDroppedByWatermark = 0, customMetrics = new java.util.HashMap(Map("stateOnCurrentVersionSizeBytes" -> 2L, "loadedMapCacheHitCount" -> 1L, "loadedMapCacheMissCount" -> 0L) .mapValues(long2Long).asJava) @@ -355,7 +355,7 @@ object StreamingQueryStatusAndProgressSuite { // empty maps should be handled correctly eventTime = new java.util.HashMap(Map.empty[String, String].asJava), stateOperators = Array(new StateOperatorProgress( - numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2, numLateInputs = 0)), + numRowsTotal = 0, numRowsUpdated = 1, memoryUsedBytes = 2, numRowsDroppedByWatermark = 0)), sources = Array( new SourceProgress( description = "source", From f0e6d0ec13d9cdadf341d1b976623345bcdb1028 Mon Sep 17 00:00:00 2001 From: GuoPhilipse Date: Tue, 16 Jun 2020 08:35:35 +0000 Subject: [PATCH 15/28] [SPARK-31710][SQL] Fail casting numeric to timestamp by default ## What changes were proposed in this pull request? we fail casting from numeric to timestamp by default. ## Why are the changes needed? casting from numeric to timestamp is not a non-standard,meanwhile it may generate different result between spark and other systems,for example hive ## Does this PR introduce any user-facing change? Yes,user cannot cast numeric to timestamp directly,user have to use the following function to achieve the same effect:TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS ## How was this patch tested? unit test added Closes #28593 from GuoPhilipse/31710-fix-compatibility. Lead-authored-by: GuoPhilipse Co-authored-by: GuoPhilipse <46367746+GuoPhilipse@users.noreply.github.com> Signed-off-by: Wenchen Fan --- docs/sql-migration-guide.md | 2 + python/pyspark/sql/dataframe.py | 5 +- python/pyspark/sql/functions.py | 13 +++ python/pyspark/sql/tests/test_dataframe.py | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 13 ++- .../expressions/datetimeExpressions.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 12 +++ .../sql/catalyst/expressions/CastSuite.scala | 99 ++++++++++++------- .../org/apache/spark/sql/functions.scala | 9 ++ .../sql-tests/inputs/udf/udf-window.sql | 16 +-- .../resources/sql-tests/inputs/window.sql | 16 +-- .../sql-tests/results/udf/udf-window.sql.out | 16 +-- .../sql-tests/results/window.sql.out | 16 +-- .../apache/spark/sql/DateFunctionsSuite.scala | 4 +- .../spark/sql/StatisticsCollectionSuite.scala | 3 +- .../streaming/MicroBatchExecutionSuite.scala | 4 +- .../sources/ForeachWriterSuite.scala | 6 +- .../streaming/EventTimeWatermarkSuite.scala | 40 ++++---- .../sql/streaming/FileStreamSinkSuite.scala | 2 +- .../FlatMapGroupsWithStateSuite.scala | 7 +- .../streaming/StreamingAggregationSuite.scala | 2 +- .../StreamingDeduplicationSuite.scala | 10 +- .../sql/streaming/StreamingJoinSuite.scala | 35 ++++--- .../execution/HiveCompatibilitySuite.scala | 6 ++ ...esting-0-6a01a94ef1b0d29152c88cd3083fd70b} | 0 ...cast #3-0-732ed232ac592c5e7f7c913a88874fd2 | 1 - ...cast #4-0-6d2da5cfada03605834e38bc4075bc79 | 1 - .../sql/hive/execution/HiveQuerySuite.scala | 27 +++-- .../sql/hive/execution/HiveUDFSuite.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 10 +- 30 files changed, 233 insertions(+), 150 deletions(-) rename sql/hive/src/test/resources/golden/{constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 => constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b} (100%) delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 delete mode 100644 sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 0130923e694b1..d6550c30b9553 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -30,6 +30,8 @@ license: | - In Spark 3.1, `from_unixtime`, `unix_timestamp`,`to_unix_timestamp`, `to_timestamp` and `to_date` will fail if the specified datetime pattern is invalid. In Spark 3.0 or earlier, they result `NULL`. + - In Spark 3.1, casting numeric to timestamp will be forbidden by default. It's strongly recommended to use dedicated functions: TIMESTAMP_SECONDS, TIMESTAMP_MILLIS and TIMESTAMP_MICROS. Or you can set `spark.sql.legacy.allowCastNumericToTimestamp` to true to work around it. See more details in SPARK-31710. + ## Upgrading from Spark SQL 2.4 to 3.0 ### Dataset/DataFrame APIs diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 03e3b9ca4bd05..3ad899bcc3670 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -534,7 +534,10 @@ def withWatermark(self, eventTime, delayThreshold): .. note:: Evolving - >>> sdf.select('name', sdf.time.cast('timestamp')).withWatermark('time', '10 minutes') + >>> from pyspark.sql.functions import timestamp_seconds + >>> sdf.select( + ... 'name', + ... timestamp_seconds(sdf.time).alias('time')).withWatermark('time', '10 minutes') DataFrame[name: string, time: timestamp] """ if not eventTime or type(eventTime) is not str: diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index de0d38e2aed12..0c8c34dd87996 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1427,6 +1427,19 @@ def to_utc_timestamp(timestamp, tz): return Column(sc._jvm.functions.to_utc_timestamp(_to_java_column(timestamp), tz)) +@since(3.1) +def timestamp_seconds(col): + """ + >>> from pyspark.sql.functions import timestamp_seconds + >>> time_df = spark.createDataFrame([(1230219000,)], ['unix_time']) + >>> time_df.select(timestamp_seconds(time_df.unix_time).alias('ts')).collect() + [Row(ts=datetime.datetime(2008, 12, 25, 7, 30))] + """ + + sc = SparkContext._active_spark_context + return Column(sc._jvm.functions.timestamp_seconds(_to_java_column(col))) + + @since(2.0) @ignore_unicode_prefix def window(timeColumn, windowDuration, slideDuration=None, startTime=None): diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 062e61663a332..30c3fd4c8d167 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -644,7 +644,7 @@ def test_to_pandas_from_mixed_dataframe(self): CAST(col6 AS DOUBLE) AS double, CAST(col7 AS BOOLEAN) AS boolean, CAST(col8 AS STRING) AS string, - CAST(col9 AS TIMESTAMP) AS timestamp + timestamp_seconds(col9) AS timestamp FROM VALUES (1, 1, 1, 1, 1, 1, 1, 1, 1), (NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL) """ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ef70915a5c969..5576e71b57024 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -59,7 +59,8 @@ object Cast { case (StringType, TimestampType) => true case (BooleanType, TimestampType) => true case (DateType, TimestampType) => true - case (_: NumericType, TimestampType) => true + case (_: NumericType, TimestampType) => + SQLConf.get.getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) case (StringType, DateType) => true case (TimestampType, DateType) => true @@ -266,7 +267,15 @@ abstract class CastBase extends UnaryExpression with TimeZoneAwareExpression wit TypeCheckResult.TypeCheckSuccess } else { TypeCheckResult.TypeCheckFailure( - s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}") + if (child.dataType.isInstanceOf[NumericType] && dataType.isInstanceOf[TimestampType]) { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}," + + "you can enable the casting by setting " + + s"${SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key} to true," + + "but we strongly recommend using function " + + "TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead." + } else { + s"cannot cast ${child.dataType.catalogString} to ${dataType.catalogString}" + }) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index c5cf447c103b7..b46c3fb349ee1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1549,7 +1549,7 @@ case class ParseToDate(left: Expression, format: Option[Expression], child: Expr def this(left: Expression, format: Expression) { this(left, Option(format), - Cast(Cast(UnixTimestamp(left, format), TimestampType), DateType)) + Cast(SecondsToTimestamp(UnixTimestamp(left, format)), DateType)) } def this(left: Expression) = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 33f40b47d072b..7f63d79a21ed6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2608,6 +2608,15 @@ object SQLConf { .checkValue(_ > 0, "The timeout value must be positive") .createWithDefault(10L) + val LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP = + buildConf("spark.sql.legacy.allowCastNumericToTimestamp") + .internal() + .doc("When true, allow casting numeric to timestamp," + + "when false, forbid the cast, more details in SPARK-31710") + .version("3.1.0") + .booleanConf + .createWithDefault(false) + /** * Holds information about keys that have been deprecated. * @@ -3196,6 +3205,9 @@ class SQLConf extends Serializable with Logging { def integerGroupingIdEnabled: Boolean = getConf(SQLConf.LEGACY_INTEGER_GROUPING_ID) + def legacyAllowCastNumericToTimestamp: Boolean = + getConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 6af995cab64fe..35b4017980138 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -50,7 +50,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { } protected def checkNullCast(from: DataType, to: DataType): Unit = { - checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(Literal.create(null, from), to, UTC_OPT), null) + } } test("null cast") { @@ -239,7 +241,9 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkCast(1.5, 1.5f) checkCast(1.5, "1.5") - checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble) + } } test("cast from string") { @@ -305,17 +309,20 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(cast(cast(cast( cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType), 5.toLong) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - 5.toShort) - checkEvaluation( - cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), - DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), - null) - checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), - ByteType), TimestampType), LongType), StringType), ShortType), - 5.toShort) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + 5.toShort) + checkEvaluation( + cast(cast(cast(cast(cast(cast("5", TimestampType, UTC_OPT), ByteType), + DecimalType.SYSTEM_DEFAULT), LongType), StringType), ShortType), + null) + checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.SYSTEM_DEFAULT), + ByteType), TimestampType), LongType), StringType), ShortType), + 5.toShort) + } checkEvaluation(cast("23", DoubleType), 23d) checkEvaluation(cast("23", IntegerType), 23) @@ -376,29 +383,32 @@ abstract class CastSuiteBase extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(ts, LongType), 15.toLong) checkEvaluation(cast(ts, FloatType), 15.003f) checkEvaluation(cast(ts, DoubleType), 15.003) - checkEvaluation(cast(cast(tss, ShortType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation(cast(cast(tss, LongType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), - millis.toFloat / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), - millis.toDouble / MILLIS_PER_SECOND) - checkEvaluation( - cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), - Decimal(1)) - // A test for higher precision than millis - checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(tss, ShortType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, IntegerType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation(cast(cast(tss, LongType), TimestampType), + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), + millis.toFloat / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), + millis.toDouble / MILLIS_PER_SECOND) + checkEvaluation( + cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), + Decimal(1)) - checkEvaluation(cast(Double.NaN, TimestampType), null) - checkEvaluation(cast(1.0 / 0.0, TimestampType), null) - checkEvaluation(cast(Float.NaN, TimestampType), null) - checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + // A test for higher precision than millis + checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) + + checkEvaluation(cast(Double.NaN, TimestampType), null) + checkEvaluation(cast(1.0 / 0.0, TimestampType), null) + checkEvaluation(cast(Float.NaN, TimestampType), null) + checkEvaluation(cast(1.0f / 0.0f, TimestampType), null) + } } test("cast from array") { @@ -1026,8 +1036,11 @@ class CastSuite extends CastSuiteBase { test("cast from int 2") { checkEvaluation(cast(1, LongType), 1.toLong) - checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) - checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> "true") { + checkEvaluation(cast(cast(1000, TimestampType), LongType), 1000.toLong) + checkEvaluation(cast(cast(-1200, TimestampType), LongType), -1200.toLong) + } checkEvaluation(cast(123, DecimalType.USER_DEFAULT), Decimal(123)) checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123)) @@ -1310,6 +1323,20 @@ class CastSuite extends CastSuiteBase { checkEvaluation(cast(negativeTs, LongType), expectedSecs) } } + + test("SPARK-31710:fail casting from numeric to timestamp by default") { + Seq(true, false).foreach { enable => + withSQLConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP.key -> enable.toString) { + assert(cast(2.toByte, TimestampType).resolved == enable) + assert(cast(10.toShort, TimestampType).resolved == enable) + assert(cast(3, TimestampType).resolved == enable) + assert(cast(10L, TimestampType).resolved == enable) + assert(cast(Decimal(1.2), TimestampType).resolved == enable) + assert(cast(1.7f, TimestampType).resolved == enable) + assert(cast(2.3d, TimestampType).resolved == enable) + } + } + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 0cca3e7b47c56..62ad5ea9b5935 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -3358,6 +3358,15 @@ object functions { window(timeColumn, windowDuration, windowDuration, "0 second") } + /** + * Creates timestamp from the number of seconds since UTC epoch. + * @group = datetime_funcs + * @since = 3.1.0 + */ + def timestamp_seconds(e: Column): Column = withExpr { + SecondsToTimestamp(e.expr) + } + ////////////////////////////////////////////////////////////////////////////////////////////// // Collection functions ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql index bcbf87f8a04c5..1659f1c819592 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-window.sql @@ -1,15 +1,15 @@ --This test file was converted from window.sql. -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index 3d05dfda6c3fa..72d812d6a4e49 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -5,15 +5,15 @@ -- Test data. CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate); -- RowsBetween diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out index a915c1bd6c717..a84070535b658 100644 --- a/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-window.sql.out @@ -4,15 +4,15 @@ -- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 625088f90ced9..ede044a44fdaa 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -4,15 +4,15 @@ -- !query CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES -(null, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), "a"), -(1, 2L, 2.5D, date("2017-08-02"), timestamp(1502000000), "a"), -(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "a"), -(1, null, 1.0D, date("2017-08-01"), timestamp(1501545600), "b"), -(2, 3L, 3.3D, date("2017-08-03"), timestamp(1503000000), "b"), -(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp(1609372800), "b"), +(null, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "a"), +(1, 2L, 2.5D, date("2017-08-02"), timestamp_seconds(1502000000), "a"), +(2, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "a"), +(1, null, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), "b"), +(2, 3L, 3.3D, date("2017-08-03"), timestamp_seconds(1503000000), "b"), +(3, 2147483650L, 100.001D, date("2020-12-31"), timestamp_seconds(1609372800), "b"), (null, null, null, null, null, null), -(3, 1L, 1.0D, date("2017-08-01"), timestamp(1501545600), null) +(3, 1L, 1.0D, date("2017-08-01"), timestamp_seconds(1501545600), null) AS testData(val, val_long, val_double, val_date, val_timestamp, cate) -- !query schema struct<> diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 5cc9e156db1b5..9caa4c0377009 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -639,7 +639,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val now = sql("select unix_timestamp()").collect().head.getLong(0) checkAnswer( - sql(s"select cast ($now as timestamp)"), + sql(s"select timestamp_seconds($now)"), Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) } } @@ -716,7 +716,7 @@ class DateFunctionsSuite extends QueryTest with SharedSparkSession { val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") checkAnswer(df.select(to_timestamp(col("ss"))), - df.select(unix_timestamp(col("ss")).cast("timestamp"))) + df.select(timestamp_seconds(unix_timestamp(col("ss"))))) checkAnswer(df.select(to_timestamp(col("ss"))), Seq( Row(ts1), Row(ts2))) if (legacyParserPolicy == "legacy") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 708b98e8fe15a..91ec1b5ab2937 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.test.SQLTestData.ArrayData @@ -467,7 +468,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared withTable(table) { TimeZone.setDefault(srcTimeZone) spark.range(start, end) - .select('id.cast(TimestampType).cast(t).as(column)) + .select(timestamp_seconds($"id").cast(t).as(column)) .write.saveAsTable(table) sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala index c228740df07c8..c0f25e3a5053f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecutionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.streaming import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.StreamTest class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { @@ -33,7 +33,7 @@ class MicroBatchExecutionSuite extends StreamTest with BeforeAndAfter { test("SPARK-24156: do not plan a no-data batch again after it has already been planned") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala index 0c17320acade9..e87bd11f0dca5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterSuite.scala @@ -26,7 +26,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkException import org.apache.spark.sql.ForeachWriter import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSparkSession @@ -163,7 +163,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -197,7 +197,7 @@ class ForeachWriterSuite extends StreamTest with SharedSparkSession with BeforeA val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index b5e313d2e107c..705e980df86a0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark import org.apache.spark.sql.catalyst.util.DateTimeTestUtils.UTC import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.MemorySink -import org.apache.spark.sql.functions.{count, window} +import org.apache.spark.sql.functions.{count, timestamp_seconds, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ import org.apache.spark.util.Utils @@ -129,7 +129,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // No event time metrics when there is no watermarking val inputData1 = MemoryStream[Int] val aggWithoutWatermark = inputData1.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) .select($"window".getField("start").cast("long").as[Long], $"count".as[Long]) @@ -146,7 +146,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData2 = MemoryStream[Int] val aggWithWatermark = inputData2.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -169,7 +169,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -224,7 +224,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche // All event time metrics where watermarking is set val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -286,7 +286,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -311,7 +311,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -341,7 +341,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val input = MemoryStream[Long] val aggWithWatermark = input.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "2 years 5 months") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -373,7 +373,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("recovery") { val inputData = MemoryStream[Int] val df = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -408,14 +408,14 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val first = MemoryStream[Int] val firstDf = first.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select('value) val second = MemoryStream[Int] val secondDf = second.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "5 seconds") .select('value) @@ -485,7 +485,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -510,7 +510,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy($"eventTime") .agg(count("*") as 'count) @@ -549,8 +549,8 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("the new watermark should override the old one") { val df = MemoryStream[(Long, Long)].toDF() - .withColumn("first", $"_1".cast("timestamp")) - .withColumn("second", $"_2".cast("timestamp")) + .withColumn("first", timestamp_seconds($"_1")) + .withColumn("second", timestamp_seconds($"_2")) .withWatermark("first", "1 minute") .withWatermark("second", "2 minutes") @@ -562,7 +562,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("EventTime watermark should be ignored in batch query.") { val df = testData - .withColumn("eventTime", $"key".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"key")) .withWatermark("eventTime", "1 minute") .select("eventTime") .as[Long] @@ -601,7 +601,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("SPARK-27340 Alias on TimeWindow expression cause watermark metadata lost") { val inputData = MemoryStream[Int] val aliasWindow = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .select(window($"eventTime", "5 seconds") as 'aliasWindow) // Check the eventTime metadata is kept in the top level alias. @@ -631,7 +631,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val windowedAggregation = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) @@ -767,10 +767,10 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche input1: MemoryStream[Int], input2: MemoryStream[Int]): Dataset[_] = { val df1 = input1.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") val df2 = input2.toDF - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "15 seconds") df1.union(df2).select($"eventTime".cast("int")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index 877965100f018..a25451bef62fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -210,7 +210,7 @@ abstract class FileStreamSinkSuite extends StreamTest { val inputData = MemoryStream[Long] val inputDF = inputData.toDF.toDF("time") val outputDf = inputDF - .selectExpr("CAST(time AS timestamp) AS timestamp") + .selectExpr("timestamp_seconds(time) AS timestamp") .withWatermark("timestamp", "10 seconds") .groupBy(window($"timestamp", "5 seconds")) .count() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala index b04f8b0d4d174..e2887e78b0508 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateSuite.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.RDDScanExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, MemoryStateStore, StateStore, StateStoreId, StateStoreMetrics, UnsafeRowPair} +import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{DataType, IntegerType} @@ -826,7 +827,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -901,7 +902,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Int)] val result = inputData.toDS - .select($"_1".as("key"), $"_2".cast("timestamp").as("eventTime")) + .select($"_1".as("key"), timestamp_seconds($"_2").as("eventTime")) .withWatermark("eventTime", "10 seconds") .as[(String, Long)] .groupByKey(_._1) @@ -1111,7 +1112,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val inputData = MemoryStream[(String, Long)] val result = inputData.toDF().toDF("key", "time") - .selectExpr("key", "cast(time as timestamp) as timestamp") + .selectExpr("key", "timestamp_seconds(time) as timestamp") .withWatermark("timestamp", "10 second") .as[(String, Long)] .groupByKey(x => x._1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala index 85e1b85b84d26..cb69460ca1580 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingAggregationSuite.scala @@ -188,7 +188,7 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { testWithAllStateVersions("state metrics - append mode") { val inputData = MemoryStream[Int] val aggWithWatermark = inputData.toDF() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .groupBy(window($"eventTime", "5 seconds") as 'window) .agg(count("*") as 'count) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala index f6f150e7bbbb1..1f346aac8d2c2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingDeduplicationSuite.scala @@ -86,7 +86,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("deduplicate with watermark") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) @@ -113,7 +113,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("deduplicate with aggregate - append mode") { val inputData = MemoryStream[Int] val windowedaggregate = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .withWatermark("eventTime", "10 seconds") @@ -230,7 +230,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("SPARK-19841: watermarkPredicate should filter based on keys") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("time", "id") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id", "time") // Change the column positions .select($"id") @@ -249,7 +249,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { test("SPARK-21546: dropDuplicates should ignore watermark when it's not a key") { val input = MemoryStream[(Int, Int)] val df = input.toDS.toDF("id", "time") - .withColumn("time", $"time".cast("timestamp")) + .withColumn("time", timestamp_seconds($"time")) .withWatermark("time", "1 second") .dropDuplicates("id") .select($"id", $"time".cast("long")) @@ -265,7 +265,7 @@ class StreamingDeduplicationSuite extends StateStoreMetricsTest { def testWithFlag(flag: Boolean): Unit = withClue(s"with $flagKey = $flag") { val inputData = MemoryStream[Int] val result = inputData.toDS() - .withColumn("eventTime", $"value".cast("timestamp")) + .withColumn("eventTime", timestamp_seconds($"value")) .withWatermark("eventTime", "10 seconds") .dropDuplicates() .select($"eventTime".cast("long").as[Long]) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index 833fc24e8b66d..caca749f9dd1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -87,11 +87,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -127,12 +128,13 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val input2 = MemoryStream[Int] val df1 = input1.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", ('value * 2) as "leftValue") + .select('value as "key", timestamp_seconds($"value") as "timestamp", + ('value * 2) as "leftValue") .withWatermark("timestamp", "10 seconds") .select('key, window('timestamp, "10 second"), 'leftValue) val df2 = input2.toDF - .select('value as "key", 'value.cast("timestamp") as "timestamp", + .select('value as "key", timestamp_seconds($"value") as "timestamp", ('value * 3) as "rightValue") .select('key, window('timestamp, "10 second"), 'rightValue) @@ -177,11 +179,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -235,11 +238,12 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "20 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "30 seconds") val condition = expr( @@ -425,7 +429,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream inner join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -500,7 +504,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val df = input.toDF .select( 'value as "key", - 'value.cast("timestamp") as s"${prefix}Time", + timestamp_seconds($"value") as s"${prefix}Time", ('value * multiplier) as s"${prefix}Value") .withWatermark(s"${prefix}Time", "10 seconds") @@ -682,11 +686,12 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val rightInput = MemoryStream[(Int, Int)] val df1 = leftInput.toDF.toDF("leftKey", "time") - .select('leftKey, 'time.cast("timestamp") as "leftTime", ('leftKey * 2) as "leftValue") + .select('leftKey, timestamp_seconds($"time") as "leftTime", ('leftKey * 2) as "leftValue") .withWatermark("leftTime", "10 seconds") val df2 = rightInput.toDF.toDF("rightKey", "time") - .select('rightKey, 'time.cast("timestamp") as "rightTime", ('rightKey * 3) as "rightValue") + .select('rightKey, timestamp_seconds($"time") as "rightTime", + ('rightKey * 3) as "rightValue") .withWatermark("rightTime", "10 seconds") val joined = @@ -777,7 +782,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) @@ -840,7 +845,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) // we're just flipping "left" and "right" from left outer join and apply right outer join @@ -883,7 +888,7 @@ class StreamingOuterJoinSuite extends StreamTest with StateStoreMetricsTest with test("SPARK-26187 restore the stream-stream outer join query from Spark 2.4") { val inputStream = MemoryStream[(Int, Long)] val df = inputStream.toDS() - .select(col("_1").as("value"), col("_2").cast("timestamp").as("timestamp")) + .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) val leftStream = df.select(col("value").as("leftId"), col("timestamp").as("leftTime")) diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index db1f6fbd97d90..82af7dceb27f2 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -39,6 +39,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone + private val originalLegacyAllowCastNumericToTimestamp = + TestHive.conf.legacyAllowCastNumericToTimestamp def testCases: Seq[(String, File)] = { hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) @@ -58,6 +60,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") + // Ensures that cast numeric to timestamp enabled so that we can test them + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, true) RuleExecutor.resetMetrics() } @@ -68,6 +72,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) + TestHive.setConf(SQLConf.LEGACY_ALLOW_CAST_NUMERIC_TO_TIMESTAMP, + originalLegacyAllowCastNumericToTimestamp) // For debugging dump some statistics about how much time was spent in various optimizer rules logWarning(RuleExecutor.dumpTimeSpent()) diff --git a/sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 b/sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b similarity index 100% rename from sql/hive/src/test/resources/golden/constant null testing-0-237a6af90a857da1efcbe98f6bbbf9d6 rename to sql/hive/src/test/resources/golden/constant null testing-0-6a01a94ef1b0d29152c88cd3083fd70b diff --git a/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 b/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 deleted file mode 100644 index 5625e59da8873..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #3-0-732ed232ac592c5e7f7c913a88874fd2 +++ /dev/null @@ -1 +0,0 @@ -1.2 diff --git a/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 b/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 deleted file mode 100644 index 1d94c8a014fb4..0000000000000 --- a/sql/hive/src/test/resources/golden/timestamp cast #4-0-6d2da5cfada03605834e38bc4075bc79 +++ /dev/null @@ -1 +0,0 @@ --1.2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b10a8cb8bf2bf..2b42444ceeaa1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -201,14 +201,17 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |IF(TRUE, CAST(NULL AS BINARY), CAST("1" AS BINARY)) AS COL18, |IF(FALSE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL19, |IF(TRUE, CAST(NULL AS DATE), CAST("1970-01-01" AS DATE)) AS COL20, - |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL21, + |IF(TRUE, CAST(NULL AS TIMESTAMP), CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL21, |IF(FALSE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL22, |IF(TRUE, CAST(NULL AS DECIMAL), CAST(1 AS DECIMAL)) AS COL23 |FROM src LIMIT 1""".stripMargin) test("constant null testing timestamp") { - val r1 = sql("SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), CAST(1 AS TIMESTAMP)) AS COL20") - .collect().head + var r1 = sql( + """ + |SELECT IF(FALSE, CAST(NULL AS TIMESTAMP), + |CAST('1969-12-31 16:00:01' AS TIMESTAMP)) AS COL20 + """.stripMargin).collect().head assert(new Timestamp(1000) == r1.getTimestamp(0)) } @@ -552,28 +555,22 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd // Jdk version leads to different query output for double, so not use createQueryTest here test("timestamp cast #1") { - val res = sql("SELECT CAST(CAST(1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(1 == res.getDouble(0)) } test("timestamp cast #2") { - val res = sql("SELECT CAST(CAST(-1 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1").collect().head + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1) AS DOUBLE) FROM src LIMIT 1").collect().head assert(-1 == res.get(0)) } - createQueryTest("timestamp cast #3", - "SELECT CAST(CAST(1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - createQueryTest("timestamp cast #4", - "SELECT CAST(CAST(-1.2 AS TIMESTAMP) AS DOUBLE) FROM src LIMIT 1") - - test("timestamp cast #5") { - val res = sql("SELECT CAST(CAST(1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + test("timestamp cast #3") { + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(1200) AS INT) FROM src LIMIT 1").collect().head assert(1200 == res.getInt(0)) } - test("timestamp cast #6") { - val res = sql("SELECT CAST(CAST(-1200 AS TIMESTAMP) AS INT) FROM src LIMIT 1").collect().head + test("timestamp cast #4") { + val res = sql("SELECT CAST(TIMESTAMP_SECONDS(-1200) AS INT) FROM src LIMIT 1").collect().head assert(-1200 == res.getInt(0)) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 7bca2af379934..057f2f4ce01be 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -434,8 +434,8 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { withTempView("tab1") { Seq(Tuple1(1451400761)).toDF("test_date").createOrReplaceTempView("tab1") sql(s"CREATE TEMPORARY FUNCTION testUDFToDate AS '${classOf[GenericUDFToDate].getName}'") - val count = sql("select testUDFToDate(cast(test_date as timestamp))" + - " from tab1 group by testUDFToDate(cast(test_date as timestamp))").count() + val count = sql("select testUDFToDate(timestamp_seconds(test_date))" + + " from tab1 group by testUDFToDate(timestamp_seconds(test_date))").count() sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFToDate") assert(count == 1) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index d12eae0e410b1..2fe6a59a27c1b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -1172,7 +1172,7 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi test("SPARK-6785: HiveQuerySuite - Date comparison test 2") { checkAnswer( - sql("SELECT CAST(CAST(0 AS timestamp) AS date) > CAST(0 AS timestamp) FROM src LIMIT 1"), + sql("SELECT CAST(timestamp_seconds(0) AS date) > timestamp_seconds(0) FROM src LIMIT 1"), Row(false)) } @@ -1182,10 +1182,10 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi sql( """ | SELECT - | CAST(CAST(0 AS timestamp) AS date), - | CAST(CAST(CAST(0 AS timestamp) AS date) AS string), - | CAST(0 AS timestamp), - | CAST(CAST(0 AS timestamp) AS string), + | CAST(timestamp_seconds(0) AS date), + | CAST(CAST(timestamp_seconds(0) AS date) AS string), + | timestamp_seconds(0), + | CAST(timestamp_seconds(0) AS string), | CAST(CAST(CAST('1970-01-01 23:00:00' AS timestamp) AS date) AS timestamp) | FROM src LIMIT 1 """.stripMargin), From d24d27f1bc39e915df23d65f8fda0d83e716b308 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 16 Jun 2020 19:20:44 +0900 Subject: [PATCH 16/28] [SPARK-31997][SQL][TESTS] Drop test_udtf table when SingleSessionSuite test completed MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? `SingleSessionSuite` not do `DROP TABLE IF EXISTS test_udtf` when test completed, then if we do mvn test `HiveThriftBinaryServerSuite`, the test case named `SPARK-11595 ADD JAR with input path having URL scheme` will FAILED because it want to re-create an exists table test_udtf. ### Why are the changes needed? test suite shouldn't rely on their execution order ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? Manual test,mvn test SingleSessionSuite and HiveThriftBinaryServerSuite in order Closes #28838 from LuciferYang/drop-test-table. Authored-by: yangjie01 Signed-off-by: HyukjinKwon --- .../spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index ff54879cb5084..3fd46dc82f03f 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -883,7 +883,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { s"--conf ${HIVE_THRIFT_SERVER_SINGLESESSION.key}=true" :: Nil test("share the temporary functions across JDBC connections") { - withMultipleConnectionJdbcStatement()( + withMultipleConnectionJdbcStatement("test_udtf")( { statement => val jarPath = "../hive/src/test/resources/TestUDTF.jar" val jarURL = s"file://${System.getProperty("user.dir")}/$jarPath" From 6e9ff72195e52c4fb61630900a545864c2be7772 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 12:56:27 +0000 Subject: [PATCH 17/28] [SPARK-31984][SQL] Make micros rebasing functions via local timestamps pure ### What changes were proposed in this pull request? 1. Set the given time zone as the first parameter of `RebaseDateTime`.`rebaseJulianToGregorianMicros()` and `rebaseGregorianToJulianMicros()` to Java 7 `GregorianCalendar`. ```scala val cal = new Calendar.Builder() // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems .setCalendarType("gregory") ... .setTimeZone(tz) .build() ``` This makes the instance of the calendar independent from the default JVM time zone. 2. Change type of the first parameter from `ZoneId` to `TimeZone`. This allows to avoid unnecessary conversion from `TimeZone` to `ZoneId`, for example in ```scala def rebaseJulianToGregorianMicros(micros: Long): Long = { ... if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { rebaseJulianToGregorianMicros(timeZone.toZoneId, micros) ``` and back to `TimeZone` inside of `rebaseJulianToGregorianMicros(zoneId: ZoneId, ...)` 3. Modify tests in `RebaseDateTimeSuite`, and set the default JVM time zone only for functions that depend on it. ### Why are the changes needed? 1. Ignoring passed parameter and using a global variable is bad practice. 2. Dependency from the global state doesn't allow to run the functions in parallel otherwise there is non-zero probability that the functions may return wrong result if the default JVM is changed during their execution. 3. This open opportunity for parallelisation of JSON files generation `gregorian-julian-rebase-micros.json` and `julian-gregorian-rebase-micros.json`. Currently, the tests `generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'` generate the JSON files by iterating over all time zones sequentially w/ step of 1 week. Due to the large step, we can miss some spikes in diffs between 2 calendars (Java 8 Gregorian and Java 7 hybrid calendars) as the PR https://github.com/apache/spark/pull/28787 fixed and https://github.com/apache/spark/pull/28816 should fix. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running existing tests from `RebaseDateTimeSuite`. Closes #28824 from MaxGekk/pure-micros-rebasing. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/util/RebaseDateTime.scala | 19 +- .../catalyst/util/RebaseDateTimeSuite.scala | 163 +++++++++--------- 2 files changed, 94 insertions(+), 88 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 8b23bed6d2788..24fe9fa42c19f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -320,13 +320,14 @@ object RebaseDateTime { * Julian calendar: 1582-01-01 00:00:00.123456 -> -12243196799876544 * The code below converts -12244061221876544 to -12243196799876544. * - * @param zoneId The time zone ID at which the rebasing should be performed. + * @param tz The time zone at which the rebasing should be performed. * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' * in Proleptic Gregorian calendar. It can be negative. * @return The rebased microseconds since the epoch in Julian calendar. */ - private[sql] def rebaseGregorianToJulianMicros(zoneId: ZoneId, micros: Long): Long = { + private[sql] def rebaseGregorianToJulianMicros(tz: TimeZone, micros: Long): Long = { val instant = microsToInstant(micros) + val zoneId = tz.toZoneId val zonedDateTime = instant.atZone(zoneId) var ldt = zonedDateTime.toLocalDateTime if (ldt.isAfter(julianEndTs) && ldt.isBefore(gregorianStartTs)) { @@ -337,6 +338,7 @@ object RebaseDateTime { .setCalendarType("gregory") .setDate(ldt.getYear, ldt.getMonthValue - 1, ldt.getDayOfMonth) .setTimeOfDay(ldt.getHour, ldt.getMinute, ldt.getSecond) + .setTimeZone(tz) .build() // A local timestamp can have 2 instants in the cases of switching from: // 1. Summer to winter time. @@ -379,7 +381,7 @@ object RebaseDateTime { val tzId = timeZone.getID val rebaseRecord = gregJulianRebaseMap.getOrNull(tzId) if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { - rebaseGregorianToJulianMicros(timeZone.toZoneId, micros) + rebaseGregorianToJulianMicros(timeZone, micros) } else { rebaseMicros(rebaseRecord, micros) } @@ -401,17 +403,17 @@ object RebaseDateTime { * Proleptic Gregorian calendar: 1582-01-01 00:00:00.123456 -> -12244061221876544 * The code below converts -12243196799876544 to -12244061221876544. * - * @param zoneId The time zone ID at which the rebasing should be performed. + * @param tz The time zone at which the rebasing should be performed. * @param micros The number of microseconds since the epoch '1970-01-01T00:00:00Z' * in the Julian calendar. It can be negative. * @return The rebased microseconds since the epoch in Proleptic Gregorian calendar. */ - private[sql] def rebaseJulianToGregorianMicros(zoneId: ZoneId, micros: Long): Long = { + private[sql] def rebaseJulianToGregorianMicros(tz: TimeZone, micros: Long): Long = { val cal = new Calendar.Builder() - // `gregory` is a hybrid calendar that supports both - // the Julian and Gregorian calendar systems + // `gregory` is a hybrid calendar that supports both the Julian and Gregorian calendar systems .setCalendarType("gregory") .setInstant(microsToMillis(micros)) + .setTimeZone(tz) .build() val localDateTime = LocalDateTime.of( cal.get(YEAR), @@ -427,6 +429,7 @@ object RebaseDateTime { (Math.floorMod(micros, MICROS_PER_SECOND) * NANOS_PER_MICROS).toInt) .`with`(ChronoField.ERA, cal.get(ERA)) .plusDays(cal.get(DAY_OF_MONTH) - 1) + val zoneId = tz.toZoneId val zonedDateTime = localDateTime.atZone(zoneId) // In the case of local timestamp overlapping, we need to choose the correct time instant // which is related to the original local timestamp. We look ahead of 1 day, and if the next @@ -479,7 +482,7 @@ object RebaseDateTime { val tzId = timeZone.getID val rebaseRecord = julianGregRebaseMap.getOrNull(tzId) if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { - rebaseJulianToGregorianMicros(timeZone.toZoneId, micros) + rebaseJulianToGregorianMicros(timeZone, micros) } else { rebaseMicros(rebaseRecord, micros) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index b3363169b391e..254bf01c89b4f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -201,21 +201,21 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("optimization of micros rebasing - Gregorian to Julian") { outstandingZoneIds.foreach { zid => withClue(s"zone id = $zid") { - withDefaultTimeZone(zid) { - val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant) - val end = instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant) - var micros = start - do { - val rebased = rebaseGregorianToJulianMicros(zid, micros) - val rebasedAndOptimized = rebaseGregorianToJulianMicros(micros) - assert(rebasedAndOptimized === rebased) - micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong - } while (micros <= end) - } + val start = instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + val end = instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0) + .atZone(zid) + .toInstant) + var micros = start + do { + val rebased = rebaseGregorianToJulianMicros(TimeZone.getTimeZone(zid), micros) + val rebasedAndOptimized = withDefaultTimeZone(zid) { + rebaseGregorianToJulianMicros(micros) + } + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong + } while (micros <= end) } } } @@ -223,26 +223,26 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("optimization of micros rebasing - Julian to Gregorian") { outstandingZoneIds.foreach { zid => withClue(s"zone id = $zid") { - withDefaultTimeZone(zid) { - val start = rebaseGregorianToJulianMicros( - instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) - val end = rebaseGregorianToJulianMicros( - instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0).atZone(zid).toInstant)) - var micros = start - do { - val rebased = rebaseJulianToGregorianMicros(zid, micros) - val rebasedAndOptimized = rebaseJulianToGregorianMicros(micros) - assert(rebasedAndOptimized === rebased) - micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong - } while (micros <= end) - } + val start = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + val end = rebaseGregorianToJulianMicros( + instantToMicros(LocalDateTime.of(2100, 1, 1, 0, 0, 0).atZone(zid).toInstant)) + var micros = start + do { + val rebased = rebaseJulianToGregorianMicros(TimeZone.getTimeZone(zid), micros) + val rebasedAndOptimized = withDefaultTimeZone(zid) { + rebaseJulianToGregorianMicros(micros) + } + assert(rebasedAndOptimized === rebased) + micros += (MICROS_PER_DAY * 30 * (0.5 + Math.random())).toLong + } while (micros <= end) } } } private def generateRebaseJson( - adjustFunc: Long => Long, - rebaseFunc: (ZoneId, Long) => Long, + adjustFunc: (TimeZone, Long) => Long, + rebaseFunc: (TimeZone, Long) => Long, dir: String, fileName: String): Unit = { import java.nio.file.{Files, Paths} @@ -260,14 +260,15 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { .sortBy(_.getId) .foreach { zid => withDefaultTimeZone(zid) { - val start = adjustFunc(instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant)) + val tz = TimeZone.getTimeZone(zid) + val start = adjustFunc( + tz, + instantToMicros(LocalDateTime.of(1, 1, 1, 0, 0, 0).atZone(zid).toInstant)) // sun.util.calendar.ZoneInfo resolves DST after 2037 year incorrectly. // See https://bugs.java.com/bugdatabase/view_bug.do?bug_id=8073446 - val end = adjustFunc(instantToMicros(LocalDateTime.of(2037, 1, 1, 0, 0, 0) - .atZone(zid) - .toInstant)) + val end = adjustFunc( + tz, + instantToMicros(LocalDateTime.of(2037, 1, 1, 0, 0, 0).atZone(zid).toInstant)) var micros = start var diff = Long.MaxValue @@ -276,7 +277,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { val switches = new ArrayBuffer[Long]() val diffs = new ArrayBuffer[Long]() while (micros < end) { - val rebased = rebaseFunc(zid, micros) + val rebased = rebaseFunc(tz, micros) val curDiff = rebased - micros if (curDiff != diff) { if (step > MICROS_PER_SECOND) { @@ -308,7 +309,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { ignore("generate 'gregorian-julian-rebase-micros.json'") { generateRebaseJson( - adjustFunc = identity[Long], + adjustFunc = (_: TimeZone, micros: Long) => micros, rebaseFunc = rebaseGregorianToJulianMicros, dir = "/Users/maximgekk/tmp", fileName = "gregorian-julian-rebase-micros.json") @@ -383,26 +384,27 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { test("rebase not-existed timestamps in the hybrid calendar") { outstandingZoneIds.foreach { zid => - withDefaultTimeZone(zid) { - Seq( - "1582-10-04T23:59:59.999999" -> "1582-10-04 23:59:59.999999", - "1582-10-05T00:00:00.000000" -> "1582-10-15 00:00:00.000000", - "1582-10-06T01:02:03.000001" -> "1582-10-15 01:02:03.000001", - "1582-10-07T00:00:00.000000" -> "1582-10-15 00:00:00.000000", - "1582-10-08T23:59:59.999999" -> "1582-10-15 23:59:59.999999", - "1582-10-09T23:59:59.001001" -> "1582-10-15 23:59:59.001001", - "1582-10-10T00:11:22.334455" -> "1582-10-15 00:11:22.334455", - "1582-10-11T11:12:13.111111" -> "1582-10-15 11:12:13.111111", - "1582-10-12T10:11:12.131415" -> "1582-10-15 10:11:12.131415", - "1582-10-13T00:00:00.123321" -> "1582-10-15 00:00:00.123321", - "1582-10-14T23:59:59.999999" -> "1582-10-15 23:59:59.999999", - "1582-10-15T00:00:00.000000" -> "1582-10-15 00:00:00.000000" - ).foreach { case (gregTs, hybridTs) => - withClue(s"tz = ${zid.getId} greg ts = $gregTs hybrid ts = $hybridTs") { - val hybridMicros = parseToJulianMicros(hybridTs) - val gregorianMicros = parseToGregMicros(gregTs, zid) - - assert(rebaseGregorianToJulianMicros(zid, gregorianMicros) === hybridMicros) + Seq( + "1582-10-04T23:59:59.999999" -> "1582-10-04 23:59:59.999999", + "1582-10-05T00:00:00.000000" -> "1582-10-15 00:00:00.000000", + "1582-10-06T01:02:03.000001" -> "1582-10-15 01:02:03.000001", + "1582-10-07T00:00:00.000000" -> "1582-10-15 00:00:00.000000", + "1582-10-08T23:59:59.999999" -> "1582-10-15 23:59:59.999999", + "1582-10-09T23:59:59.001001" -> "1582-10-15 23:59:59.001001", + "1582-10-10T00:11:22.334455" -> "1582-10-15 00:11:22.334455", + "1582-10-11T11:12:13.111111" -> "1582-10-15 11:12:13.111111", + "1582-10-12T10:11:12.131415" -> "1582-10-15 10:11:12.131415", + "1582-10-13T00:00:00.123321" -> "1582-10-15 00:00:00.123321", + "1582-10-14T23:59:59.999999" -> "1582-10-15 23:59:59.999999", + "1582-10-15T00:00:00.000000" -> "1582-10-15 00:00:00.000000" + ).foreach { case (gregTs, hybridTs) => + withClue(s"tz = ${zid.getId} greg ts = $gregTs hybrid ts = $hybridTs") { + val hybridMicros = withDefaultTimeZone(zid) { parseToJulianMicros(hybridTs) } + val gregorianMicros = parseToGregMicros(gregTs, zid) + + val tz = TimeZone.getTimeZone(zid) + assert(rebaseGregorianToJulianMicros(tz, gregorianMicros) === hybridMicros) + withDefaultTimeZone(zid) { assert(rebaseGregorianToJulianMicros(gregorianMicros) === hybridMicros) } } @@ -416,38 +418,39 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { // clocks were moved backward to become Sunday, 18 November, 1945 01:00:00 AM. // In this way, the overlap happened w/o Daylight Saving Time. val hkZid = getZoneId("Asia/Hong_Kong") + var expected = "1945-11-18 01:30:00.0" + var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) + var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + var overlapInterval = MICROS_PER_HOUR + if (earlierMicros + overlapInterval != laterMicros) { + // Old JDK might have an outdated time zone database. + // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition + // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" + expected = "1945-09-14 23:30:00.0" + ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) + earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) + laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) + // If time zone db doesn't have overlapping at all, set the overlap interval to zero. + overlapInterval = laterMicros - earlierMicros + } + val hkTz = TimeZone.getTimeZone(hkZid) + val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkTz, earlierMicros) + val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkTz, laterMicros) + assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) withDefaultTimeZone(hkZid) { - var expected = "1945-11-18 01:30:00.0" - var ldt = LocalDateTime.of(1945, 11, 18, 1, 30, 0) - var earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) - var laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - var overlapInterval = MICROS_PER_HOUR - if (earlierMicros + overlapInterval != laterMicros) { - // Old JDK might have an outdated time zone database. - // See https://bugs.openjdk.java.net/browse/JDK-8228469: "Hong Kong ... Its 1945 transition - // from JST to HKT was on 11-18 at 02:00, not 09-15 at 00:00" - expected = "1945-09-14 23:30:00.0" - ldt = LocalDateTime.of(1945, 9, 14, 23, 30, 0) - earlierMicros = instantToMicros(ldt.atZone(hkZid).withEarlierOffsetAtOverlap().toInstant) - laterMicros = instantToMicros(ldt.atZone(hkZid).withLaterOffsetAtOverlap().toInstant) - // If time zone db doesn't have overlapping at all, set the overlap interval to zero. - overlapInterval = laterMicros - earlierMicros - } - val rebasedEarlierMicros = rebaseGregorianToJulianMicros(hkZid, earlierMicros) - val rebasedLaterMicros = rebaseGregorianToJulianMicros(hkZid, laterMicros) def toTsStr(micros: Long): String = toJavaTimestamp(micros).toString assert(toTsStr(rebasedEarlierMicros) === expected) assert(toTsStr(rebasedLaterMicros) === expected) - assert(rebasedEarlierMicros + overlapInterval === rebasedLaterMicros) // Check optimized rebasing assert(rebaseGregorianToJulianMicros(earlierMicros) === rebasedEarlierMicros) assert(rebaseGregorianToJulianMicros(laterMicros) === rebasedLaterMicros) // Check reverse rebasing assert(rebaseJulianToGregorianMicros(rebasedEarlierMicros) === earlierMicros) assert(rebaseJulianToGregorianMicros(rebasedLaterMicros) === laterMicros) - // Check reverse not-optimized rebasing - assert(rebaseJulianToGregorianMicros(hkZid, rebasedEarlierMicros) === earlierMicros) - assert(rebaseJulianToGregorianMicros(hkZid, rebasedLaterMicros) === laterMicros) } + // Check reverse not-optimized rebasing + assert(rebaseJulianToGregorianMicros(hkTz, rebasedEarlierMicros) === earlierMicros) + assert(rebaseJulianToGregorianMicros(hkTz, rebasedLaterMicros) === laterMicros) } } From 36435658b116e5bf9b3bfc3e276a068406eddb30 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Tue, 16 Jun 2020 14:07:03 +0000 Subject: [PATCH 18/28] [SPARK-31710][SQL][FOLLOWUP] Replace CAST by TIMESTAMP_SECONDS in benchmarks ### What changes were proposed in this pull request? Replace `CAST(... AS TIMESTAMP` by `TIMESTAMP_SECONDS` in the following benchmarks: - ExtractBenchmark - DateTimeBenchmark - FilterPushdownBenchmark - InExpressionBenchmark ### Why are the changes needed? The benchmarks fail w/o the changes: ``` [info] Running benchmark: datetime +/- interval [info] Running case: date + interval(m) [error] Exception in thread "main" org.apache.spark.sql.AnalysisException: cannot resolve 'CAST(`id` AS TIMESTAMP)' due to data type mismatch: cannot cast bigint to timestamp,you can enable the casting by setting spark.sql.legacy.allowCastNumericToTimestamp to true,but we strongly recommend using function TIMESTAMP_SECONDS/TIMESTAMP_MILLIS/TIMESTAMP_MICROS instead.; line 1 pos 5; [error] 'Project [(cast(cast(id#0L as timestamp) as date) + 1 months) AS (CAST(CAST(id AS TIMESTAMP) AS DATE) + INTERVAL '1 months')#2] [error] +- Range (0, 10000000, step=1, splits=Some(1)) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? By running the affected benchmarks. Closes #28843 from MaxGekk/GuoPhilipse-31710-fix-compatibility-followup. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../benchmark/DateTimeBenchmark.scala | 18 +++++++++--------- .../execution/benchmark/ExtractBenchmark.scala | 8 ++++---- .../benchmark/FilterPushdownBenchmark.scala | 10 +++++----- .../benchmark/InExpressionBenchmark.scala | 10 +++++----- 4 files changed, 23 insertions(+), 23 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala index c7b8737b7a753..b06ca71b04ecc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -54,7 +54,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { private def run(cardinality: Int, func: String): Unit = { codegenBenchmark(s"$func of timestamp", cardinality) { - doBenchmark(cardinality, s"$func(cast(id as timestamp))") + doBenchmark(cardinality, s"$func(timestamp_seconds(id))") } } @@ -64,7 +64,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { val N = 10000000 runBenchmark("datetime +/- interval") { val benchmark = new Benchmark("datetime +/- interval", N, output = output) - val ts = "cast(id as timestamp)" + val ts = "timestamp_seconds(id)" val dt = s"cast($ts as date)" benchmark.addCase("date + interval(m)") { _ => doBenchmark(N, s"$dt + interval 1 month") @@ -105,7 +105,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { benchmark.run() } runBenchmark("Extract components") { - run(N, "cast to timestamp", "cast(id as timestamp)") + run(N, "cast to timestamp", "timestamp_seconds(id)") run(N, "year") run(N, "quarter") run(N, "month") @@ -124,7 +124,7 @@ object DateTimeBenchmark extends SqlBasedBenchmark { run(N, "current_timestamp", "current_timestamp") } runBenchmark("Date arithmetic") { - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" run(N, "cast to date", dateExpr) run(N, "last_day", s"last_day($dateExpr)") run(N, "next_day", s"next_day($dateExpr, 'TU')") @@ -133,31 +133,31 @@ object DateTimeBenchmark extends SqlBasedBenchmark { run(N, "add_months", s"add_months($dateExpr, 10)") } runBenchmark("Formatting dates") { - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" run(N, "format date", s"date_format($dateExpr, 'MMM yyyy')") } runBenchmark("Formatting timestamps") { run(N, "from_unixtime", "from_unixtime(id, 'yyyy-MM-dd HH:mm:ss.SSSSSS')") } runBenchmark("Convert timestamps") { - val timestampExpr = "cast(id as timestamp)" + val timestampExpr = "timestamp_seconds(id)" run(N, "from_utc_timestamp", s"from_utc_timestamp($timestampExpr, 'CET')") run(N, "to_utc_timestamp", s"to_utc_timestamp($timestampExpr, 'CET')") } runBenchmark("Intervals") { - val (start, end) = ("cast(id as timestamp)", "cast((id+8640000) as timestamp)") + val (start, end) = ("timestamp_seconds(id)", "timestamp_seconds(id+8640000)") run(N, "cast interval", start, end) run(N, "datediff", s"datediff($start, $end)") run(N, "months_between", s"months_between($start, $end)") run(1000000, "window", s"window($start, 100, 10, 1)") } runBenchmark("Truncation") { - val timestampExpr = "cast(id as timestamp)" + val timestampExpr = "timestamp_seconds(id)" Seq("YEAR", "YYYY", "YY", "MON", "MONTH", "MM", "DAY", "DD", "HOUR", "MINUTE", "SECOND", "WEEK", "QUARTER").foreach { level => run(N, s"date_trunc $level", s"date_trunc('$level', $timestampExpr)") } - val dateExpr = "cast(cast(id as timestamp) as date)" + val dateExpr = "cast(timestamp_seconds(id) as date)" Seq("year", "yyyy", "yy", "mon", "month", "mm").foreach { level => run(N, s"trunc $level", s"trunc('$level', $dateExpr)") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala index 287854dc3646c..8372698fb47ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ExtractBenchmark.scala @@ -59,10 +59,10 @@ object ExtractBenchmark extends SqlBasedBenchmark { } private def castExpr(from: String): String = from match { - case "timestamp" => "cast(id as timestamp)" - case "date" => "cast(cast(id as timestamp) as date)" - case "interval" => "(cast(cast(id as timestamp) as date) - date'0001-01-01') + " + - "(cast(id as timestamp) - timestamp'1000-01-01 01:02:03.123456')" + case "timestamp" => "timestamp_seconds(id)" + case "date" => "cast(timestamp_seconds(id) as date)" + case "interval" => "(cast(timestamp_seconds(id) as date) - date'0001-01-01') + " + + "(timestamp_seconds(id) - timestamp'1000-01-01 01:02:03.123456')" case other => throw new IllegalArgumentException( s"Unsupported column type $other. Valid column types are 'timestamp' and 'date'") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index b3f65d40ad95b..9ade8b14f59b0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.{DataFrame, SparkSession} -import org.apache.spark.sql.functions.monotonically_increasing_id +import org.apache.spark.sql.functions.{monotonically_increasing_id, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType} @@ -332,11 +332,11 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { val columns = (1 to width).map(i => s"CAST(id AS string) c$i") val df = spark.range(numRows).selectExpr(columns: _*) - .withColumn("value", monotonically_increasing_id().cast(TimestampType)) + .withColumn("value", timestamp_seconds(monotonically_increasing_id())) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => + Seq(s"value = timestamp_seconds($mid)").foreach { whereExpr => val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" .replace("value AND value", "value") filterPushDownBenchmark(numRows, title, whereExpr) @@ -348,8 +348,8 @@ object FilterPushdownBenchmark extends SqlBasedBenchmark { filterPushDownBenchmark( numRows, s"Select $percent% timestamp stored as $fileType rows " + - s"(value < CAST(${numRows * percent / 100} AS timestamp))", - s"value < CAST(${numRows * percent / 100} as timestamp)", + s"(value < timestamp_seconds(${numRows * percent / 100}))", + s"value < timestamp_seconds(${numRows * percent / 100})", selectExpr ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala index caf3387875813..704227e4b4db4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark import org.apache.spark.sql.DataFrame -import org.apache.spark.sql.functions.{array, struct} +import org.apache.spark.sql.functions.{array, struct, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -128,15 +128,15 @@ object InExpressionBenchmark extends SqlBasedBenchmark { private def runTimestampBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { val name = s"$numItems timestamps" - val values = (1 to numItems).map(m => s"CAST('1970-01-01 01:00:00.$m' AS timestamp)") - val df = spark.range(0, numRows).select($"id".cast(TimestampType)) + val values = (1 to numItems).map(m => s"timestamp'1970-01-01 01:00:00.$m'") + val df = spark.range(0, numRows).select(timestamp_seconds($"id").as("id")) runBenchmark(name, df, values, numRows, minNumIters) } private def runDateBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { val name = s"$numItems dates" - val values = (1 to numItems).map(n => 1970 + n).map(y => s"CAST('$y-01-01' AS date)") - val df = spark.range(0, numRows).select($"id".cast(TimestampType).cast(DateType)) + val values = (1 to numItems).map(n => 1970 + n).map(y => s"date'$y-01-01'") + val df = spark.range(0, numRows).select(timestamp_seconds($"id").cast(DateType).as("id")) runBenchmark(name, df, values, numRows, minNumIters) } From 8d577092eda6a0d6a632419d7ee463db011aca2c Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Tue, 16 Jun 2020 09:13:00 -0700 Subject: [PATCH 19/28] [SPARK-31705][SQL][FOLLOWUP] Avoid the unnecessary CNF computation for full-outer joins ### What changes were proposed in this pull request? To avoid the unnecessary CNF computation for full-outer joins, this PR fixes code for filtering out full-outer joins at the entrance of the rule. ### Why are the changes needed? To mitigate optimizer overhead. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. Closes #28810 from maropu/SPARK-31705. Authored-by: Takeshi Yamamuro Signed-off-by: Yuming Wang --- .../sql/catalyst/expressions/predicates.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 22 ++++++++++++------- .../PushCNFPredicateThroughJoin.scala | 14 ++++++++---- 3 files changed, 25 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index c9b57367e0f44..05a5ff45b8fb1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -211,7 +211,7 @@ trait PredicateHelper extends Logging { * @return the CNF result as sequence of disjunctive expressions. If the number of expressions * exceeds threshold on converting `Or`, `Seq.empty` is returned. */ - def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { + protected def conjunctiveNormalForm(condition: Expression): Seq[Expression] = { val postOrderNodes = postOrderTraversal(condition) val resultStack = new mutable.Stack[Seq[Expression]] val maxCnfNodeCount = SQLConf.get.maxCnfNodeCount diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 8e57e9737c73f..e800ee3b93f51 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -1289,11 +1289,17 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (leftEvaluateCondition, rightEvaluateCondition, commonCondition ++ nonDeterministic) } + private def canPushThrough(joinType: JoinType): Boolean = joinType match { + case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { // push the where condition down into join filter - case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) => + case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) + if canPushThrough(joinType) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) joinType match { @@ -1333,13 +1339,13 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (rightFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) - case FullOuter => f // DO Nothing for Full Outer Join - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } // push down the join filter into sub query scanning if applicable - case j @ Join(left, right, joinType, joinCondition, hint) => + case j @ Join(left, right, joinType, joinCondition, hint) if canPushThrough(joinType) => val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) @@ -1369,9 +1375,9 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) Join(newLeft, newRight, joinType, newJoinCond, hint) - case FullOuter => j - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala index f406b7d77ab63..109e5f993c02e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushCNFPredicateThroughJoin.scala @@ -29,8 +29,15 @@ import org.apache.spark.sql.catalyst.rules.Rule * when predicate pushdown happens. */ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + + private def canPushThrough(joinType: JoinType): Boolean = joinType match { + case _: InnerLike | LeftSemi | RightOuter | LeftOuter | LeftAnti | ExistenceJoin(_) => true + case _ => false + } + def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case j @ Join(left, right, joinType, Some(joinCondition), hint) => + case j @ Join(left, right, joinType, Some(joinCondition), hint) + if canPushThrough(joinType) => val predicates = conjunctiveNormalForm(joinCondition) if (predicates.isEmpty) { j @@ -53,9 +60,8 @@ object PushCNFPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelpe Join(newLeft, right, RightOuter, Some(joinCondition), hint) case LeftOuter | LeftAnti | ExistenceJoin(_) => Join(left, newRight, joinType, Some(joinCondition), hint) - case FullOuter => j - case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") - case UsingJoin(_, _) => sys.error("Untransformed Using join node") + case other => + throw new IllegalStateException(s"Unexpected join type: $other") } } } From 2ec9b866285fc059cae6816033babca64b4da7ec Mon Sep 17 00:00:00 2001 From: Zhen Li Date: Tue, 16 Jun 2020 12:59:57 -0500 Subject: [PATCH 20/28] [SPARK-31929][WEBUI] Close leveldbiterator when leveldb.close ### What changes were proposed in this pull request? Close LevelDBIterator when LevelDB.close() is called. ### Why are the changes needed? This pull request would prevent JNI resources leaking from Level DB instance and its' iterators. In before implementation JNI resources from LevelDBIterator are cleaned by finalize() function. This behavior is also mentioned in comments of ["LevelDBIterator.java"](https://github.com/apache/spark/blob/master/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java) by squito . But if DB instance is already closed, then iterator's close method would be ignored. LevelDB's iterator would keep level db files opened (for the case table cache is filled up), till iterator.close() is called. Then these JNI resources (file handle) would be leaked. This JNI resource leaking issue would cause the problem described in [SPARK-31929](https://issues.apache.org/jira/browse/SPARK-31929) on Windows: in spark history server, leaked file handle for level db files would trigger "IOException" when HistoryServerDiskManager try to remove them for releasing disk space. ![IOException](https://user-images.githubusercontent.com/10524738/84134659-7c388680-aa7b-11ea-807f-04dcfa7886a0.JPG) ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Add unit test and manually tested it. Closes #28769 from zhli1142015/close-leveldbiterator-when-leveldb.close. Authored-by: Zhen Li Signed-off-by: Sean Owen --- .../apache/spark/util/kvstore/LevelDB.java | 32 ++++++++++++++++- .../spark/util/kvstore/LevelDBIterator.java | 1 + .../spark/util/kvstore/LevelDBSuite.java | 36 +++++++++++++++++++ .../apache/spark/status/AppStatusStore.scala | 8 ++++- 4 files changed, 75 insertions(+), 2 deletions(-) diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 2ca4b0b2cb9f9..98f33b70fea23 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -19,8 +19,10 @@ import java.io.File; import java.io.IOException; +import java.lang.ref.SoftReference; import java.util.*; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; import static java.nio.charset.StandardCharsets.UTF_8; @@ -64,6 +66,13 @@ public class LevelDB implements KVStore { private final ConcurrentMap typeAliases; private final ConcurrentMap, LevelDBTypeInfo> types; + /** + * Trying to close a JNI LevelDB handle with a closed DB causes JVM crashes. This is used to + * ensure that all iterators are correctly closed before LevelDB is closed. Use soft reference + * to ensure that the iterator can be GCed, when it is only referenced here. + */ + private final ConcurrentLinkedQueue>> iteratorTracker; + public LevelDB(File path) throws Exception { this(path, new KVStoreSerializer()); } @@ -94,6 +103,8 @@ public LevelDB(File path, KVStoreSerializer serializer) throws Exception { aliases = new HashMap<>(); } typeAliases = new ConcurrentHashMap<>(aliases); + + iteratorTracker = new ConcurrentLinkedQueue<>(); } @Override @@ -189,7 +200,9 @@ public KVStoreView view(Class type) throws Exception { @Override public Iterator iterator() { try { - return new LevelDBIterator<>(type, LevelDB.this, this); + LevelDBIterator it = new LevelDBIterator<>(type, LevelDB.this, this); + iteratorTracker.add(new SoftReference<>(it)); + return it; } catch (Exception e) { throw Throwables.propagate(e); } @@ -238,6 +251,14 @@ public void close() throws IOException { } try { + if (iteratorTracker != null) { + for (SoftReference> ref: iteratorTracker) { + LevelDBIterator it = ref.get(); + if (it != null) { + it.close(); + } + } + } _db.close(); } catch (IOException ioe) { throw ioe; @@ -252,6 +273,7 @@ public void close() throws IOException { * with a closed DB can cause JVM crashes, so this ensures that situation does not happen. */ void closeIterator(LevelDBIterator it) throws IOException { + notifyIteratorClosed(it); synchronized (this._db) { DB _db = this._db.get(); if (_db != null) { @@ -260,6 +282,14 @@ void closeIterator(LevelDBIterator it) throws IOException { } } + /** + * Remove iterator from iterator tracker. `LevelDBIterator` calls it to notify + * iterator is closed. + */ + void notifyIteratorClosed(LevelDBIterator it) { + iteratorTracker.removeIf(ref -> it.equals(ref.get())); + } + /** Returns metadata about indices for the given type. */ LevelDBTypeInfo getTypeInfo(Class type) throws Exception { LevelDBTypeInfo ti = types.get(type); diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java index 94e8c9fc5796c..e8fb4fac5ba17 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java @@ -185,6 +185,7 @@ public boolean skip(long n) { @Override public synchronized void close() throws IOException { + db.notifyIteratorClosed(this); if (!closed) { it.close(); closed = true; diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 0b755ba0e8000..f6566617765d4 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -19,6 +19,7 @@ import java.io.File; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; import java.util.stream.Collectors; @@ -276,6 +277,41 @@ public void testNegativeIndexValues() throws Exception { assertEquals(expected, results); } + @Test + public void testCloseLevelDBIterator() throws Exception { + // SPARK-31929: test when LevelDB.close() is called, related LevelDBIterators + // are closed. And files opened by iterators are also closed. + File dbPathForCloseTest = File + .createTempFile( + "test_db_close.", + ".ldb"); + dbPathForCloseTest.delete(); + LevelDB dbForCloseTest = new LevelDB(dbPathForCloseTest); + for (int i = 0; i < 8192; i++) { + dbForCloseTest.write(createCustomType1(i)); + } + String key = dbForCloseTest + .view(CustomType1.class).iterator().next().key; + assertEquals("key0", key); + Iterator it0 = dbForCloseTest + .view(CustomType1.class).max(1).iterator(); + while (it0.hasNext()) { + it0.next(); + } + System.gc(); + Iterator it1 = dbForCloseTest + .view(CustomType1.class).iterator(); + assertEquals("key0", it1.next().key); + try (KVStoreIterator it2 = dbForCloseTest + .view(CustomType1.class).closeableIterator()) { + assertEquals("key0", it2.next().key); + } + dbForCloseTest.close(); + assertTrue(dbPathForCloseTest.exists()); + FileUtils.deleteQuietly(dbPathForCloseTest); + assertTrue(!dbPathForCloseTest.exists()); + } + private CustomType1 createCustomType1(int i) { CustomType1 t = new CustomType1(); t.key = "key" + i; diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 31a6f7d901131..106d272948b9f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -39,7 +39,13 @@ private[spark] class AppStatusStore( def applicationInfo(): v1.ApplicationInfo = { try { // The ApplicationInfo may not be available when Spark is starting up. - store.view(classOf[ApplicationInfoWrapper]).max(1).iterator().next().info + Utils.tryWithResource( + store.view(classOf[ApplicationInfoWrapper]) + .max(1) + .closeableIterator() + ) { it => + it.next().info + } } catch { case _: NoSuchElementException => throw new NoSuchElementException("Failed to get the application information. " + From 7f6a8ab16656533f460f2421dd2381cc91c6cbf4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 16 Jun 2020 16:46:32 -0700 Subject: [PATCH 21/28] [SPARK-31777][ML][PYSPARK] Add user-specified fold column to CrossValidator ### What changes were proposed in this pull request? This patch adds user-specified fold column support to `CrossValidator`. User can assign fold numbers to dataset instead of letting Spark do random splits. ### Why are the changes needed? This gives `CrossValidator` users more flexibility in splitting folds. ### Does this PR introduce _any_ user-facing change? Yes, a new `foldCol` param is added to `CrossValidator`. User can use it to specify custom fold splitting. ### How was this patch tested? Added unit tests. Closes #28704 from viirya/SPARK-31777. Authored-by: Liang-Chi Hsieh Signed-off-by: Liang-Chi Hsieh --- .../spark/ml/tuning/CrossValidator.scala | 43 +++++++-- .../org/apache/spark/mllib/util/MLUtils.scala | 34 ++++++- .../spark/ml/tuning/CrossValidatorSuite.scala | 64 ++++++++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 30 +++++++ python/pyspark/ml/tests/test_tuning.py | 74 ++++++++++++++++ python/pyspark/ml/tuning.py | 88 +++++++++++++++---- 6 files changed, 308 insertions(+), 25 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 858cc57982a3f..e99c55b0cdd85 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -30,13 +30,13 @@ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.evaluation.Evaluator -import org.apache.spark.ml.param.{IntParam, ParamMap, ParamValidators} +import org.apache.spark.ml.param.{IntParam, Param, ParamMap, ParamValidators} import org.apache.spark.ml.param.shared.{HasCollectSubModels, HasParallelism} import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.util.ThreadUtils /** @@ -56,6 +56,19 @@ private[ml] trait CrossValidatorParams extends ValidatorParams { def getNumFolds: Int = $(numFolds) setDefault(numFolds -> 3) + + /** + * Param for the column name of user specified fold number. Once this is specified, + * `CrossValidator` won't do random k-fold split. Note that this column should be + * integer type with range [0, numFolds) and Spark will throw exception on out-of-range + * fold numbers. + */ + val foldCol: Param[String] = new Param[String](this, "foldCol", + "the column name of user specified fold number") + + def getFoldCol: String = $(foldCol) + + setDefault(foldCol, "") } /** @@ -94,6 +107,10 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) @Since("2.0.0") def setSeed(value: Long): this.type = set(seed, value) + /** @group setParam */ + @Since("3.1.0") + def setFoldCol(value: String): this.type = set(foldCol, value) + /** * Set the maximum level of parallelism to evaluate models in parallel. * Default is 1 for serial evaluation @@ -132,7 +149,7 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, numFolds, seed, parallelism) + instr.logParams(this, numFolds, seed, parallelism, foldCol) logTuningParams(instr) val collectSubModelsParam = $(collectSubModels) @@ -142,10 +159,15 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) } else None // Compute metrics for each model over each split - val splits = MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)) + val (splits, schemaWithoutFold) = if ($(foldCol) == "") { + (MLUtils.kFold(dataset.toDF.rdd, $(numFolds), $(seed)), schema) + } else { + val filteredSchema = StructType(schema.filter(_.name != $(foldCol)).toArray) + (MLUtils.kFold(dataset.toDF, $(numFolds), $(foldCol)), filteredSchema) + } val metrics = splits.zipWithIndex.map { case ((training, validation), splitIndex) => - val trainingDataset = sparkSession.createDataFrame(training, schema).cache() - val validationDataset = sparkSession.createDataFrame(validation, schema).cache() + val trainingDataset = sparkSession.createDataFrame(training, schemaWithoutFold).cache() + val validationDataset = sparkSession.createDataFrame(validation, schemaWithoutFold).cache() instr.logDebug(s"Train split $splitIndex with multiple sets of parameters.") // Fit models in a Future for training in parallel @@ -183,7 +205,14 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) } @Since("1.4.0") - override def transformSchema(schema: StructType): StructType = transformSchemaImpl(schema) + override def transformSchema(schema: StructType): StructType = { + if ($(foldCol) != "") { + val foldColDt = schema.apply($(foldCol)).dataType + require(foldColDt.isInstanceOf[IntegerType], + s"The specified `foldCol` column ${$(foldCol)} must be integer type, but got $foldColDt.") + } + transformSchemaImpl(schema) + } @Since("1.4.0") override def copy(extra: ParamMap): CrossValidator = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 9198334ba02a1..d177364d012f4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.mllib.util import scala.annotation.varargs import scala.reflect.ClassTag -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml.linalg.{MatrixUDT => MLMatrixUDT, VectorUDT => MLVectorUDT} @@ -28,7 +28,7 @@ import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.{PartitionwiseSampledRDD, RDD} -import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.functions._ @@ -248,6 +248,36 @@ object MLUtils extends Logging { }.toArray } + /** + * Version of `kFold()` taking a fold column name. + */ + @Since("3.1.0") + def kFold(df: DataFrame, numFolds: Int, foldColName: String): Array[(RDD[Row], RDD[Row])] = { + val foldCol = df.col(foldColName) + val checker = udf { foldNum: Int => + // Valid fold number is in range [0, numFolds). + if (foldNum < 0 || foldNum >= numFolds) { + throw new SparkException(s"Fold number must be in range [0, $numFolds), but got $foldNum.") + } + true + } + (0 until numFolds).map { fold => + val training = df + .filter(checker(foldCol) && foldCol =!= fold) + .drop(foldColName).rdd + val validation = df + .filter(checker(foldCol) && foldCol === fold) + .drop(foldColName).rdd + if (training.isEmpty()) { + throw new SparkException(s"The training data at fold $fold is empty.") + } + if (validation.isEmpty()) { + throw new SparkException(s"The validation data at fold $fold is empty.") + } + (training, validation) + }.toArray + } + /** * Returns a new vector with `1.0` (bias) appended to the input vector. */ diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index a30428ec2d283..d7cbfa8e2e2c5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.regression.LinearRegression import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType class CrossValidatorSuite @@ -40,10 +41,14 @@ class CrossValidatorSuite import testImplicits._ @transient var dataset: Dataset[_] = _ + @transient var datasetWithFold: Dataset[_] = _ override def beforeAll(): Unit = { super.beforeAll() dataset = sc.parallelize(generateLogisticInput(1.0, 1.0, 100, 42), 2).toDF() + val dfWithRandom = dataset.repartition(1).withColumn("random", rand(100L)) + val foldCol = when(col("random") < 0.33, 0).when(col("random") < 0.66, 1).otherwise(2) + datasetWithFold = dfWithRandom.withColumn("fold", foldCol).drop("random").repartition(2) } test("cross validation with logistic regression") { @@ -75,6 +80,65 @@ class CrossValidatorSuite } } + test("cross validation with logistic regression with fold col") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 10)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new CrossValidator() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setNumFolds(3) + .setFoldCol("fold") + val cvModel = cv.fit(datasetWithFold) + + MLTestingUtils.checkCopyAndUids(cv, cvModel) + + val parent = cvModel.bestModel.parent.asInstanceOf[LogisticRegression] + assert(parent.getRegParam === 0.001) + assert(parent.getMaxIter === 10) + assert(cvModel.avgMetrics.length === lrParamMaps.length) + + val result = cvModel.transform(dataset).select("prediction").as[Double].collect() + testTransformerByGlobalCheckFunc[(Double, Vector)](dataset.toDF(), cvModel, "prediction") { + rows => + val result2 = rows.map(_.getDouble(0)) + assert(result === result2) + } + } + + test("cross validation with logistic regression with wrong fold col") { + val lr = new LogisticRegression + val lrParamMaps = new ParamGridBuilder() + .addGrid(lr.regParam, Array(0.001, 1000.0)) + .addGrid(lr.maxIter, Array(0, 10)) + .build() + val eval = new BinaryClassificationEvaluator + val cv = new CrossValidator() + .setEstimator(lr) + .setEstimatorParamMaps(lrParamMaps) + .setEvaluator(eval) + .setNumFolds(3) + .setFoldCol("fold1") + val err1 = intercept[IllegalArgumentException] { + cv.fit(datasetWithFold) + } + assert(err1.getMessage.contains("fold1 does not exist. Available: label, features, fold")) + + // Fold column must be integer type. + val foldCol = udf(() => 1L) + val datasetWithWrongFoldType = dataset.withColumn("fold1", foldCol()) + val err2 = intercept[IllegalArgumentException] { + cv.fit(datasetWithWrongFoldType) + } + assert(err2 + .getMessage + .contains("The specified `foldCol` column fold1 must be integer type, but got LongType.")) + } + test("cross validation with linear regression") { val dataset = sc.parallelize( LinearDataGenerator.generateLinearInput( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 665708a780c48..fb3bc9f798490 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -353,4 +353,34 @@ class MLUtilsSuite extends SparkFunSuite with MLlibTestSparkContext { convertMatrixColumnsFromML(df, "p._2") } } + + test("kFold with fold column") { + val data = sc.parallelize(1 to 100, 2).map(x => (x, if (x <= 50) 0 else 1)).toDF("i", "fold") + val collectedData = data.collect().map(_.getInt(0)).sorted + val twoFoldedRdd = kFold(data, 2, "fold") + assert(twoFoldedRdd(0)._1.collect().map(_.getInt(0)).sorted === + twoFoldedRdd(1)._2.collect().map(_.getInt(0)).sorted) + assert(twoFoldedRdd(0)._2.collect().map(_.getInt(0)).sorted === + twoFoldedRdd(1)._1.collect().map(_.getInt(0)).sorted) + + val result1 = twoFoldedRdd(0)._1.union(twoFoldedRdd(0)._2).collect().map(_.getInt(0)).sorted + assert(result1 === collectedData, + "Each training+validation set combined should contain all of the data.") + val result2 = twoFoldedRdd(1)._1.union(twoFoldedRdd(1)._2).collect().map(_.getInt(0)).sorted + assert(result2 === collectedData, + "Each training+validation set combined should contain all of the data.") + } + + test("kFold with fold column: invalid fold numbers") { + val data = sc.parallelize(Seq(0, 1, 2), 2).toDF( "fold") + val err1 = intercept[SparkException] { + kFold(data, 2, "fold")(0)._1.collect() + } + assert(err1.getMessage.contains("Fold number must be in range [0, 2), but got 2.")) + + val err2 = intercept[SparkException] { + kFold(data, 4, "fold")(0)._1.collect() + } + assert(err2.getMessage.contains("The validation data at fold 3 is empty.")) + } } diff --git a/python/pyspark/ml/tests/test_tuning.py b/python/pyspark/ml/tests/test_tuning.py index 6bcc3f93e1b3b..0aa5d47ca6bc4 100644 --- a/python/pyspark/ml/tests/test_tuning.py +++ b/python/pyspark/ml/tests/test_tuning.py @@ -380,6 +380,80 @@ def test_save_load_pipeline_estimator(self): original_nested_pipeline_model.stages): self.assertEqual(loadedStage.uid, originalStage.uid) + def test_user_specified_folds(self): + from pyspark.sql import functions as F + + dataset = self.spark.createDataFrame( + [(Vectors.dense([0.0]), 0.0), + (Vectors.dense([0.4]), 1.0), + (Vectors.dense([0.5]), 0.0), + (Vectors.dense([0.6]), 1.0), + (Vectors.dense([1.0]), 1.0)] * 10, + ["features", "label"]).repartition(2, "features") + + dataset_with_folds = dataset.repartition(1).withColumn("random", rand(100)) \ + .withColumn("fold", F.when(F.col("random") < 0.33, 0) + .when(F.col("random") < 0.66, 1) + .otherwise(2)).repartition(2, "features") + + lr = LogisticRegression() + grid = ParamGridBuilder().addGrid(lr.maxIter, [20]).build() + evaluator = BinaryClassificationEvaluator() + + cv = CrossValidator(estimator=lr, estimatorParamMaps=grid, evaluator=evaluator, numFolds=3) + cv_with_user_folds = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=3, + foldCol="fold") + + self.assertEqual(cv.getEstimator().uid, cv_with_user_folds.getEstimator().uid) + + cvModel1 = cv.fit(dataset) + cvModel2 = cv_with_user_folds.fit(dataset_with_folds) + for index in range(len(cvModel1.avgMetrics)): + print(abs(cvModel1.avgMetrics[index] - cvModel2.avgMetrics[index])) + self.assertTrue(abs(cvModel1.avgMetrics[index] - cvModel2.avgMetrics[index]) + < 0.1) + + # test save/load of CrossValidator + temp_path = tempfile.mkdtemp() + cvPath = temp_path + "/cv" + cv_with_user_folds.save(cvPath) + loadedCV = CrossValidator.load(cvPath) + self.assertEqual(loadedCV.getFoldCol(), cv_with_user_folds.getFoldCol()) + + def test_invalid_user_specified_folds(self): + from pyspark.sql import functions as F + + dataset_with_folds = self.spark.createDataFrame( + [(Vectors.dense([0.0]), 0.0, 0), + (Vectors.dense([0.4]), 1.0, 1), + (Vectors.dense([0.5]), 0.0, 2), + (Vectors.dense([0.6]), 1.0, 0), + (Vectors.dense([1.0]), 1.0, 1)] * 10, + ["features", "label", "fold"]) + + lr = LogisticRegression() + grid = ParamGridBuilder().addGrid(lr.maxIter, [20]).build() + evaluator = BinaryClassificationEvaluator() + + cv = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=2, + foldCol="fold") + with self.assertRaisesRegexp(Exception, "Fold number must be in range"): + cv.fit(dataset_with_folds) + + cv = CrossValidator(estimator=lr, + estimatorParamMaps=grid, + evaluator=evaluator, + numFolds=4, + foldCol="fold") + with self.assertRaisesRegexp(Exception, "The validation data at fold 3 is empty"): + cv.fit(dataset_with_folds) + class TrainValidationSplitTests(SparkSessionTestCase): diff --git a/python/pyspark/ml/tuning.py b/python/pyspark/ml/tuning.py index cb4542c5d025f..e00753b2ffc20 100644 --- a/python/pyspark/ml/tuning.py +++ b/python/pyspark/ml/tuning.py @@ -27,7 +27,8 @@ from pyspark.ml.param.shared import HasCollectSubModels, HasParallelism, HasSeed from pyspark.ml.util import * from pyspark.ml.wrapper import JavaParams -from pyspark.sql.functions import rand +from pyspark.sql.functions import col, lit, rand, UserDefinedFunction +from pyspark.sql.types import BooleanType __all__ = ['ParamGridBuilder', 'CrossValidator', 'CrossValidatorModel', 'TrainValidationSplit', 'TrainValidationSplitModel'] @@ -200,6 +201,12 @@ class _CrossValidatorParams(_ValidatorParams): numFolds = Param(Params._dummy(), "numFolds", "number of folds for cross validation", typeConverter=TypeConverters.toInt) + foldCol = Param(Params._dummy(), "foldCol", "Param for the column name of user " + + "specified fold number. Once this is specified, :py:class:`CrossValidator` " + + "won't do random k-fold split. Note that this column should be integer type " + + "with range [0, numFolds) and Spark will throw exception on out-of-range " + + "fold numbers.", typeConverter=TypeConverters.toString) + @since("1.4.0") def getNumFolds(self): """ @@ -207,6 +214,13 @@ def getNumFolds(self): """ return self.getOrDefault(self.numFolds) + @since("3.1.0") + def getFoldCol(self): + """ + Gets the value of foldCol or its default value. + """ + return self.getOrDefault(self.foldCol) + class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollectSubModels, MLReadable, MLWritable): @@ -255,23 +269,23 @@ class CrossValidator(Estimator, _CrossValidatorParams, HasParallelism, HasCollec @keyword_only def __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3, - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): """ __init__(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3,\ - seed=None, parallelism=1, collectSubModels=False) + seed=None, parallelism=1, collectSubModels=False, foldCol="") """ super(CrossValidator, self).__init__() - self._setDefault(numFolds=3, parallelism=1) + self._setDefault(numFolds=3, parallelism=1, foldCol="") kwargs = self._input_kwargs self._set(**kwargs) @keyword_only @since("1.4.0") def setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3, - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): """ setParams(self, estimator=None, estimatorParamMaps=None, evaluator=None, numFolds=3,\ - seed=None, parallelism=1, collectSubModels=False): + seed=None, parallelism=1, collectSubModels=False, foldCol=""): Sets params for cross validator. """ kwargs = self._input_kwargs @@ -305,6 +319,13 @@ def setNumFolds(self, value): """ return self._set(numFolds=value) + @since("3.1.0") + def setFoldCol(self, value): + """ + Sets the value of :py:attr:`foldCol`. + """ + return self._set(foldCol=value) + def setSeed(self, value): """ Sets the value of :py:attr:`seed`. @@ -329,10 +350,6 @@ def _fit(self, dataset): numModels = len(epm) eva = self.getOrDefault(self.evaluator) nFolds = self.getOrDefault(self.numFolds) - seed = self.getOrDefault(self.seed) - h = 1.0 / nFolds - randCol = self.uid + "_rand" - df = dataset.select("*", rand(seed).alias(randCol)) metrics = [0.0] * numModels pool = ThreadPool(processes=min(self.getParallelism(), numModels)) @@ -341,12 +358,10 @@ def _fit(self, dataset): if collectSubModelsParam: subModels = [[None for j in range(numModels)] for i in range(nFolds)] + datasets = self._kFold(dataset) for i in range(nFolds): - validateLB = i * h - validateUB = (i + 1) * h - condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) - validation = df.filter(condition).cache() - train = df.filter(~condition).cache() + validation = datasets[i][1].cache() + train = datasets[i][0].cache() tasks = _parallelFitTasks(est, train, eva, validation, epm, collectSubModelsParam) for j, metric, subModel in pool.imap_unordered(lambda f: f(), tasks): @@ -364,6 +379,45 @@ def _fit(self, dataset): bestModel = est.fit(dataset, epm[bestIndex]) return self._copyValues(CrossValidatorModel(bestModel, metrics, subModels)) + def _kFold(self, dataset): + nFolds = self.getOrDefault(self.numFolds) + foldCol = self.getOrDefault(self.foldCol) + + datasets = [] + if not foldCol: + # Do random k-fold split. + seed = self.getOrDefault(self.seed) + h = 1.0 / nFolds + randCol = self.uid + "_rand" + df = dataset.select("*", rand(seed).alias(randCol)) + for i in range(nFolds): + validateLB = i * h + validateUB = (i + 1) * h + condition = (df[randCol] >= validateLB) & (df[randCol] < validateUB) + validation = df.filter(condition) + train = df.filter(~condition) + datasets.append((train, validation)) + else: + # Use user-specified fold numbers. + def checker(foldNum): + if foldNum < 0 or foldNum >= nFolds: + raise ValueError( + "Fold number must be in range [0, %s), but got %s." % (nFolds, foldNum)) + return True + + checker_udf = UserDefinedFunction(checker, BooleanType()) + for i in range(nFolds): + training = dataset.filter(checker_udf(dataset[foldCol]) & (col(foldCol) != lit(i))) + validation = dataset.filter( + checker_udf(dataset[foldCol]) & (col(foldCol) == lit(i))) + if training.rdd.getNumPartitions() == 0 or len(training.take(1)) == 0: + raise ValueError("The training data at fold %s is empty." % i) + if validation.rdd.getNumPartitions() == 0 or len(validation.take(1)) == 0: + raise ValueError("The validation data at fold %s is empty." % i) + datasets.append((training, validation)) + + return datasets + @since("1.4.0") def copy(self, extra=None): """ @@ -407,10 +461,11 @@ def _from_java(cls, java_stage): seed = java_stage.getSeed() parallelism = java_stage.getParallelism() collectSubModels = java_stage.getCollectSubModels() + foldCol = java_stage.getFoldCol() # Create a new instance of this stage. py_stage = cls(estimator=estimator, estimatorParamMaps=epms, evaluator=evaluator, numFolds=numFolds, seed=seed, parallelism=parallelism, - collectSubModels=collectSubModels) + collectSubModels=collectSubModels, foldCol=foldCol) py_stage._resetUid(java_stage.uid()) return py_stage @@ -431,6 +486,7 @@ def _to_java(self): _java_obj.setNumFolds(self.getNumFolds()) _java_obj.setParallelism(self.getParallelism()) _java_obj.setCollectSubModels(self.getCollectSubModels()) + _java_obj.setFoldCol(self.getFoldCol()) return _java_obj From eeb81200e221decca6116a863cd50148e622cc75 Mon Sep 17 00:00:00 2001 From: Gabor Somogyi Date: Tue, 16 Jun 2020 18:22:12 -0700 Subject: [PATCH 22/28] [SPARK-31337][SQL] Support MS SQL Kerberos login in JDBC connector ### What changes were proposed in this pull request? When loading DataFrames from JDBC datasource with Kerberos authentication, remote executors (yarn-client/cluster etc. modes) fail to establish a connection due to lack of Kerberos ticket or ability to generate it. This is a real issue when trying to ingest data from kerberized data sources (SQL Server, Oracle) in enterprise environment where exposing simple authentication access is not an option due to IT policy issues. In this PR I've added MS SQL support. What this PR contains: * Added `MSSQLConnectionProvider` * Added `MSSQLConnectionProviderSuite` * Changed MS SQL JDBC driver to use the latest (test scope only) * Changed `MsSqlServerIntegrationSuite` docker image to use the latest * Added a version comment to `MariaDBConnectionProvider` to increase trackability ### Why are the changes needed? Missing JDBC kerberos support. ### Does this PR introduce _any_ user-facing change? Yes, now user is able to connect to MS SQL using kerberos. ### How was this patch tested? * Additional + existing unit tests * Existing integration tests * Test on cluster manually Closes #28635 from gaborgsomogyi/SPARK-31337. Authored-by: Gabor Somogyi Signed-off-by: Marcelo Vanzin --- external/docker-integration-tests/pom.xml | 1 - .../jdbc/MsSqlServerIntegrationSuite.scala | 2 +- pom.xml | 6 ++ sql/core/pom.xml | 5 + .../jdbc/connection/ConnectionProvider.scala | 4 + .../connection/MSSQLConnectionProvider.scala | 97 +++++++++++++++++++ .../MariaDBConnectionProvider.scala | 2 +- .../MSSQLConnectionProviderSuite.scala | 51 ++++++++++ 8 files changed, 165 insertions(+), 3 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 7f9e92f585168..298e3d36c145a 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -166,7 +166,6 @@ com.microsoft.sqlserver mssql-jdbc - 7.2.1.jre8 test diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala index 42d64873c44d9..6c633af1fde84 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "mcr.microsoft.com/mssql/server:2017-GA-ubuntu" + override val imageName = "mcr.microsoft.com/mssql/server:2019-GA-ubuntu-16.04" override val env = Map( "SA_PASSWORD" -> "Sapass123", "ACCEPT_EULA" -> "Y" diff --git a/pom.xml b/pom.xml index 8d552e08f0cd9..3abbf871ad2de 100644 --- a/pom.xml +++ b/pom.xml @@ -978,6 +978,12 @@ 11.5.0.0 test + + com.microsoft.sqlserver + mssql-jdbc + 8.2.2.jre8 + test + org.apache.curator curator-recipes diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e4ef1467a960e..0855fa13fa79a 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -145,6 +145,11 @@ jcc test + + com.microsoft.sqlserver + mssql-jdbc + test + org.apache.parquet parquet-avro diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala index 73e73e59be574..6c310ced37883 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/ConnectionProvider.scala @@ -60,6 +60,10 @@ private[jdbc] object ConnectionProvider extends Logging { logDebug("DB2 connection provider found") new DB2ConnectionProvider(driver, options) + case MSSQLConnectionProvider.driverClass => + logDebug("MS SQL connection provider found") + new MSSQLConnectionProvider(driver, options) + case _ => throw new IllegalArgumentException(s"Driver ${options.driverClass} does not support " + "Kerberos authentication") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala new file mode 100644 index 0000000000000..2950aa9b4db94 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProvider.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +import java.security.PrivilegedExceptionAction +import java.sql.{Connection, Driver} +import java.util.Properties + +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions + +private[sql] class MSSQLConnectionProvider( + driver: Driver, + options: JDBCOptions, + parserMethod: String = "parseAndMergeProperties" + ) extends SecureConnectionProvider(driver, options) { + override val appEntry: String = { + val configName = "jaasConfigurationName" + val appEntryDefault = "SQLJDBCDriver" + + val parseURL = try { + // The default parser method signature is the following: + // private Properties parseAndMergeProperties(String Url, Properties suppliedProperties) + val m = driver.getClass.getDeclaredMethod(parserMethod, classOf[String], classOf[Properties]) + m.setAccessible(true) + Some(m) + } catch { + case _: NoSuchMethodException => None + } + + parseURL match { + case Some(m) => + logDebug("Property parser method found, using it") + m.invoke(driver, options.url, null).asInstanceOf[Properties] + .getProperty(configName, appEntryDefault) + + case None => + logDebug("Property parser method not found, using custom parsing mechanism") + options.url.split(';').map(_.split('=')) + .find(kv => kv.length == 2 && kv(0) == configName) + .getOrElse(Array(configName, appEntryDefault))(1) + } + } + + override def getConnection(): Connection = { + setAuthenticationConfigIfNeeded() + UserGroupInformation.loginUserFromKeytabAndReturnUGI(options.principal, options.keytab).doAs( + new PrivilegedExceptionAction[Connection]() { + override def run(): Connection = { + MSSQLConnectionProvider.super.getConnection() + } + } + ) + } + + override def getAdditionalProperties(): Properties = { + val result = new Properties() + // These props needed to reach internal kerberos authentication in the JDBC driver + result.put("integratedSecurity", "true") + result.put("authenticationScheme", "JavaKerberos") + result + } + + override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { + val (parent, configEntry) = getConfigWithAppEntry() + /** + * Couple of things to mention here (v8.2.2 client): + * 1. MS SQL supports JAAS application name configuration + * 2. MS SQL sets a default JAAS config if "java.security.auth.login.config" is not set + */ + val entryUsesKeytab = configEntry != null && + configEntry.exists(_.getOptions().get("useKeyTab") == "true") + if (configEntry == null || configEntry.isEmpty || !entryUsesKeytab) { + setAuthenticationConfig(parent) + } + } +} + +private[sql] object MSSQLConnectionProvider { + val driverClass = "com.microsoft.sqlserver.jdbc.SQLServerDriver" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala index 8e3381077cbbf..589f13cf6ad5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MariaDBConnectionProvider.scala @@ -30,7 +30,7 @@ private[jdbc] class MariaDBConnectionProvider(driver: Driver, options: JDBCOptio override def setAuthenticationConfigIfNeeded(): Unit = SecurityConfigurationLock.synchronized { val (parent, configEntry) = getConfigWithAppEntry() /** - * Couple of things to mention here: + * Couple of things to mention here (v2.5.4 client): * 1. MariaDB doesn't support JAAS application name configuration * 2. MariaDB sets a default JAAS config if "java.security.auth.login.config" is not set */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala new file mode 100644 index 0000000000000..249f1e36347ed --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/jdbc/connection/MSSQLConnectionProviderSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.jdbc.connection + +class MSSQLConnectionProviderSuite extends ConnectionProviderSuiteBase { + test("setAuthenticationConfigIfNeeded default parser must set authentication if not set") { + val driver = registerDriver(MSSQLConnectionProvider.driverClass) + val defaultProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql")) + val customProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql")) + + testProviders(defaultProvider, customProvider) + } + + test("setAuthenticationConfigIfNeeded custom parser must set authentication if not set") { + val parserMethod = "IntentionallyNotExistingMethod" + val driver = registerDriver(MSSQLConnectionProvider.driverClass) + val defaultProvider = new MSSQLConnectionProvider( + driver, options("jdbc:sqlserver://localhost/mssql"), parserMethod) + val customProvider = new MSSQLConnectionProvider( + driver, + options("jdbc:sqlserver://localhost/mssql;jaasConfigurationName=custommssql"), + parserMethod) + + testProviders(defaultProvider, customProvider) + } + + private def testProviders( + defaultProvider: SecureConnectionProvider, + customProvider: SecureConnectionProvider) = { + assert(defaultProvider.appEntry !== customProvider.appEntry) + testSecureConnectionProvider(defaultProvider) + testSecureConnectionProvider(customProvider) + } +} From afd8a8b964d0d1f46f531571613d2c68fe631d25 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 17 Jun 2020 12:07:36 +0900 Subject: [PATCH 23/28] [SPARK-31989][SQL] Generate JSON rebasing files w/ 30 minutes step ### What changes were proposed in this pull request? 1. Change the max step from 1 week to 30 minutes in the tests `RebaseDateTimeSuite`.`generate 'gregorian-julian-rebase-micros.json'` and `generate 'julian-gregorian-rebase-micros.json'`. 2. Parallelise JSON files generation in the function `generateRebaseJson` by using `ThreadUtils.parmap`. ### Why are the changes needed? 1. To prevent the bugs that are fixed by https://github.com/apache/spark/pull/28787 and https://github.com/apache/spark/pull/28816. 2. The parallelisation speeds up JSON file generation. ### Does this PR introduce _any_ user-facing change? Yes ### How was this patch tested? By generating the JSON file `julian-gregorian-rebase-micros.json`. Closes #28827 from MaxGekk/rebase-30-min. Authored-by: Max Gekk Signed-off-by: HyukjinKwon --- .../sql/catalyst/util/RebaseDateTimeSuite.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala index 254bf01c89b4f..9b43635a70701 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/RebaseDateTimeSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.catalyst.util.RebaseDateTime._ +import org.apache.spark.util.ThreadUtils class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { @@ -254,11 +255,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { import com.fasterxml.jackson.module.scala.{DefaultScalaModule, ScalaObjectMapper} case class RebaseRecord(tz: String, switches: Array[Long], diffs: Array[Long]) - - val result = new ArrayBuffer[RebaseRecord]() - ALL_TIMEZONES - .sortBy(_.getId) - .foreach { zid => + val rebaseRecords = ThreadUtils.parmap(ALL_TIMEZONES, "JSON-rebase-gen", 16) { zid => withDefaultTimeZone(zid) { val tz = TimeZone.getTimeZone(zid) val start = adjustFunc( @@ -272,7 +269,7 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { var micros = start var diff = Long.MaxValue - val maxStep = DAYS_PER_WEEK * MICROS_PER_DAY + val maxStep = 30 * MICROS_PER_MINUTE var step: Long = MICROS_PER_SECOND val switches = new ArrayBuffer[Long]() val diffs = new ArrayBuffer[Long]() @@ -294,9 +291,11 @@ class RebaseDateTimeSuite extends SparkFunSuite with Matchers with SQLHelper { } micros += step } - result.append(RebaseRecord(zid.getId, switches.toArray, diffs.toArray)) + RebaseRecord(zid.getId, switches.toArray, diffs.toArray) } } + val result = new ArrayBuffer[RebaseRecord]() + rebaseRecords.sortBy(_.tz).foreach(result.append(_)) val mapper = (new ObjectMapper() with ScalaObjectMapper) .registerModule(DefaultScalaModule) .writerWithDefaultPrettyPrinter() From feeca63198466640ac461a2a34922493fa6162a8 Mon Sep 17 00:00:00 2001 From: HyukjinKwon Date: Wed, 17 Jun 2020 12:10:12 +0900 Subject: [PATCH 24/28] [SPARK-32011][PYTHON][CORE] Remove warnings about pin-thread modes and guide to use collectWithJobGroup ### What changes were proposed in this pull request? This PR proposes to remove the warning about multi-thread in local properties, and change the guide to use `collectWithJobGroup` for multi-threads for now because: - It is too noisy to users who don't use multiple threads - the number of this single thread case is arguably more prevailing. - There was a critical issue found about pin-thread mode SPARK-32010, which will be fixed in Spark 3.1. - To smoothly migrate, `RDD.collectWithJobGroup` was added, which will be deprecated in Spark 3.1 with SPARK-32010 fixed. I will target to deprecate `RDD.collectWithJobGroup`, and make this pin-thread mode stable in Spark 3.1. In the future releases, I plan to make this mode as a default mode, and remove `RDD.collectWithJobGroup` away. ### Why are the changes needed? To avoid guiding users a feature with a critical issue, and provide a proper workaround for now. ### Does this PR introduce _any_ user-facing change? Yes, warning message and documentation. ### How was this patch tested? Manually tested: Before: ``` >>> spark.sparkContext.setLocalProperty("a", "b") /.../spark/python/pyspark/util.py:141: UserWarning: Currently, 'setLocalProperty' (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties for each thread on PVM. To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). However, note that it cannot inherit the local properties from the parent thread although it isolates each thread on PVM and JVM with its own local properties. To work around this, you should manually copy and set the local properties from the parent thread to the child thread when you create another thread. ``` After: ``` >>> spark.sparkContext.setLocalProperty("a", "b") ``` Closes #28845 from HyukjinKwon/SPARK-32011. Authored-by: HyukjinKwon Signed-off-by: HyukjinKwon --- python/pyspark/context.py | 37 ++++++------------------------------- python/pyspark/util.py | 27 --------------------------- 2 files changed, 6 insertions(+), 58 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 96353bb9228d5..32d69edb171db 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -41,7 +41,6 @@ from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler -from pyspark.util import _warn_pin_thread if sys.version > '3': xrange = range @@ -1026,17 +1025,9 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): .. note:: Currently, setting a group ID (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local - properties for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + properties for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup` for now. """ - _warn_pin_thread("setJobGroup") self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): @@ -1047,17 +1038,9 @@ def setLocalProperty(self, key, value): .. note:: Currently, setting a local property with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate local properties - for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup`. """ - _warn_pin_thread("setLocalProperty") self._jsc.setLocalProperty(key, value) def getLocalProperty(self, key): @@ -1074,17 +1057,9 @@ def setJobDescription(self, value): .. note:: Currently, setting a job description (set to local properties) with multiple threads does not properly work. Internally threads on PVM and JVM are not synced, and JVM thread can be reused for multiple threads on PVM, which fails to isolate - local properties for each thread on PVM. - - To work around this, you can set `PYSPARK_PIN_THREAD` to - `'true'` (see SPARK-22340). However, note that it cannot inherit the local properties - from the parent thread although it isolates each thread on PVM and JVM with its own - local properties. - - To work around this, you should manually copy and set the local - properties from the parent thread to the child thread when you create another thread. + local properties for each thread on PVM. To work around this, You can use + :meth:`RDD.collectWithJobGroup` for now. """ - _warn_pin_thread("setJobDescription") self._jsc.setJobDescription(value) def sparkUser(self): diff --git a/python/pyspark/util.py b/python/pyspark/util.py index cc614a2c0eaed..d9429372a6bfc 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -114,33 +114,6 @@ def wrapper(*args, **kwargs): return wrapper -def _warn_pin_thread(name): - if os.environ.get("PYSPARK_PIN_THREAD", "false").lower() == "true": - msg = ( - "PYSPARK_PIN_THREAD feature is enabled. " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread.") - else: - msg = ( - "Currently, '%s' (set to local properties) with multiple threads does " - "not properly work. " - "\n" - "Internally threads on PVM and JVM are not synced, and JVM thread can be reused " - "for multiple threads on PVM, which fails to isolate local properties for each " - "thread on PVM. " - "\n" - "To work around this, you can set PYSPARK_PIN_THREAD to true (see SPARK-22340). " - "However, note that it cannot inherit the local properties from the parent thread " - "although it isolates each thread on PVM and JVM with its own local properties. " - "\n" - "To work around this, you should manually copy and set the local properties from " - "the parent thread to the child thread when you create another thread." % name) - warnings.warn(msg, UserWarning) - - def _print_missing_jar(lib_name, pkg_name, jar_name, spark_version): print(""" ________________________________________________________________________________________________ From 93bb70f3fe65b567cd55cb3b8fb2a748e2a54b24 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 16 Jun 2020 21:05:29 -0700 Subject: [PATCH 25/28] [SPARK-29148][CORE][FOLLOWUP] Fix warning message to show a correct executor id ### What changes were proposed in this pull request? This aims to replace `executorIdsToBeRemoved` with `executorIdToBeRemoved`. ### Why are the changes needed? Since a wrong variable is used currently, `ArrayBuffer()` is always displayed. ``` 20/06/16 19:33:31 WARN ExecutorAllocationManager: Not removing executor ArrayBuffer() because the ResourceProfile was UNKNOWN! ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manual. Closes #28847 from dongjoon-hyun/SPARK-29148. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/ExecutorAllocationManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 4df58354c99ca..e0ac2b3e0f4b8 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -518,7 +518,7 @@ private[spark] class ExecutorAllocationManager( if (testing) { throw new SparkException("ResourceProfile Id was UNKNOWN, this is not expected") } - logWarning(s"Not removing executor $executorIdsToBeRemoved because the " + + logWarning(s"Not removing executor $executorIdToBeRemoved because the " + "ResourceProfile was UNKNOWN!") } else { // get the running total as we remove or initialize it to the count - pendingRemoval From 350aa859fe1273fa96effd1c96afdfaca2c1b0d5 Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Wed, 17 Jun 2020 06:28:47 +0000 Subject: [PATCH 26/28] [SPARK-32006][SQL] Create date/timestamp formatters once before collect in `hiveResultString()` ### What changes were proposed in this pull request? 1. Add method `getTimeFormatters` to `HiveResult` which creates timestamp and date formatters. 2. Move creation of `dateFormatter` and `timestampFormatter` from the constructor of the `HiveResult` object to `HiveResult. hiveResultString()` via `getTimeFormatters`. This allows to resolve time zone ID from Spark's session time zone `spark.sql.session.timeZone` and create date/timestamp formatters only once before collecting `java.sql.Timestamp`/`java.sql.Date` values. 3. Create date/timestamp formatters once in SparkExecuteStatementOperation. ### Why are the changes needed? To fix perf regression comparing to Spark 2.4 ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - By existing test suite `HiveResultSuite` and etc. - Re-generate benchmarks results of `DateTimeBenchmark` in the environment: | Item | Description | | ---- | ----| | Region | us-west-2 (Oregon) | | Instance | r3.xlarge | | AMI | ubuntu/images/hvm-ssd/ubuntu-bionic-18.04-amd64-server-20190722.1 (ami-06f2f779464715dc5) | | Java | OpenJDK 64-Bit Server VM 1.8.0_252 and OpenJDK 64-Bit Server VM 11.0.7+10 | Closes #28842 from MaxGekk/opt-toHiveString-oss-master. Authored-by: Max Gekk Signed-off-by: Wenchen Fan --- .../DateTimeBenchmark-jdk11-results.txt | 262 +++++++++--------- .../benchmarks/DateTimeBenchmark-results.txt | 260 ++++++++--------- .../spark/sql/execution/HiveResult.scala | 59 ++-- .../spark/sql/execution/HiveResultSuite.scala | 19 +- .../SparkExecuteStatementOperation.scala | 22 +- .../ThriftServerQueryTestSuite.scala | 17 +- 6 files changed, 325 insertions(+), 314 deletions(-) diff --git a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt index 70d888227141d..d84dccbf6c266 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-jdk11-results.txt @@ -6,18 +6,18 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1660 1745 120 6.0 166.0 1.0X -date + interval(m, d) 1672 1685 19 6.0 167.2 1.0X -date + interval(m, d, ms) 6462 6481 27 1.5 646.2 0.3X -date - interval(m) 1456 1480 35 6.9 145.6 1.1X -date - interval(m, d) 1501 1509 11 6.7 150.1 1.1X -date - interval(m, d, ms) 6457 6466 12 1.5 645.7 0.3X -timestamp + interval(m) 2941 2944 4 3.4 294.1 0.6X -timestamp + interval(m, d) 3008 3012 6 3.3 300.8 0.6X -timestamp + interval(m, d, ms) 3329 3333 6 3.0 332.9 0.5X -timestamp - interval(m) 2964 2982 26 3.4 296.4 0.6X -timestamp - interval(m, d) 3030 3039 13 3.3 303.0 0.5X -timestamp - interval(m, d, ms) 3312 3313 1 3.0 331.2 0.5X +date + interval(m) 1550 1609 83 6.5 155.0 1.0X +date + interval(m, d) 1572 1575 5 6.4 157.2 1.0X +date + interval(m, d, ms) 6512 6512 0 1.5 651.2 0.2X +date - interval(m) 1469 1489 28 6.8 146.9 1.1X +date - interval(m, d) 1558 1572 19 6.4 155.8 1.0X +date - interval(m, d, ms) 6602 6605 4 1.5 660.2 0.2X +timestamp + interval(m) 2945 2961 23 3.4 294.5 0.5X +timestamp + interval(m, d) 3075 3083 12 3.3 307.5 0.5X +timestamp + interval(m, d, ms) 3421 3430 13 2.9 342.1 0.5X +timestamp - interval(m) 3050 3061 17 3.3 305.0 0.5X +timestamp - interval(m, d) 3195 3201 8 3.1 319.5 0.5X +timestamp - interval(m, d, ms) 3442 3450 11 2.9 344.2 0.5X ================================================================================================ @@ -28,92 +28,92 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 333 334 0 30.0 33.3 1.0X -cast to timestamp wholestage on 349 368 12 28.6 34.9 1.0X +cast to timestamp wholestage off 320 326 8 31.2 32.0 1.0X +cast to timestamp wholestage on 289 297 5 34.6 28.9 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1229 1229 1 8.1 122.9 1.0X -year of timestamp wholestage on 1218 1223 5 8.2 121.8 1.0X +year of timestamp wholestage off 1266 1266 1 7.9 126.6 1.0X +year of timestamp wholestage on 1233 1253 15 8.1 123.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1593 1594 2 6.3 159.3 1.0X -quarter of timestamp wholestage on 1515 1529 14 6.6 151.5 1.1X +quarter of timestamp wholestage off 1594 1600 8 6.3 159.4 1.0X +quarter of timestamp wholestage on 1529 1532 3 6.5 152.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1222 1246 34 8.2 122.2 1.0X -month of timestamp wholestage on 1207 1232 31 8.3 120.7 1.0X +month of timestamp wholestage off 1239 1257 25 8.1 123.9 1.0X +month of timestamp wholestage on 1235 1243 5 8.1 123.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 2453 2455 2 4.1 245.3 1.0X -weekofyear of timestamp wholestage on 2357 2380 22 4.2 235.7 1.0X +weekofyear of timestamp wholestage off 2209 2216 9 4.5 220.9 1.0X +weekofyear of timestamp wholestage on 1831 1838 9 5.5 183.1 1.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1216 1219 5 8.2 121.6 1.0X -day of timestamp wholestage on 1205 1221 25 8.3 120.5 1.0X +day of timestamp wholestage off 1238 1238 0 8.1 123.8 1.0X +day of timestamp wholestage on 1223 1235 12 8.2 122.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1268 1274 9 7.9 126.8 1.0X -dayofyear of timestamp wholestage on 1253 1268 10 8.0 125.3 1.0X +dayofyear of timestamp wholestage off 1302 1304 3 7.7 130.2 1.0X +dayofyear of timestamp wholestage on 1269 1276 6 7.9 126.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1223 1224 1 8.2 122.3 1.0X -dayofmonth of timestamp wholestage on 1231 1246 14 8.1 123.1 1.0X +dayofmonth of timestamp wholestage off 1251 1253 3 8.0 125.1 1.0X +dayofmonth of timestamp wholestage on 1225 1232 9 8.2 122.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1398 1406 12 7.2 139.8 1.0X -dayofweek of timestamp wholestage on 1387 1399 15 7.2 138.7 1.0X +dayofweek of timestamp wholestage off 1424 1424 1 7.0 142.4 1.0X +dayofweek of timestamp wholestage on 1385 1389 4 7.2 138.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1327 1333 9 7.5 132.7 1.0X -weekday of timestamp wholestage on 1329 1333 4 7.5 132.9 1.0X +weekday of timestamp wholestage off 1366 1366 0 7.3 136.6 1.0X +weekday of timestamp wholestage on 1320 1325 5 7.6 132.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 1005 1016 15 9.9 100.5 1.0X -hour of timestamp wholestage on 934 940 4 10.7 93.4 1.1X +hour of timestamp wholestage off 985 986 1 10.2 98.5 1.0X +hour of timestamp wholestage on 974 981 10 10.3 97.4 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 1003 1009 8 10.0 100.3 1.0X -minute of timestamp wholestage on 934 938 7 10.7 93.4 1.1X +minute of timestamp wholestage off 1044 1047 5 9.6 104.4 1.0X +minute of timestamp wholestage on 984 994 17 10.2 98.4 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 997 998 2 10.0 99.7 1.0X -second of timestamp wholestage on 925 935 8 10.8 92.5 1.1X +second of timestamp wholestage off 999 1003 6 10.0 99.9 1.0X +second of timestamp wholestage on 961 974 8 10.4 96.1 1.0X ================================================================================================ @@ -124,15 +124,15 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 297 297 0 33.7 29.7 1.0X -current_date wholestage on 280 282 2 35.7 28.0 1.1X +current_date wholestage off 297 302 7 33.6 29.7 1.0X +current_date wholestage on 270 283 22 37.1 27.0 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 307 337 43 32.6 30.7 1.0X -current_timestamp wholestage on 260 284 29 38.4 26.0 1.2X +current_timestamp wholestage off 302 310 11 33.1 30.2 1.0X +current_timestamp wholestage on 264 351 98 37.9 26.4 1.1X ================================================================================================ @@ -143,43 +143,43 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1066 1073 10 9.4 106.6 1.0X -cast to date wholestage on 997 1003 6 10.0 99.7 1.1X +cast to date wholestage off 1083 1083 1 9.2 108.3 1.0X +cast to date wholestage on 1040 1044 5 9.6 104.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1238 1242 6 8.1 123.8 1.0X -last_day wholestage on 1259 1272 12 7.9 125.9 1.0X +last_day wholestage off 1258 1258 0 7.9 125.8 1.0X +last_day wholestage on 1244 1254 8 8.0 124.4 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1116 1138 32 9.0 111.6 1.0X -next_day wholestage on 1052 1063 11 9.5 105.2 1.1X +next_day wholestage off 1133 1135 3 8.8 113.3 1.0X +next_day wholestage on 1093 1100 7 9.1 109.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1048 1049 1 9.5 104.8 1.0X -date_add wholestage on 1035 1039 3 9.7 103.5 1.0X +date_add wholestage off 1065 1074 14 9.4 106.5 1.0X +date_add wholestage on 1044 1053 6 9.6 104.4 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1119 1127 11 8.9 111.9 1.0X -date_sub wholestage on 1028 1039 7 9.7 102.8 1.1X +date_sub wholestage off 1069 1076 9 9.4 106.9 1.0X +date_sub wholestage on 1047 1052 8 9.6 104.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1421 1421 0 7.0 142.1 1.0X -add_months wholestage on 1423 1434 11 7.0 142.3 1.0X +add_months wholestage off 1417 1430 18 7.1 141.7 1.0X +add_months wholestage on 1439 1445 5 6.9 143.9 1.0X ================================================================================================ @@ -190,8 +190,8 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5293 5296 5 1.9 529.3 1.0X -format date wholestage on 5143 5157 19 1.9 514.3 1.0X +format date wholestage off 5228 5232 6 1.9 522.8 1.0X +format date wholestage on 5172 5193 17 1.9 517.2 1.0X ================================================================================================ @@ -202,8 +202,8 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 7136 7136 1 1.4 713.6 1.0X -from_unixtime wholestage on 7049 7068 29 1.4 704.9 1.0X +from_unixtime wholestage off 6941 6952 16 1.4 694.1 1.0X +from_unixtime wholestage on 6898 6926 32 1.4 689.8 1.0X ================================================================================================ @@ -214,15 +214,15 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 1325 1329 6 7.5 132.5 1.0X -from_utc_timestamp wholestage on 1269 1273 4 7.9 126.9 1.0X +from_utc_timestamp wholestage off 1339 1342 5 7.5 133.9 1.0X +from_utc_timestamp wholestage on 1285 1292 5 7.8 128.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1684 1691 10 5.9 168.4 1.0X -to_utc_timestamp wholestage on 1641 1648 9 6.1 164.1 1.0X +to_utc_timestamp wholestage off 1697 1717 29 5.9 169.7 1.0X +to_utc_timestamp wholestage on 1656 1665 13 6.0 165.6 1.0X ================================================================================================ @@ -233,29 +233,29 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 343 346 4 29.1 34.3 1.0X -cast interval wholestage on 281 282 1 35.6 28.1 1.2X +cast interval wholestage off 333 344 16 30.1 33.3 1.0X +cast interval wholestage on 288 290 2 34.7 28.8 1.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1831 1840 13 5.5 183.1 1.0X -datediff wholestage on 1759 1769 15 5.7 175.9 1.0X +datediff wholestage off 1857 1860 4 5.4 185.7 1.0X +datediff wholestage on 1795 1808 10 5.6 179.5 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 5729 5747 25 1.7 572.9 1.0X -months_between wholestage on 5710 5720 9 1.8 571.0 1.0X +months_between wholestage off 5826 5834 11 1.7 582.6 1.0X +months_between wholestage on 5737 5763 18 1.7 573.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 2183 2189 9 0.5 2182.6 1.0X -window wholestage on 46835 46944 88 0.0 46834.8 0.0X +window wholestage off 2220 2246 36 0.5 2220.4 1.0X +window wholestage on 46696 46794 89 0.0 46696.1 0.0X ================================================================================================ @@ -266,134 +266,134 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 2668 2672 5 3.7 266.8 1.0X -date_trunc YEAR wholestage on 2719 2731 9 3.7 271.9 1.0X +date_trunc YEAR wholestage off 2658 2659 1 3.8 265.8 1.0X +date_trunc YEAR wholestage on 2691 2700 8 3.7 269.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 2672 2677 8 3.7 267.2 1.0X -date_trunc YYYY wholestage on 2710 2726 12 3.7 271.0 1.0X +date_trunc YYYY wholestage off 2671 2679 11 3.7 267.1 1.0X +date_trunc YYYY wholestage on 2700 2706 6 3.7 270.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 2670 2673 4 3.7 267.0 1.0X -date_trunc YY wholestage on 2711 2720 7 3.7 271.1 1.0X +date_trunc YY wholestage off 2674 2689 20 3.7 267.4 1.0X +date_trunc YY wholestage on 2697 2716 17 3.7 269.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 2674 2674 0 3.7 267.4 1.0X -date_trunc MON wholestage on 2667 2677 10 3.7 266.7 1.0X +date_trunc MON wholestage off 2695 2700 7 3.7 269.5 1.0X +date_trunc MON wholestage on 2711 2722 11 3.7 271.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 2675 2686 16 3.7 267.5 1.0X -date_trunc MONTH wholestage on 2667 2674 6 3.7 266.7 1.0X +date_trunc MONTH wholestage off 2682 2685 4 3.7 268.2 1.0X +date_trunc MONTH wholestage on 2709 2727 15 3.7 270.9 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 2673 2674 1 3.7 267.3 1.0X -date_trunc MM wholestage on 2664 2669 4 3.8 266.4 1.0X +date_trunc MM wholestage off 2683 2693 14 3.7 268.3 1.0X +date_trunc MM wholestage on 2706 2722 16 3.7 270.6 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 2281 2288 10 4.4 228.1 1.0X -date_trunc DAY wholestage on 2302 2312 8 4.3 230.2 1.0X +date_trunc DAY wholestage off 2292 2299 10 4.4 229.2 1.0X +date_trunc DAY wholestage on 2290 2311 14 4.4 229.0 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 2281 2283 3 4.4 228.1 1.0X -date_trunc DD wholestage on 2291 2302 11 4.4 229.1 1.0X +date_trunc DD wholestage off 2302 2309 9 4.3 230.2 1.0X +date_trunc DD wholestage on 2282 2292 6 4.4 228.2 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 2331 2332 1 4.3 233.1 1.0X -date_trunc HOUR wholestage on 2290 2304 11 4.4 229.0 1.0X +date_trunc HOUR wholestage off 2288 2288 0 4.4 228.8 1.0X +date_trunc HOUR wholestage on 2277 2290 14 4.4 227.7 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 379 385 9 26.4 37.9 1.0X -date_trunc MINUTE wholestage on 371 376 5 27.0 37.1 1.0X +date_trunc MINUTE wholestage off 400 419 26 25.0 40.0 1.0X +date_trunc MINUTE wholestage on 401 405 4 24.9 40.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 375 376 1 26.7 37.5 1.0X -date_trunc SECOND wholestage on 370 376 8 27.0 37.0 1.0X +date_trunc SECOND wholestage off 408 414 9 24.5 40.8 1.0X +date_trunc SECOND wholestage on 408 413 8 24.5 40.8 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 2597 2604 10 3.9 259.7 1.0X -date_trunc WEEK wholestage on 2591 2605 13 3.9 259.1 1.0X +date_trunc WEEK wholestage off 2623 2631 12 3.8 262.3 1.0X +date_trunc WEEK wholestage on 2613 2621 8 3.8 261.3 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 3501 3511 14 2.9 350.1 1.0X -date_trunc QUARTER wholestage on 3477 3489 9 2.9 347.7 1.0X +date_trunc QUARTER wholestage off 3518 3520 3 2.8 351.8 1.0X +date_trunc QUARTER wholestage on 3501 3510 11 2.9 350.1 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 332 334 3 30.1 33.2 1.0X -trunc year wholestage on 332 346 17 30.1 33.2 1.0X +trunc year wholestage off 315 333 26 31.8 31.5 1.0X +trunc year wholestage on 352 360 7 28.4 35.2 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 331 331 0 30.2 33.1 1.0X -trunc yyyy wholestage on 336 339 4 29.8 33.6 1.0X +trunc yyyy wholestage off 321 321 1 31.2 32.1 1.0X +trunc yyyy wholestage on 354 358 5 28.3 35.4 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 330 342 17 30.3 33.0 1.0X -trunc yy wholestage on 333 337 3 30.0 33.3 1.0X +trunc yy wholestage off 312 313 1 32.0 31.2 1.0X +trunc yy wholestage on 355 360 5 28.2 35.5 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 334 335 1 30.0 33.4 1.0X -trunc mon wholestage on 333 347 9 30.0 33.3 1.0X +trunc mon wholestage off 324 327 4 30.9 32.4 1.0X +trunc mon wholestage on 355 357 2 28.2 35.5 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 332 333 1 30.1 33.2 1.0X -trunc month wholestage on 333 340 7 30.0 33.3 1.0X +trunc month wholestage off 313 318 8 32.0 31.3 1.0X +trunc month wholestage on 354 358 5 28.3 35.4 0.9X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 328 336 11 30.5 32.8 1.0X -trunc mm wholestage on 333 343 11 30.0 33.3 1.0X +trunc mm wholestage off 314 325 15 31.8 31.4 1.0X +trunc mm wholestage on 353 366 17 28.4 35.3 0.9X ================================================================================================ @@ -404,36 +404,36 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 170 171 1 5.9 170.1 1.0X -to timestamp str wholestage on 172 174 2 5.8 171.6 1.0X +to timestamp str wholestage off 168 169 0 5.9 168.4 1.0X +to timestamp str wholestage on 168 173 7 6.0 167.6 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1437 1439 3 0.7 1437.0 1.0X -to_timestamp wholestage on 1288 1292 5 0.8 1288.1 1.1X +to_timestamp wholestage off 1390 1390 0 0.7 1389.8 1.0X +to_timestamp wholestage on 1204 1215 11 0.8 1204.2 1.2X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1352 1353 2 0.7 1352.0 1.0X -to_unix_timestamp wholestage on 1314 1319 5 0.8 1314.4 1.0X +to_unix_timestamp wholestage off 1277 1281 4 0.8 1277.5 1.0X +to_unix_timestamp wholestage on 1203 1213 11 0.8 1202.6 1.1X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 211 215 6 4.7 210.7 1.0X -to date str wholestage on 217 217 1 4.6 216.5 1.0X +to date str wholestage off 218 219 1 4.6 218.2 1.0X +to date str wholestage on 211 214 5 4.7 210.8 1.0X OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 3281 3295 20 0.3 3280.9 1.0X -to_date wholestage on 3223 3239 17 0.3 3222.8 1.0X +to_date wholestage off 3016 3041 35 0.3 3016.1 1.0X +to_date wholestage on 3015 3023 9 0.3 3014.6 1.0X ================================================================================================ @@ -444,18 +444,18 @@ OpenJDK 64-Bit Server VM 11.0.7+10-post-Ubuntu-2ubuntu218.04 on Linux 4.15.0-106 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 446 447 1 11.2 89.1 1.0X -From java.time.LocalDate 354 356 1 14.1 70.8 1.3X -Collect java.sql.Date 2722 3091 495 1.8 544.4 0.2X -Collect java.time.LocalDate 1786 1836 60 2.8 357.2 0.2X -From java.sql.Timestamp 275 287 19 18.2 55.0 1.6X -From java.time.Instant 325 328 3 15.4 65.0 1.4X -Collect longs 1300 1321 25 3.8 260.0 0.3X -Collect java.sql.Timestamp 1450 1557 102 3.4 290.0 0.3X -Collect java.time.Instant 1499 1599 87 3.3 299.9 0.3X -java.sql.Date to Hive string 17536 18367 1059 0.3 3507.2 0.0X -java.time.LocalDate to Hive string 12089 12897 725 0.4 2417.8 0.0X -java.sql.Timestamp to Hive string 48014 48625 752 0.1 9602.9 0.0X -java.time.Instant to Hive string 37346 37445 93 0.1 7469.1 0.0X +From java.sql.Date 430 442 18 11.6 86.0 1.0X +From java.time.LocalDate 351 354 3 14.3 70.2 1.2X +Collect java.sql.Date 2095 2853 733 2.4 418.9 0.2X +Collect java.time.LocalDate 1691 1910 209 3.0 338.3 0.3X +From java.sql.Timestamp 276 280 4 18.1 55.2 1.6X +From java.time.Instant 324 328 4 15.4 64.8 1.3X +Collect longs 1348 1450 126 3.7 269.5 0.3X +Collect java.sql.Timestamp 1441 1478 62 3.5 288.3 0.3X +Collect java.time.Instant 1471 1579 100 3.4 294.3 0.3X +java.sql.Date to Hive string 12049 12909 862 0.4 2409.8 0.0X +java.time.LocalDate to Hive string 12045 12130 74 0.4 2408.9 0.0X +java.sql.Timestamp to Hive string 12854 13376 510 0.4 2570.9 0.0X +java.time.Instant to Hive string 15057 15184 115 0.3 3011.4 0.0X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt index 0795f11a57f28..ebfcb45f30ce0 100644 --- a/sql/core/benchmarks/DateTimeBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -6,18 +6,18 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datetime +/- interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date + interval(m) 1555 1634 113 6.4 155.5 1.0X -date + interval(m, d) 1774 1797 33 5.6 177.4 0.9X -date + interval(m, d, ms) 6293 6335 59 1.6 629.3 0.2X -date - interval(m) 1461 1468 10 6.8 146.1 1.1X -date - interval(m, d) 1741 1741 0 5.7 174.1 0.9X -date - interval(m, d, ms) 6503 6518 21 1.5 650.3 0.2X -timestamp + interval(m) 2384 2385 1 4.2 238.4 0.7X -timestamp + interval(m, d) 2683 2684 2 3.7 268.3 0.6X -timestamp + interval(m, d, ms) 2987 3001 19 3.3 298.7 0.5X -timestamp - interval(m) 2391 2395 5 4.2 239.1 0.7X -timestamp - interval(m, d) 2674 2684 14 3.7 267.4 0.6X -timestamp - interval(m, d, ms) 3005 3007 3 3.3 300.5 0.5X +date + interval(m) 1636 1653 24 6.1 163.6 1.0X +date + interval(m, d) 1802 1818 23 5.5 180.2 0.9X +date + interval(m, d, ms) 6330 6348 26 1.6 633.0 0.3X +date - interval(m) 1462 1484 32 6.8 146.2 1.1X +date - interval(m, d) 1732 1732 1 5.8 173.2 0.9X +date - interval(m, d, ms) 6494 6505 16 1.5 649.4 0.3X +timestamp + interval(m) 2446 2446 0 4.1 244.6 0.7X +timestamp + interval(m, d) 2670 2703 46 3.7 267.0 0.6X +timestamp + interval(m, d, ms) 2992 3012 29 3.3 299.2 0.5X +timestamp - interval(m) 2447 2449 3 4.1 244.7 0.7X +timestamp - interval(m, d) 2739 2739 0 3.7 273.9 0.6X +timestamp - interval(m, d, ms) 2977 2983 8 3.4 297.7 0.5X ================================================================================================ @@ -28,92 +28,92 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to timestamp wholestage off 313 320 10 31.9 31.3 1.0X -cast to timestamp wholestage on 325 341 18 30.8 32.5 1.0X +cast to timestamp wholestage off 312 321 13 32.1 31.2 1.0X +cast to timestamp wholestage on 290 311 14 34.5 29.0 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz year of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -year of timestamp wholestage off 1216 1216 1 8.2 121.6 1.0X -year of timestamp wholestage on 1226 1243 13 8.2 122.6 1.0X +year of timestamp wholestage off 1226 1228 3 8.2 122.6 1.0X +year of timestamp wholestage on 1214 1222 10 8.2 121.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz quarter of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -quarter of timestamp wholestage off 1417 1421 5 7.1 141.7 1.0X -quarter of timestamp wholestage on 1358 1365 8 7.4 135.8 1.0X +quarter of timestamp wholestage off 1437 1447 14 7.0 143.7 1.0X +quarter of timestamp wholestage on 1354 1359 4 7.4 135.4 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz month of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -month of timestamp wholestage off 1219 1220 1 8.2 121.9 1.0X -month of timestamp wholestage on 1222 1227 7 8.2 122.2 1.0X +month of timestamp wholestage off 1219 1219 1 8.2 121.9 1.0X +month of timestamp wholestage on 1205 1211 7 8.3 120.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekofyear of timestamp wholestage off 1950 1950 0 5.1 195.0 1.0X -weekofyear of timestamp wholestage on 1890 1899 8 5.3 189.0 1.0X +weekofyear of timestamp wholestage off 1849 1854 7 5.4 184.9 1.0X +weekofyear of timestamp wholestage on 1829 1835 5 5.5 182.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz day of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -day of timestamp wholestage off 1212 1213 2 8.3 121.2 1.0X -day of timestamp wholestage on 1216 1227 13 8.2 121.6 1.0X +day of timestamp wholestage off 1224 1230 8 8.2 122.4 1.0X +day of timestamp wholestage on 1204 1215 10 8.3 120.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofyear of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofyear of timestamp wholestage off 1282 1284 3 7.8 128.2 1.0X -dayofyear of timestamp wholestage on 1269 1274 5 7.9 126.9 1.0X +dayofyear of timestamp wholestage off 1272 1275 5 7.9 127.2 1.0X +dayofyear of timestamp wholestage on 1246 1256 7 8.0 124.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofmonth of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofmonth of timestamp wholestage off 1214 1219 7 8.2 121.4 1.0X -dayofmonth of timestamp wholestage on 1216 1224 6 8.2 121.6 1.0X +dayofmonth of timestamp wholestage off 1226 1233 11 8.2 122.6 1.0X +dayofmonth of timestamp wholestage on 1205 1211 5 8.3 120.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz dayofweek of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -dayofweek of timestamp wholestage off 1403 1430 39 7.1 140.3 1.0X -dayofweek of timestamp wholestage on 1378 1386 8 7.3 137.8 1.0X +dayofweek of timestamp wholestage off 1420 1427 9 7.0 142.0 1.0X +dayofweek of timestamp wholestage on 1375 1385 11 7.3 137.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz weekday of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -weekday of timestamp wholestage off 1344 1353 13 7.4 134.4 1.0X +weekday of timestamp wholestage off 1345 1347 3 7.4 134.5 1.0X weekday of timestamp wholestage on 1316 1322 5 7.6 131.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz hour of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -hour of timestamp wholestage off 992 1000 10 10.1 99.2 1.0X -hour of timestamp wholestage on 960 962 3 10.4 96.0 1.0X +hour of timestamp wholestage off 983 984 1 10.2 98.3 1.0X +hour of timestamp wholestage on 942 953 8 10.6 94.2 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz minute of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -minute of timestamp wholestage off 989 1000 16 10.1 98.9 1.0X -minute of timestamp wholestage on 965 974 13 10.4 96.5 1.0X +minute of timestamp wholestage off 1008 1010 3 9.9 100.8 1.0X +minute of timestamp wholestage on 942 945 3 10.6 94.2 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz second of timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -second of timestamp wholestage off 974 977 5 10.3 97.4 1.0X -second of timestamp wholestage on 959 966 8 10.4 95.9 1.0X +second of timestamp wholestage off 975 976 1 10.3 97.5 1.0X +second of timestamp wholestage on 938 944 4 10.7 93.8 1.0X ================================================================================================ @@ -124,15 +124,15 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_date wholestage off 281 282 2 35.6 28.1 1.0X -current_date wholestage on 294 300 5 34.0 29.4 1.0X +current_date wholestage off 295 296 2 33.9 29.5 1.0X +current_date wholestage on 267 274 6 37.5 26.7 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz current_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -current_timestamp wholestage off 282 296 19 35.4 28.2 1.0X -current_timestamp wholestage on 304 331 31 32.9 30.4 0.9X +current_timestamp wholestage off 298 303 7 33.5 29.8 1.0X +current_timestamp wholestage on 261 275 12 38.2 26.1 1.1X ================================================================================================ @@ -143,43 +143,43 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast to date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast to date wholestage off 1060 1061 1 9.4 106.0 1.0X -cast to date wholestage on 1021 1026 10 9.8 102.1 1.0X +cast to date wholestage off 1071 1073 3 9.3 107.1 1.0X +cast to date wholestage on 998 1014 31 10.0 99.8 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz last_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -last_day wholestage off 1278 1280 3 7.8 127.8 1.0X -last_day wholestage on 1560 1566 6 6.4 156.0 0.8X +last_day wholestage off 1260 1261 1 7.9 126.0 1.0X +last_day wholestage on 1245 1261 17 8.0 124.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz next_day: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -next_day wholestage off 1091 1093 3 9.2 109.1 1.0X -next_day wholestage on 1070 1076 9 9.3 107.0 1.0X +next_day wholestage off 1118 1120 2 8.9 111.8 1.0X +next_day wholestage on 1043 1047 3 9.6 104.3 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_add: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_add wholestage off 1041 1047 8 9.6 104.1 1.0X -date_add wholestage on 1044 1050 4 9.6 104.4 1.0X +date_add wholestage off 1046 1048 3 9.6 104.6 1.0X +date_add wholestage on 1040 1048 11 9.6 104.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_sub: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_sub wholestage off 1038 1040 3 9.6 103.8 1.0X -date_sub wholestage on 1057 1061 4 9.5 105.7 1.0X +date_sub wholestage off 1081 1081 0 9.3 108.1 1.0X +date_sub wholestage on 1030 1035 6 9.7 103.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz add_months: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -add_months wholestage off 1401 1401 1 7.1 140.1 1.0X -add_months wholestage on 1438 1442 4 7.0 143.8 1.0X +add_months wholestage off 1393 1400 10 7.2 139.3 1.0X +add_months wholestage on 1391 1396 5 7.2 139.1 1.0X ================================================================================================ @@ -190,8 +190,8 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz format date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -format date wholestage off 5482 5803 454 1.8 548.2 1.0X -format date wholestage on 5502 5518 9 1.8 550.2 1.0X +format date wholestage off 5424 5426 2 1.8 542.4 1.0X +format date wholestage on 5408 5448 37 1.8 540.8 1.0X ================================================================================================ @@ -202,8 +202,8 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_unixtime: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_unixtime wholestage off 8538 8553 22 1.2 853.8 1.0X -from_unixtime wholestage on 8545 8552 6 1.2 854.5 1.0X +from_unixtime wholestage off 8839 8841 3 1.1 883.9 1.0X +from_unixtime wholestage on 8788 8826 24 1.1 878.8 1.0X ================================================================================================ @@ -214,15 +214,15 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz from_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -from_utc_timestamp wholestage off 1094 1099 8 9.1 109.4 1.0X -from_utc_timestamp wholestage on 1109 1114 5 9.0 110.9 1.0X +from_utc_timestamp wholestage off 1105 1111 8 9.0 110.5 1.0X +from_utc_timestamp wholestage on 1073 1081 8 9.3 107.3 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_utc_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_utc_timestamp wholestage off 1466 1469 4 6.8 146.6 1.0X -to_utc_timestamp wholestage on 1401 1408 7 7.1 140.1 1.0X +to_utc_timestamp wholestage off 1462 1465 4 6.8 146.2 1.0X +to_utc_timestamp wholestage on 1394 1408 13 7.2 139.4 1.0X ================================================================================================ @@ -233,29 +233,29 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz cast interval: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -cast interval wholestage off 332 332 0 30.1 33.2 1.0X -cast interval wholestage on 315 324 10 31.7 31.5 1.1X +cast interval wholestage off 325 328 4 30.8 32.5 1.0X +cast interval wholestage on 286 290 3 35.0 28.6 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz datediff: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -datediff wholestage off 1796 1802 8 5.6 179.6 1.0X -datediff wholestage on 1758 1764 10 5.7 175.8 1.0X +datediff wholestage off 1822 1824 3 5.5 182.2 1.0X +datediff wholestage on 1757 1761 5 5.7 175.7 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz months_between: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -months_between wholestage off 4833 4836 4 2.1 483.3 1.0X -months_between wholestage on 4777 4780 2 2.1 477.7 1.0X +months_between wholestage off 4886 4893 10 2.0 488.6 1.0X +months_between wholestage on 4785 4799 12 2.1 478.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz window: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -window wholestage off 1812 1908 136 0.6 1811.7 1.0X -window wholestage on 46279 46376 74 0.0 46278.8 0.0X +window wholestage off 2024 2052 40 0.5 2023.7 1.0X +window wholestage on 46599 46660 45 0.0 46599.0 0.0X ================================================================================================ @@ -266,134 +266,134 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YEAR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YEAR wholestage off 2367 2368 1 4.2 236.7 1.0X -date_trunc YEAR wholestage on 2321 2334 22 4.3 232.1 1.0X +date_trunc YEAR wholestage off 2361 2366 7 4.2 236.1 1.0X +date_trunc YEAR wholestage on 2325 2328 3 4.3 232.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YYYY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YYYY wholestage off 2330 2334 5 4.3 233.0 1.0X -date_trunc YYYY wholestage on 2326 2332 5 4.3 232.6 1.0X +date_trunc YYYY wholestage off 2366 2374 12 4.2 236.6 1.0X +date_trunc YYYY wholestage on 2316 2328 13 4.3 231.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc YY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc YY wholestage off 2334 2335 1 4.3 233.4 1.0X -date_trunc YY wholestage on 2315 2324 6 4.3 231.5 1.0X +date_trunc YY wholestage off 2359 2359 0 4.2 235.9 1.0X +date_trunc YY wholestage on 2315 2325 7 4.3 231.5 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MON: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MON wholestage off 2327 2330 4 4.3 232.7 1.0X -date_trunc MON wholestage on 2279 2289 12 4.4 227.9 1.0X +date_trunc MON wholestage off 2360 2369 12 4.2 236.0 1.0X +date_trunc MON wholestage on 2306 2314 9 4.3 230.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MONTH: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MONTH wholestage off 2330 2332 2 4.3 233.0 1.0X -date_trunc MONTH wholestage on 2277 2284 6 4.4 227.7 1.0X +date_trunc MONTH wholestage off 2359 2360 2 4.2 235.9 1.0X +date_trunc MONTH wholestage on 2304 2308 4 4.3 230.4 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MM: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MM wholestage off 2328 2329 2 4.3 232.8 1.0X -date_trunc MM wholestage on 2279 2284 4 4.4 227.9 1.0X +date_trunc MM wholestage off 2356 2358 2 4.2 235.6 1.0X +date_trunc MM wholestage on 2302 2309 6 4.3 230.2 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DAY: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DAY wholestage off 1974 1984 14 5.1 197.4 1.0X -date_trunc DAY wholestage on 1914 1922 7 5.2 191.4 1.0X +date_trunc DAY wholestage off 1962 1964 3 5.1 196.2 1.0X +date_trunc DAY wholestage on 1916 1921 6 5.2 191.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc DD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc DD wholestage off 1967 1976 12 5.1 196.7 1.0X -date_trunc DD wholestage on 1913 1917 4 5.2 191.3 1.0X +date_trunc DD wholestage off 1956 1957 2 5.1 195.6 1.0X +date_trunc DD wholestage on 1916 1922 6 5.2 191.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc HOUR: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc HOUR wholestage off 1970 1970 0 5.1 197.0 1.0X -date_trunc HOUR wholestage on 1945 1946 2 5.1 194.5 1.0X +date_trunc HOUR wholestage off 1968 1970 3 5.1 196.8 1.0X +date_trunc HOUR wholestage on 1949 1961 9 5.1 194.9 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc MINUTE: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc MINUTE wholestage off 361 361 1 27.7 36.1 1.0X -date_trunc MINUTE wholestage on 331 336 4 30.2 33.1 1.1X +date_trunc MINUTE wholestage off 368 373 7 27.2 36.8 1.0X +date_trunc MINUTE wholestage on 338 343 6 29.6 33.8 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc SECOND: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc SECOND wholestage off 360 361 1 27.8 36.0 1.0X -date_trunc SECOND wholestage on 335 348 15 29.8 33.5 1.1X +date_trunc SECOND wholestage off 379 379 1 26.4 37.9 1.0X +date_trunc SECOND wholestage on 327 340 13 30.6 32.7 1.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc WEEK: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc WEEK wholestage off 2232 2236 6 4.5 223.2 1.0X -date_trunc WEEK wholestage on 2225 2232 6 4.5 222.5 1.0X +date_trunc WEEK wholestage off 2227 2242 21 4.5 222.7 1.0X +date_trunc WEEK wholestage on 2231 2241 9 4.5 223.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz date_trunc QUARTER: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -date_trunc QUARTER wholestage off 3083 3086 4 3.2 308.3 1.0X -date_trunc QUARTER wholestage on 3073 3086 16 3.3 307.3 1.0X +date_trunc QUARTER wholestage off 3158 3160 3 3.2 315.8 1.0X +date_trunc QUARTER wholestage on 3150 3163 12 3.2 315.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc year: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc year wholestage off 321 321 0 31.1 32.1 1.0X -trunc year wholestage on 299 303 5 33.5 29.9 1.1X +trunc year wholestage off 321 323 3 31.2 32.1 1.0X +trunc year wholestage on 302 330 18 33.1 30.2 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yyyy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yyyy wholestage off 323 327 5 30.9 32.3 1.0X -trunc yyyy wholestage on 299 302 3 33.4 29.9 1.1X +trunc yyyy wholestage off 320 324 6 31.2 32.0 1.0X +trunc yyyy wholestage on 294 329 20 34.0 29.4 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc yy: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc yy wholestage off 315 315 1 31.8 31.5 1.0X -trunc yy wholestage on 299 304 4 33.4 29.9 1.1X +trunc yy wholestage off 322 322 0 31.1 32.2 1.0X +trunc yy wholestage on 293 320 37 34.1 29.3 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mon: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mon wholestage off 320 321 1 31.2 32.0 1.0X -trunc mon wholestage on 299 307 10 33.4 29.9 1.1X +trunc mon wholestage off 320 322 2 31.2 32.0 1.0X +trunc mon wholestage on 291 312 26 34.4 29.1 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc month: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc month wholestage off 316 317 1 31.6 31.6 1.0X -trunc month wholestage on 299 302 5 33.5 29.9 1.1X +trunc month wholestage off 318 331 18 31.4 31.8 1.0X +trunc month wholestage on 297 329 28 33.7 29.7 1.1X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz trunc mm: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -trunc mm wholestage off 313 313 1 32.0 31.3 1.0X -trunc mm wholestage on 298 302 4 33.5 29.8 1.0X +trunc mm wholestage off 318 319 1 31.4 31.8 1.0X +trunc mm wholestage on 312 335 15 32.1 31.2 1.0X ================================================================================================ @@ -404,36 +404,36 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to timestamp str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to timestamp str wholestage off 217 217 0 4.6 217.3 1.0X -to timestamp str wholestage on 209 212 2 4.8 209.5 1.0X +to timestamp str wholestage off 217 221 5 4.6 217.5 1.0X +to timestamp str wholestage on 210 214 5 4.8 210.0 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_timestamp wholestage off 1676 1677 2 0.6 1675.6 1.0X -to_timestamp wholestage on 1599 1606 8 0.6 1599.5 1.0X +to_timestamp wholestage off 1714 1718 5 0.6 1714.4 1.0X +to_timestamp wholestage on 1418 1433 14 0.7 1418.5 1.2X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_unix_timestamp wholestage off 1582 1589 9 0.6 1582.1 1.0X -to_unix_timestamp wholestage on 1634 1637 3 0.6 1633.8 1.0X +to_unix_timestamp wholestage off 1436 1441 6 0.7 1436.2 1.0X +to_unix_timestamp wholestage on 1421 1426 7 0.7 1420.6 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to date str: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to date str wholestage off 275 282 9 3.6 275.0 1.0X -to date str wholestage on 264 265 2 3.8 263.5 1.0X +to date str wholestage off 267 267 0 3.8 266.6 1.0X +to date str wholestage on 260 262 2 3.8 260.1 1.0X OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aws Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz to_date: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -to_date wholestage off 3170 3188 25 0.3 3170.1 1.0X -to_date wholestage on 3134 3143 10 0.3 3134.3 1.0X +to_date wholestage off 3419 3436 25 0.3 3419.0 1.0X +to_date wholestage on 3344 3352 7 0.3 3343.5 1.0X ================================================================================================ @@ -444,18 +444,18 @@ OpenJDK 64-Bit Server VM 1.8.0_252-8u252-b09-1~18.04-b09 on Linux 4.15.0-1063-aw Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz To/from Java's date-time: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -From java.sql.Date 407 413 7 12.3 81.5 1.0X -From java.time.LocalDate 340 344 5 14.7 68.1 1.2X -Collect java.sql.Date 1700 2658 1422 2.9 340.0 0.2X -Collect java.time.LocalDate 1473 1494 30 3.4 294.6 0.3X -From java.sql.Timestamp 252 266 13 19.8 50.5 1.6X -From java.time.Instant 236 243 7 21.1 47.3 1.7X -Collect longs 1280 1337 79 3.9 256.1 0.3X -Collect java.sql.Timestamp 1485 1501 15 3.4 297.0 0.3X -Collect java.time.Instant 1441 1465 37 3.5 288.1 0.3X -java.sql.Date to Hive string 18745 20895 1364 0.3 3749.0 0.0X -java.time.LocalDate to Hive string 15296 15450 143 0.3 3059.2 0.0X -java.sql.Timestamp to Hive string 46421 47210 946 0.1 9284.2 0.0X -java.time.Instant to Hive string 34747 35187 382 0.1 6949.4 0.0X +From java.sql.Date 436 445 8 11.5 87.2 1.0X +From java.time.LocalDate 348 357 11 14.4 69.7 1.3X +Collect java.sql.Date 1723 1917 168 2.9 344.5 0.3X +Collect java.time.LocalDate 1591 1602 18 3.1 318.3 0.3X +From java.sql.Timestamp 248 252 4 20.2 49.6 1.8X +From java.time.Instant 232 238 5 21.5 46.5 1.9X +Collect longs 1398 1455 99 3.6 279.5 0.3X +Collect java.sql.Timestamp 1469 1483 13 3.4 293.9 0.3X +Collect java.time.Instant 1561 1597 40 3.2 312.2 0.3X +java.sql.Date to Hive string 13820 14798 857 0.4 2763.9 0.0X +java.time.LocalDate to Hive string 14374 14779 357 0.3 2874.8 0.0X +java.sql.Timestamp to Hive string 14872 15461 653 0.3 2974.5 0.0X +java.time.Instant to Hive string 17062 17789 759 0.3 3412.4 0.0X diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 9f99bf5011569..4d388e40fb8bd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -33,6 +33,23 @@ import org.apache.spark.unsafe.types.CalendarInterval * Runs a query returning the result in Hive compatible form. */ object HiveResult { + case class TimeFormatters(date: DateFormatter, timestamp: TimestampFormatter) + + def getTimeFormatters: TimeFormatters = { + // The date formatter does not depend on Spark's session time zone controlled by + // the SQL config `spark.sql.session.timeZone`. The `zoneId` parameter is used only in + // parsing of special date values like `now`, `yesterday` and etc. but not in date formatting. + // While formatting of: + // - `java.time.LocalDate`, zone id is not used by `DateTimeFormatter` at all. + // - `java.sql.Date`, the date formatter delegates formatting to the legacy formatter + // which uses the default system time zone `TimeZone.getDefault`. This works correctly + // due to `DateTimeUtils.toJavaDate` which is based on the system time zone too. + val dateFormatter = DateFormatter(ZoneOffset.UTC) + val timestampFormatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + TimeFormatters(dateFormatter, timestampFormatter) + } + /** * Returns the result as a hive compatible sequence of strings. This is used in tests and * `SparkSQLDriver` for CLI applications. @@ -55,11 +72,12 @@ object HiveResult { case command @ ExecutedCommandExec(_: ShowViewsCommand) => command.executeCollect().map(_.getString(1)) case other => + val timeFormatters = getTimeFormatters val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq // We need the types so we can output struct field names val types = executedPlan.output.map(_.dataType) // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(e => toHiveString(e))) + result.map(_.zip(types).map(e => toHiveString(e, false, timeFormatters))) .map(_.mkString("\t")) } @@ -72,47 +90,32 @@ object HiveResult { } } - // We can create the date formatter only once because it does not depend on Spark's - // session time zone controlled by the SQL config `spark.sql.session.timeZone`. - // The `zoneId` parameter is used only in parsing of special date values like `now`, - // `yesterday` and etc. but not in date formatting. While formatting of: - // - `java.time.LocalDate`, zone id is not used by `DateTimeFormatter` at all. - // - `java.sql.Date`, the date formatter delegates formatting to the legacy formatter - // which uses the default system time zone `TimeZone.getDefault`. This works correctly - // due to `DateTimeUtils.toJavaDate` which is based on the system time zone too. - private val dateFormatter = DateFormatter( - format = DateFormatter.defaultPattern, - // We can set any time zone id. UTC was taken for simplicity. - zoneId = ZoneOffset.UTC, - locale = DateFormatter.defaultLocale, - // Use `FastDateFormat` as the legacy formatter because it is thread-safe. - legacyFormat = LegacyDateFormats.FAST_DATE_FORMAT, - isParsing = false) - private def timestampFormatter = TimestampFormatter.getFractionFormatter( - DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) - /** Formats a datum (based on the given data type) and returns the string representation. */ - def toHiveString(a: (Any, DataType), nested: Boolean = false): String = a match { + def toHiveString( + a: (Any, DataType), + nested: Boolean, + formatters: TimeFormatters): String = a match { case (null, _) => if (nested) "null" else "NULL" case (b, BooleanType) => b.toString - case (d: Date, DateType) => dateFormatter.format(d) - case (ld: LocalDate, DateType) => dateFormatter.format(ld) - case (t: Timestamp, TimestampType) => timestampFormatter.format(t) - case (i: Instant, TimestampType) => timestampFormatter.format(i) + case (d: Date, DateType) => formatters.date.format(d) + case (ld: LocalDate, DateType) => formatters.date.format(ld) + case (t: Timestamp, TimestampType) => formatters.timestamp.format(t) + case (i: Instant, TimestampType) => formatters.timestamp.format(i) case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) case (decimal: java.math.BigDecimal, DecimalType()) => decimal.toPlainString case (n, _: NumericType) => n.toString case (s: String, StringType) => if (nested) "\"" + s + "\"" else s case (interval: CalendarInterval, CalendarIntervalType) => interval.toString case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(e => toHiveString(e, true)).mkString("[", ",", "]") + seq.map(v => (v, typ)).map(e => toHiveString(e, true, formatters)).mkString("[", ",", "]") case (m: Map[_, _], MapType(kType, vType, _)) => m.map { case (key, value) => - toHiveString((key, kType), true) + ":" + toHiveString((value, vType), true) + toHiveString((key, kType), true, formatters) + ":" + + toHiveString((value, vType), true, formatters) }.toSeq.sorted.mkString("{", ",", "}") case (struct: Row, StructType(fields)) => struct.toSeq.zip(fields).map { case (v, t) => - s""""${t.name}":${toHiveString((v, t.dataType), true)}""" + s""""${t.name}":${toHiveString((v, t.dataType), true, formatters)}""" }.mkString("{", ",", "}") case (other, _: UserDefinedType[_]) => other.toString } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala index a0b212d2cf6fd..a49beda2186b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.connector.InMemoryTableCatalog +import org.apache.spark.sql.execution.HiveResult._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSparkSession} @@ -31,10 +32,10 @@ class HiveResultSuite extends SharedSparkSession { val dates = Seq("2018-12-28", "1582-10-03", "1582-10-04", "1582-10-15") val df = dates.toDF("a").selectExpr("cast(a as date) as b") val executedPlan1 = df.queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan1) + val result = hiveResultString(executedPlan1) assert(result == dates) val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan - val result2 = HiveResult.hiveResultString(executedPlan2) + val result2 = hiveResultString(executedPlan2) assert(result2 == dates.map(x => s"[$x]")) } } @@ -48,17 +49,17 @@ class HiveResultSuite extends SharedSparkSession { "1582-10-15 01:02:03") val df = timestamps.toDF("a").selectExpr("cast(a as timestamp) as b") val executedPlan1 = df.queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan1) + val result = hiveResultString(executedPlan1) assert(result == timestamps) val executedPlan2 = df.selectExpr("array(b)").queryExecution.executedPlan - val result2 = HiveResult.hiveResultString(executedPlan2) + val result2 = hiveResultString(executedPlan2) assert(result2 == timestamps.map(x => s"[$x]")) } test("toHiveString correctly handles UDTs") { val point = new ExamplePoint(50.0, 50.0) val tpe = new ExamplePointUDT() - assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") + assert(toHiveString((point, tpe), false, getTimeFormatters) === "(50.0, 50.0)") } test("decimal formatting in hive result") { @@ -66,13 +67,13 @@ class HiveResultSuite extends SharedSparkSession { Seq(2, 6, 18).foreach { scala => val executedPlan = df.selectExpr(s"CAST(value AS decimal(38, $scala))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) + val result = hiveResultString(executedPlan) assert(result.head.split("\\.").last.length === scala) } val executedPlan = Seq(java.math.BigDecimal.ZERO).toDS() .selectExpr(s"CAST(value AS decimal(38, 8))").queryExecution.executedPlan - val result = HiveResult.hiveResultString(executedPlan) + val result = hiveResultString(executedPlan) assert(result.head === "0.00000000") } @@ -84,7 +85,7 @@ class HiveResultSuite extends SharedSparkSession { spark.sql(s"CREATE TABLE $ns.$tbl (id bigint) USING $source") val df = spark.sql(s"SHOW TABLES FROM $ns") val executedPlan = df.queryExecution.executedPlan - assert(HiveResult.hiveResultString(executedPlan).head == tbl) + assert(hiveResultString(executedPlan).head == tbl) } } } @@ -101,7 +102,7 @@ class HiveResultSuite extends SharedSparkSession { val expected = "id " + "\tbigint " + "\tcol1 " - assert(HiveResult.hiveResultString(executedPlan).head == expected) + assert(hiveResultString(executedPlan).head == expected) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index b193c73563ae0..1404ece76449e 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -36,7 +36,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} -import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.execution.command.SetCommand import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -73,7 +73,11 @@ private[hive] class SparkExecuteStatementOperation( } } - def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int): Unit = { + def addNonNullColumnValue( + from: SparkRow, + to: ArrayBuffer[Any], + ordinal: Int, + timeFormatters: TimeFormatters): Unit = { dataTypes(ordinal) match { case StringType => to += from.getString(ordinal) @@ -100,13 +104,14 @@ private[hive] class SparkExecuteStatementOperation( // - work with spark.sql.datetime.java8API.enabled // These types have always been sent over the wire as string, converted later. case _: DateType | _: TimestampType => - val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) - to += hiveString + to += toHiveString((from.get(ordinal), dataTypes(ordinal)), false, timeFormatters) case CalendarIntervalType => - to += HiveResult.toHiveString((from.getAs[CalendarInterval](ordinal), CalendarIntervalType)) + to += toHiveString( + (from.getAs[CalendarInterval](ordinal), CalendarIntervalType), + false, + timeFormatters) case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] => - val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) - to += hiveString + to += toHiveString((from.get(ordinal), dataTypes(ordinal)), false, timeFormatters) } } @@ -159,6 +164,7 @@ private[hive] class SparkExecuteStatementOperation( if (!iter.hasNext) { resultRowSet } else { + val timeFormatters = getTimeFormatters // maxRowsL here typically maps to java.sql.Statement.getFetchSize, which is an int val maxRows = maxRowsL.toInt var curRow = 0 @@ -170,7 +176,7 @@ private[hive] class SparkExecuteStatementOperation( if (sparkRow.isNullAt(curCol)) { row += null } else { - addNonNullColumnValue(sparkRow, row, curCol) + addNonNullColumnValue(sparkRow, row, curCol, timeFormatters) } curCol += 1 } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala index 15cc3109da3f7..9b68144b52005 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerQueryTestSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkException import org.apache.spark.sql.SQLQueryTestSuite import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.util.fileToString -import org.apache.spark.sql.execution.HiveResult +import org.apache.spark.sql.execution.HiveResult.{getTimeFormatters, toHiveString, TimeFormatters} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -257,8 +257,9 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ private def getNormalizedResult(statement: Statement, sql: String): (String, Seq[String]) = { val rs = statement.executeQuery(sql) val cols = rs.getMetaData.getColumnCount + val timeFormatters = getTimeFormatters val buildStr = () => (for (i <- 1 to cols) yield { - getHiveResult(rs.getObject(i)) + getHiveResult(rs.getObject(i), timeFormatters) }).mkString("\t") val answer = Iterator.continually(rs.next()).takeWhile(identity).map(_ => buildStr()).toSeq @@ -280,18 +281,18 @@ class ThriftServerQueryTestSuite extends SQLQueryTestSuite with SharedThriftServ upperCase.startsWith("(") } - private def getHiveResult(obj: Object): String = { + private def getHiveResult(obj: Object, timeFormatters: TimeFormatters): String = { obj match { case null => - HiveResult.toHiveString((null, StringType)) + toHiveString((null, StringType), false, timeFormatters) case d: java.sql.Date => - HiveResult.toHiveString((d, DateType)) + toHiveString((d, DateType), false, timeFormatters) case t: Timestamp => - HiveResult.toHiveString((t, TimestampType)) + toHiveString((t, TimestampType), false, timeFormatters) case d: java.math.BigDecimal => - HiveResult.toHiveString((d, DecimalType.fromDecimal(Decimal(d)))) + toHiveString((d, DecimalType.fromDecimal(Decimal(d))), false, timeFormatters) case bin: Array[Byte] => - HiveResult.toHiveString((bin, BinaryType)) + toHiveString((bin, BinaryType), false, timeFormatters) case other => other.toString } From 4badef38a52849b4af0b211523de6b09f73397f1 Mon Sep 17 00:00:00 2001 From: "yi.wu" Date: Wed, 17 Jun 2020 13:28:47 +0000 Subject: [PATCH 27/28] [SPARK-32000][CORE][TESTS] Fix the flaky test for partially launched task in barrier-mode ### What changes were proposed in this pull request? This PR changes the test to get an active executorId and set it as preferred location instead of setting a fixed preferred location. ### Why are the changes needed? The test is flaky. After checking the [log](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/124086/artifact/core/), I find the root cause is: Two test cases from different test suites got submitted at the same time because of concurrent execution. In this particular case, the two test cases (from DistributedSuite and BarrierTaskContextSuite) both launch under local-cluster mode. The two applications are submitted at the SAME time so they have the same applications(app-20200615210132-0000). Thus, when the cluster of BarrierTaskContextSuite is launching executors, it failed to create the directory for the executor 0, because the path (/home/jenkins/workspace/work/app-app-20200615210132-0000/0) has been used by the cluster of DistributedSuite. Therefore, it has to launch executor 1 and 2 instead, that lead to non of the tasks can get preferred locality thus they got scheduled together and lead to the test failure. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The test can not be reproduced locally. We can only know it's been fixed when it's no longer flaky on Jenkins. Closes #28849 from Ngone51/fix-spark-32000. Authored-by: yi.wu Signed-off-by: Wenchen Fan --- .../org/apache/spark/scheduler/BarrierTaskContextSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala index 01c82f894cf98..d18ca36f1fa60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -276,11 +276,12 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext with test("SPARK-31485: barrier stage should fail if only partial tasks are launched") { initLocalClusterSparkContext(2) + val id = sc.getExecutorIds().head val rdd0 = sc.parallelize(Seq(0, 1, 2, 3), 2) val dep = new OneToOneDependency[Int](rdd0) - // set up a barrier stage with 2 tasks and both tasks prefer executor 0 (only 1 core) for + // set up a barrier stage with 2 tasks and both tasks prefer the same executor (only 1 core) for // scheduling. So, one of tasks won't be scheduled in one round of resource offer. - val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq("executor_h_0"), Seq("executor_h_0"))) + val rdd = new MyRDD(sc, 2, List(dep), Seq(Seq(s"executor_h_$id"), Seq(s"executor_h_$id"))) val errorMsg = intercept[SparkException] { rdd.barrier().mapPartitions { iter => BarrierTaskContext.get().barrier() From 9b792518b2d420d1026cfdc38729e30f45a36c91 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 18 Jun 2020 06:08:40 +0000 Subject: [PATCH 28/28] [SPARK-31960][YARN][BUILD] Only populate Hadoop classpath for no-hadoop build ### What changes were proposed in this pull request? If a Spark distribution has built-in hadoop runtime, Spark will not populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` when a job is submitted to Yarn. Users can override this behavior by setting `spark.yarn.populateHadoopClasspath` to `true`. ### Why are the changes needed? Without this, Spark will populate the hadoop classpath from `yarn.application.classpath` and `mapreduce.application.classpath` even Spark distribution has built-in hadoop. This results jar conflict and many unexpected behaviors in runtime. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually test with two builds, with-hadoop and no-hadoop builds. Closes #28788 from dbtsai/yarn-classpath. Authored-by: DB Tsai Signed-off-by: DB Tsai --- dev/.rat-excludes | 1 + docs/running-on-yarn.md | 17 ++++++++++++- resource-managers/yarn/pom.xml | 16 ++++++++++++ .../spark/deploy/yarn/config.properties | 1 + .../org/apache/spark/deploy/yarn/config.scala | 25 ++++++++++++++++--- 5 files changed, 56 insertions(+), 4 deletions(-) create mode 100644 resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 4540e5feac7c9..326e561529073 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -123,3 +123,4 @@ SessionManager.java SessionHandler.java GangliaReporter.java application_1578436911597_0052 +config.properties diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b58cd24ab051f..eeeb71b022c52 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -82,6 +82,18 @@ In `cluster` mode, the driver runs on a different machine than the client, so `S Running Spark on YARN requires a binary distribution of Spark which is built with YARN support. Binary distributions can be downloaded from the [downloads page](https://spark.apache.org/downloads.html) of the project website. +There are two variants of Spark binary distributions you can download. One is pre-built with a certain +version of Apache Hadoop; this Spark distribution contains built-in Hadoop runtime, so we call it `with-hadoop` Spark +distribution. The other one is pre-built with user-provided Hadoop; since this Spark distribution +doesn't contain a built-in Hadoop runtime, it's smaller, but users have to provide a Hadoop installation separately. +We call this variant `no-hadoop` Spark distribution. For `with-hadoop` Spark distribution, since +it contains a built-in Hadoop runtime already, by default, when a job is submitted to Hadoop Yarn cluster, to prevent jar conflict, it will not +populate Yarn's classpath into Spark. To override this behavior, you can set spark.yarn.populateHadoopClasspath=true. +For `no-hadoop` Spark distribution, Spark will populate Yarn's classpath by default in order to get Hadoop runtime. For `with-hadoop` Spark distribution, +if your application depends on certain library that is only available in the cluster, you can try to populate the Yarn classpath by setting +the property mentioned above. If you run into jar conflict issue by doing so, you will need to turn it off and include this library +in your application jar. + To build Spark yourself, refer to [Building Spark](building-spark.html). To make Spark runtime jars accessible from YARN side, you can specify `spark.yarn.archive` or `spark.yarn.jars`. For details please refer to [Spark Properties](running-on-yarn.html#spark-properties). If neither `spark.yarn.archive` nor `spark.yarn.jars` is specified, Spark will create a zip file with all jars under `$SPARK_HOME/jars` and upload it to the distributed cache. @@ -396,7 +408,10 @@ To use a custom metrics.properties for the application master and executors, upd spark.yarn.populateHadoopClasspath - true + + For with-hadoop Spark distribution, this is set to false; + for no-hadoop distribution, this is set to true. + Whether to populate Hadoop classpath from yarn.application.classpath and mapreduce.application.classpath Note that if this is set to false, diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index b84180abfa200..d081be94ba7ae 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -30,8 +30,18 @@ yarn 1.19 + false + + + hadoop-provided + + true + + + + org.apache.spark @@ -201,6 +211,12 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + src/main/resources + true + + diff --git a/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties b/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties new file mode 100644 index 0000000000000..b923dbab954f5 --- /dev/null +++ b/resource-managers/yarn/src/main/resources/org/apache/spark/deploy/yarn/config.properties @@ -0,0 +1 @@ +spark.yarn.isHadoopProvided = ${spark.yarn.isHadoopProvided} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index b4257a48664c6..1b0bf295db499 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -17,12 +17,14 @@ package org.apache.spark.deploy.yarn +import java.util.Properties import java.util.concurrent.TimeUnit +import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder import org.apache.spark.network.util.ByteUnit -package object config { +package object config extends Logging { /* Common app configuration. */ @@ -74,10 +76,11 @@ package object config { .doc("Whether to populate Hadoop classpath from `yarn.application.classpath` and " + "`mapreduce.application.classpath` Note that if this is set to `false`, it requires " + "a `with-Hadoop` Spark distribution that bundles Hadoop runtime or user has to provide " + - "a Hadoop installation separately.") + "a Hadoop installation separately. By default, for `with-hadoop` Spark distribution, " + + "this is set to `false`; for `no-hadoop` distribution, this is set to `true`.") .version("2.4.6") .booleanConf - .createWithDefault(true) + .createWithDefault(isHadoopProvided()) private[spark] val GATEWAY_ROOT_PATH = ConfigBuilder("spark.yarn.config.gatewayPath") .doc("Root of configuration paths that is present on gateway nodes, and will be replaced " + @@ -394,4 +397,20 @@ package object config { private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." + def isHadoopProvided(): Boolean = IS_HADOOP_PROVIDED + + private lazy val IS_HADOOP_PROVIDED: Boolean = { + val configPath = "org/apache/spark/deploy/yarn/config.properties" + val propertyKey = "spark.yarn.isHadoopProvided" + try { + val prop = new Properties() + prop.load(ClassLoader.getSystemClassLoader.getResourceAsStream(configPath)) + prop.getProperty(propertyKey).toBoolean + } catch { + case e: Exception => + log.warn(s"Can not load the default value of `$propertyKey` from " + + s"`$configPath` with error, ${e.toString}. Using `false` as a default value.") + false + } + } }