diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 2b459e4c73bbb..43ac6b50052ae 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1112,6 +1112,10 @@ jobs: with: distribution: zulu java-version: ${{ inputs.java }} + - name: Install R + run: | + sudo apt update + sudo apt-get install r-base - name: Start Minikube uses: medyagh/setup-minikube@v0.0.18 with: diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala index ff40f16e5a052..66ff551193101 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/CollationFactorySuite.scala @@ -456,7 +456,7 @@ class CollationFactorySuite extends AnyFunSuite with Matchers { // scalastyle:ig val e = intercept[SparkException] { fetchCollation(collationName) } - assert(e.getErrorClass === "COLLATION_INVALID_NAME") + assert(e.getCondition === "COLLATION_INVALID_NAME") assert(e.getMessageParameters.asScala === Map( "collationName" -> collationName, "proposals" -> proposals)) } diff --git a/common/utils/src/main/java/org/apache/spark/SparkThrowable.java b/common/utils/src/main/java/org/apache/spark/SparkThrowable.java index e1235b2982ba0..39808f58b08ae 100644 --- a/common/utils/src/main/java/org/apache/spark/SparkThrowable.java +++ b/common/utils/src/main/java/org/apache/spark/SparkThrowable.java @@ -35,19 +35,29 @@ */ @Evolving public interface SparkThrowable { - // Succinct, human-readable, unique, and consistent representation of the error category - // If null, error class is not set - String getErrorClass(); + /** + * Succinct, human-readable, unique, and consistent representation of the error condition. + * If null, error condition is not set. + */ + String getCondition(); + + /** + * Succinct, human-readable, unique, and consistent representation of the error category. + * If null, error class is not set. + * @deprecated Use {@link #getCondition()} instead. + */ + @Deprecated + default String getErrorClass() { return getCondition(); } // Portable error identifier across SQL engines // If null, error class or SQLSTATE is not set default String getSqlState() { - return SparkThrowableHelper.getSqlState(this.getErrorClass()); + return SparkThrowableHelper.getSqlState(this.getCondition()); } // True if this error is an internal error. default boolean isInternalError() { - return SparkThrowableHelper.isInternalError(this.getErrorClass()); + return SparkThrowableHelper.isInternalError(this.getCondition()); } default Map getMessageParameters() { diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index f2c099a07b9c1..3aa446d692d3d 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2543,6 +2543,11 @@ "Interval string does not match second-nano format of ss.nnnnnnnnn." ] }, + "TIMEZONE_INTERVAL_OUT_OF_RANGE" : { + "message" : [ + "The interval value must be in the range of [-18, +18] hours with second precision." + ] + }, "UNKNOWN_PARSING_ERROR" : { "message" : [ "Unknown error when parsing ." @@ -3830,6 +3835,12 @@ ], "sqlState" : "42000" }, + "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" : { + "message" : [ + "The expression must be inside 'partitionedBy'." + ], + "sqlState" : "42S23" + }, "PATH_ALREADY_EXISTS" : { "message" : [ "Path already exists. Set mode as \"overwrite\" to overwrite the existing path." @@ -4002,6 +4013,18 @@ ], "sqlState" : "22023" }, + "SCALAR_FUNCTION_NOT_COMPATIBLE" : { + "message" : [ + "ScalarFunction not overrides method 'produceResult(InternalRow)' with custom implementation." + ], + "sqlState" : "42K0O" + }, + "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED" : { + "message" : [ + "ScalarFunction not implements or overrides method 'produceResult(InternalRow)'." + ], + "sqlState" : "42K0P" + }, "SCALAR_SUBQUERY_IS_IN_GROUP_BY_OR_AGGREGATE_FUNCTION" : { "message" : [ "The correlated scalar subquery '' is neither present in GROUP BY, nor in an aggregate function.", @@ -5691,11 +5714,6 @@ "Expected format is 'RESET' or 'RESET key'. If you want to include special characters in key, please use quotes, e.g., RESET `key`." ] }, - "_LEGACY_ERROR_TEMP_0044" : { - "message" : [ - "The interval value must be in the range of [-18, +18] hours with second precision." - ] - }, "_LEGACY_ERROR_TEMP_0045" : { "message" : [ "Invalid time zone displacement value." @@ -6702,11 +6720,6 @@ "The pivot column has more than distinct values, this could indicate an error. If this was intended, set to at least the number of distinct values of the pivot column." ] }, - "_LEGACY_ERROR_TEMP_1325" : { - "message" : [ - "Cannot modify the value of a static config: ." - ] - }, "_LEGACY_ERROR_TEMP_1327" : { "message" : [ "Command execution is not supported in runner ." @@ -7957,11 +7970,6 @@ " is not currently supported" ] }, - "_LEGACY_ERROR_TEMP_3055" : { - "message" : [ - "ScalarFunction neither implement magic method nor override 'produceResult'" - ] - }, "_LEGACY_ERROR_TEMP_3056" : { "message" : [ "Unexpected row-level read relations (allow multiple = ): " @@ -8320,11 +8328,6 @@ "Partitions truncate is not supported" ] }, - "_LEGACY_ERROR_TEMP_3146" : { - "message" : [ - "Cannot find a compatible ScalarFunction#produceResult" - ] - }, "_LEGACY_ERROR_TEMP_3147" : { "message" : [ ": Batch scan are not supported" diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index 87811fef9836e..fb899e4eb207e 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -4631,6 +4631,18 @@ "standard": "N", "usedBy": ["Spark"] }, + "42K0O": { + "description": "ScalarFunction not overrides method 'produceResult(InternalRow)' with custom implementation.", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, + "42K0P": { + "description": "ScalarFunction not implements or overrides method 'produceResult(InternalRow)'.", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "42KD0": { "description": "Ambiguous name reference.", "origin": "Databricks", @@ -4901,6 +4913,12 @@ "standard": "N", "usedBy": ["SQL Server"] }, + "42S23": { + "description": "Partition transform expression not in 'partitionedBy'", + "origin": "Spark", + "standard": "N", + "usedBy": ["Spark"] + }, "44000": { "description": "with check option violation", "origin": "SQL/Foundation", diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala b/common/utils/src/main/scala/org/apache/spark/SparkException.scala index fcaee787fd8d3..0c0a1902ee2a1 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala @@ -69,7 +69,7 @@ class SparkException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -179,7 +179,7 @@ private[spark] class SparkUpgradeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull } /** @@ -212,7 +212,7 @@ private[spark] class SparkArithmeticException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -250,7 +250,7 @@ private[spark] class SparkUnsupportedOperationException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull } private[spark] object SparkUnsupportedOperationException { @@ -280,7 +280,7 @@ private[spark] class SparkClassNotFoundException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -296,7 +296,7 @@ private[spark] class SparkConcurrentModificationException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -346,7 +346,7 @@ private[spark] class SparkDateTimeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -362,7 +362,7 @@ private[spark] class SparkFileNotFoundException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -396,7 +396,7 @@ private[spark] class SparkNumberFormatException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -448,7 +448,7 @@ private[spark] class SparkIllegalArgumentException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -477,7 +477,7 @@ private[spark] class SparkRuntimeException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -506,7 +506,7 @@ private[spark] class SparkPythonException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -524,7 +524,7 @@ private[spark] class SparkNoSuchElementException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getQueryContext: Array[QueryContext] = context } @@ -541,7 +541,7 @@ private[spark] class SparkSecurityException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -575,7 +575,7 @@ private[spark] class SparkArrayIndexOutOfBoundsException private( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context } @@ -591,7 +591,7 @@ private[spark] class SparkSQLException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } /** @@ -606,5 +606,5 @@ private[spark] class SparkSQLFeatureNotSupportedException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala index 428c9d2a49351..b6c2b176de62b 100644 --- a/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala +++ b/common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala @@ -81,7 +81,7 @@ private[spark] object SparkThrowableHelper { import ErrorMessageFormat._ format match { case PRETTY => e.getMessage - case MINIMAL | STANDARD if e.getErrorClass == null => + case MINIMAL | STANDARD if e.getCondition == null => toJsonString { generator => val g = generator.useDefaultPrettyPrinter() g.writeStartObject() @@ -92,7 +92,7 @@ private[spark] object SparkThrowableHelper { g.writeEndObject() } case MINIMAL | STANDARD => - val errorClass = e.getErrorClass + val errorClass = e.getCondition toJsonString { generator => val g = generator.useDefaultPrettyPrinter() g.writeStartObject() diff --git a/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 259f4330224c9..1972ef05d8759 100644 --- a/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/common/utils/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -84,7 +84,7 @@ class StreamingQueryException private[sql]( s"""${classOf[StreamingQueryException].getName}: ${cause.getMessage} |$queryDebugString""".stripMargin - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava } diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 751ac275e048a..bb0858decdf8f 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -436,7 +436,7 @@ abstract class AvroLogicalTypeSuite extends QueryTest with SharedSparkSession { val ex = intercept[SparkException] { spark.read.format("avro").load(s"$dir.avro").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkArithmeticException], condition = "NUMERIC_VALUE_OUT_OF_RANGE.WITH_SUGGESTION", diff --git a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index be887bd5237b0..e9d6c2458df81 100644 --- a/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/connector/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -891,7 +891,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema("a DECIMAL(4, 3)").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -969,7 +969,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -1006,7 +1006,7 @@ abstract class AvroSuite val ex = intercept[SparkException] { spark.read.schema(s"a $sqlType").format("avro").load(path.toString).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[AnalysisException], condition = "AVRO_INCOMPATIBLE_READ_TYPE", @@ -1515,7 +1515,7 @@ abstract class AvroSuite .write.format("avro").option("avroSchema", avroSchema) .save(s"$tempDir/${UUID.randomUUID()}") } - assert(ex.getErrorClass == "TASK_WRITE_FAILED") + assert(ex.getCondition == "TASK_WRITE_FAILED") assert(ex.getCause.isInstanceOf[java.lang.NullPointerException]) assert(ex.getCause.getMessage.contains( "null value for (non-nullable) string at test_schema.Name")) @@ -2629,7 +2629,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").option("avroSchema", avroSchema).save(path3_x) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") assert(e.getCause.isInstanceOf[SparkUpgradeException]) } checkDefaultLegacyRead(oldPath) @@ -2884,7 +2884,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").option("avroSchema", avroSchema).save(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } @@ -2895,7 +2895,7 @@ abstract class AvroSuite val e = intercept[SparkException] { df.write.format("avro").save(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index 966b5acebca23..adbfda9691508 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1115,13 +1115,20 @@ class Dataset[T] private[sql] ( } /** @inheritdoc */ - protected def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): Dataset[T] = { + protected def checkpoint( + eager: Boolean, + reliableCheckpoint: Boolean, + storageLevel: Option[StorageLevel]): Dataset[T] = { sparkSession.newDataset(agnosticEncoder) { builder => val command = sparkSession.newCommand { builder => - builder.getCheckpointCommandBuilder + val checkpointBuilder = builder.getCheckpointCommandBuilder .setLocal(!reliableCheckpoint) .setEager(eager) .setRelation(this.plan.getRoot) + storageLevel.foreach { storageLevel => + checkpointBuilder.setStorageLevel( + StorageLevelProtoConverter.toConnectProtoType(storageLevel)) + } } val responseIter = sparkSession.execute(command) try { @@ -1304,6 +1311,10 @@ class Dataset[T] private[sql] ( /** @inheritdoc */ override def localCheckpoint(eager: Boolean): Dataset[T] = super.localCheckpoint(eager) + /** @inheritdoc */ + override def localCheckpoint(eager: Boolean, storageLevel: StorageLevel): Dataset[T] = + super.localCheckpoint(eager, storageLevel) + /** @inheritdoc */ override def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = super.joinWith(other, condition) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index ad10a22f833bf..c0590fbd1728f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql import java.net.URI import java.nio.file.{Files, Paths} import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.{AtomicLong, AtomicReference} +import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ import scala.reflect.runtime.universe.TypeTag @@ -525,6 +525,8 @@ class SparkSession private[sql] ( } } + override private[sql] def isUsable: Boolean = client.isSessionValid + implicit class RichColumn(c: Column) { def expr: proto.Expression = toExpr(c) def typedExpr[T](e: Encoder[T]): proto.Expression = toTypedExpr(c, e) @@ -533,7 +535,9 @@ class SparkSession private[sql] ( // The minimal builder needed to create a spark session. // TODO: implements all methods mentioned in the scaladoc of [[SparkSession]] -object SparkSession extends api.SparkSessionCompanion with Logging { +object SparkSession extends api.BaseSparkSessionCompanion with Logging { + override private[sql] type Session = SparkSession + private val MAX_CACHED_SESSIONS = 100 private val planIdGenerator = new AtomicLong private var server: Option[Process] = None @@ -549,29 +553,6 @@ object SparkSession extends api.SparkSessionCompanion with Logging { override def load(c: Configuration): SparkSession = create(c) }) - /** The active SparkSession for the current thread. */ - private val activeThreadSession = new InheritableThreadLocal[SparkSession] - - /** Reference to the root SparkSession. */ - private val defaultSession = new AtomicReference[SparkSession] - - /** - * Set the (global) default [[SparkSession]], and (thread-local) active [[SparkSession]] when - * they are not set yet or the associated [[SparkConnectClient]] is unusable. - */ - private def setDefaultAndActiveSession(session: SparkSession): Unit = { - val currentDefault = defaultSession.getAcquire - if (currentDefault == null || !currentDefault.client.isSessionValid) { - // Update `defaultSession` if it is null or the contained session is not valid. There is a - // chance that the following `compareAndSet` fails if a new default session has just been set, - // but that does not matter since that event has happened after this method was invoked. - defaultSession.compareAndSet(currentDefault, session) - } - if (getActiveSession.isEmpty) { - setActiveSession(session) - } - } - /** * Create a new Spark Connect server to connect locally. */ @@ -624,17 +605,6 @@ object SparkSession extends api.SparkSessionCompanion with Logging { new SparkSession(configuration.toSparkConnectClient, planIdGenerator) } - /** - * Hook called when a session is closed. - */ - private[sql] def onSessionClose(session: SparkSession): Unit = { - sessions.invalidate(session.client.configuration) - defaultSession.compareAndSet(session, null) - if (getActiveSession.contains(session)) { - clearActiveSession() - } - } - /** * Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]]. * @@ -781,71 +751,12 @@ object SparkSession extends api.SparkSessionCompanion with Logging { } } - /** - * Returns the default SparkSession. If the previously set default SparkSession becomes - * unusable, returns None. - * - * @since 3.5.0 - */ - def getDefaultSession: Option[SparkSession] = - Option(defaultSession.get()).filter(_.client.isSessionValid) - - /** - * Sets the default SparkSession. - * - * @since 3.5.0 - */ - def setDefaultSession(session: SparkSession): Unit = { - defaultSession.set(session) - } - - /** - * Clears the default SparkSession. - * - * @since 3.5.0 - */ - def clearDefaultSession(): Unit = { - defaultSession.set(null) - } - - /** - * Returns the active SparkSession for the current thread. If the previously set active - * SparkSession becomes unusable, returns None. - * - * @since 3.5.0 - */ - def getActiveSession: Option[SparkSession] = - Option(activeThreadSession.get()).filter(_.client.isSessionValid) - - /** - * Changes the SparkSession that will be returned in this thread and its children when - * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives - * an isolated SparkSession. - * - * @since 3.5.0 - */ - def setActiveSession(session: SparkSession): Unit = { - activeThreadSession.set(session) - } + /** @inheritdoc */ + override def getActiveSession: Option[SparkSession] = super.getActiveSession - /** - * Clears the active SparkSession for current thread. - * - * @since 3.5.0 - */ - def clearActiveSession(): Unit = { - activeThreadSession.remove() - } + /** @inheritdoc */ + override def getDefaultSession: Option[SparkSession] = super.getDefaultSession - /** - * Returns the currently active SparkSession, otherwise the default one. If there is no default - * SparkSession, throws an exception. - * - * @since 3.5.0 - */ - def active: SparkSession = { - getActiveSession - .orElse(getDefaultSession) - .getOrElse(throw new IllegalStateException("No active or default Spark session found")) - } + /** @inheritdoc */ + override def active: SparkSession = super.active } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala index 0e3a683d2701d..ce552bdd4f0f0 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CatalogSuite.scala @@ -69,7 +69,7 @@ class CatalogSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelpe val exception = intercept[SparkException] { spark.catalog.setCurrentCatalog("notExists") } - assert(exception.getErrorClass == "CATALOG_NOT_FOUND") + assert(exception.getCondition == "CATALOG_NOT_FOUND") spark.catalog.setCurrentCatalog("testcat") assert(spark.catalog.currentCatalog().equals("testcat")) val catalogsAfterChange = spark.catalog.listCatalogs().collect() diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CheckpointSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CheckpointSuite.scala index e57b051890f56..0d9685d9c710f 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CheckpointSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/CheckpointSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.apache.spark.SparkException import org.apache.spark.connect.proto import org.apache.spark.sql.test.{ConnectFunSuite, RemoteSparkSession, SQLHelper} +import org.apache.spark.storage.StorageLevel class CheckpointSuite extends ConnectFunSuite with RemoteSparkSession with SQLHelper { @@ -50,12 +51,20 @@ class CheckpointSuite extends ConnectFunSuite with RemoteSparkSession with SQLHe checkFragments(captureStdOut(block), fragmentsToCheck) } - test("checkpoint") { + test("localCheckpoint") { val df = spark.range(100).localCheckpoint() testCapturedStdOut(df.explain(), "ExistingRDD") } - test("checkpoint gc") { + test("localCheckpoint with StorageLevel") { + // We don't have a way to reach into the server and assert the storage level server side, but + // this test should cover for unexpected errors in the API. + val df = + spark.range(100).localCheckpoint(eager = true, storageLevel = StorageLevel.DISK_ONLY) + df.collect() + } + + test("localCheckpoint gc") { val df = spark.range(100).localCheckpoint(eager = true) val encoder = df.agnosticEncoder val dfId = df.plan.getRoot.getCachedRemoteRelation.getRelationId @@ -77,7 +86,7 @@ class CheckpointSuite extends ConnectFunSuite with RemoteSparkSession with SQLHe // This test is flaky because cannot guarantee GC // You can locally run this to verify the behavior. - ignore("checkpoint gc derived DataFrame") { + ignore("localCheckpoint gc derived DataFrame") { var df1 = spark.range(100).localCheckpoint(eager = true) var derived = df1.repartition(10) val encoder = df1.agnosticEncoder diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala index 88281352f2479..84ed624a95214 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala @@ -251,16 +251,16 @@ class ClientDataFrameStatSuite extends ConnectFunSuite with RemoteSparkSession { val error1 = intercept[AnalysisException] { df.stat.bloomFilter("id", -1000, 100) } - assert(error1.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error1.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") val error2 = intercept[AnalysisException] { df.stat.bloomFilter("id", 1000, -100) } - assert(error2.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error2.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") val error3 = intercept[AnalysisException] { df.stat.bloomFilter("id", 1000, -1.0) } - assert(error3.getErrorClass === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") + assert(error3.getCondition === "DATATYPE_MISMATCH.VALUE_OUT_OF_RANGE") } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index b47231948dc98..0371981b728d1 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -95,7 +95,7 @@ class ClientE2ETestSuite .collect() } assert( - ex.getErrorClass === + ex.getCondition === "INCONSISTENT_BEHAVIOR_CROSS_VERSION.PARSE_DATETIME_BY_NEW_PARSER") assert( ex.getMessageParameters.asScala == Map( @@ -122,12 +122,12 @@ class ClientE2ETestSuite Seq("1").toDS().withColumn("udf_val", throwException($"value")).collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.getMessageParameters.isEmpty) assert(ex.getCause.isInstanceOf[SparkException]) val cause = ex.getCause.asInstanceOf[SparkException] - assert(cause.getErrorClass == null) + assert(cause.getCondition == null) assert(cause.getMessageParameters.isEmpty) assert(cause.getMessage.contains("test" * 10000)) } @@ -141,7 +141,7 @@ class ClientE2ETestSuite val ex = intercept[AnalysisException] { spark.sql("select x").collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.messageParameters.isEmpty) assert(ex.getSqlState != null) assert(!ex.isInternalError) @@ -169,14 +169,14 @@ class ClientE2ETestSuite val ex = intercept[NoSuchNamespaceException] { spark.sql("use database123") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } test("table not found for spark.catalog.getTable") { val ex = intercept[AnalysisException] { spark.catalog.getTable("test_table") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } test("throw NamespaceAlreadyExistsException") { @@ -185,7 +185,7 @@ class ClientE2ETestSuite val ex = intercept[NamespaceAlreadyExistsException] { spark.sql("create database test_db") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } finally { spark.sql("drop database test_db") } @@ -197,7 +197,7 @@ class ClientE2ETestSuite val ex = intercept[TempTableAlreadyExistsException] { spark.sql("create temporary view test_view as select 1") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } finally { spark.sql("drop view test_view") } @@ -209,7 +209,7 @@ class ClientE2ETestSuite val ex = intercept[TableAlreadyExistsException] { spark.sql(s"create table testcat.test_table (id int)") } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) } } @@ -217,7 +217,7 @@ class ClientE2ETestSuite val ex = intercept[ParseException] { spark.sql("selet 1").collect() } - assert(ex.getErrorClass != null) + assert(ex.getCondition != null) assert(!ex.messageParameters.isEmpty) assert(ex.getSqlState != null) assert(!ex.isInternalError) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala index 8abc41639fdd2..dec56554d143e 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/SparkSessionSuite.scala @@ -22,6 +22,7 @@ import scala.util.control.NonFatal import io.grpc.{CallOptions, Channel, ClientCall, ClientInterceptor, MethodDescriptor} +import org.apache.spark.SparkException import org.apache.spark.sql.test.ConnectFunSuite import org.apache.spark.util.SparkSerDeUtils @@ -113,7 +114,7 @@ class SparkSessionSuite extends ConnectFunSuite { SparkSession.clearActiveSession() assert(SparkSession.getDefaultSession.isEmpty) assert(SparkSession.getActiveSession.isEmpty) - intercept[IllegalStateException](SparkSession.active) + intercept[SparkException](SparkSession.active) // Create a session val session1 = SparkSession.builder().remote(connectionString1).getOrCreate() diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index c8776af18a14a..693c807ec71ea 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -227,6 +227,8 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.SparkSession.baseRelationToDataFrame"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.createDataset"), ProblemFilters.exclude[Problem]("org.apache.spark.sql.SparkSession.executeCommand"), + ProblemFilters.exclude[DirectMissingMethodProblem]( + "org.apache.spark.sql.SparkSession.canUseSession"), // SparkSession#implicits ProblemFilters.exclude[DirectMissingMethodProblem]( diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala index 46aeaeff43d2f..ac56600392aa3 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala @@ -224,7 +224,7 @@ class SparkConnectClientSuite extends ConnectFunSuite with BeforeAndAfterEach { val error = constructor(testParams).asInstanceOf[Throwable with SparkThrowable] assert(error.getMessage.contains(testParams.message)) assert(error.getCause == null) - assert(error.getErrorClass == testParams.errorClass.get) + assert(error.getCondition == testParams.errorClass.get) assert(error.getMessageParameters.asScala == testParams.messageParameters) assert(error.getQueryContext.isEmpty) } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala index 7176c582d0bbc..10e4c11c406fe 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderSuite.scala @@ -783,7 +783,7 @@ class ArrowEncoderSuite extends ConnectFunSuite with BeforeAndAfterAll { Iterator.tabulate(10)(i => (i, "itr_" + i)) } } - assert(e.getErrorClass == "CANNOT_USE_KRYO") + assert(e.getCondition == "CANNOT_USE_KRYO") } test("transforming encoder") { diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala index 27b1ee014a719..b1a7d81916e92 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/ClientStreamingQuerySuite.scala @@ -331,7 +331,7 @@ class ClientStreamingQuerySuite extends QueryTest with RemoteSparkSession with L query.awaitTermination() } - assert(exception.getErrorClass != null) + assert(exception.getCondition != null) assert(exception.getMessageParameters().get("id") == query.id.toString) assert(exception.getMessageParameters().get("runId") == query.runId.toString) assert(exception.getCause.isInstanceOf[SparkException]) @@ -369,7 +369,7 @@ class ClientStreamingQuerySuite extends QueryTest with RemoteSparkSession with L spark.streams.awaitAnyTermination() } - assert(exception.getErrorClass != null) + assert(exception.getCondition != null) assert(exception.getMessageParameters().get("id") == query.id.toString) assert(exception.getMessageParameters().get("runId") == query.runId.toString) assert(exception.getCause.isInstanceOf[SparkException]) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala index 0a762df465848..d3098fa4f2397 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/PostgresIntegrationSuite.scala @@ -125,6 +125,10 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT "array[array['2022-01-01 09:15'::timestamp]])").executeUpdate() connection.prepareStatement("INSERT INTO array_timestamptz VALUES " + "(array[array['2022-01-01 09:15'::timestamptz]])").executeUpdate() + + connection.prepareStatement( + "CREATE TABLE datetime (name VARCHAR(32), date1 DATE, time1 TIMESTAMP)") + .executeUpdate() } test("Test multi-dimensional column types") { @@ -208,6 +212,14 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT } } + override def dataPreparation(connection: Connection): Unit = { + super.dataPreparation(connection) + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('amy', '2022-05-19', '2022-05-19 00:00:00')").executeUpdate() + connection.prepareStatement("INSERT INTO datetime VALUES " + + "('alex', '2022-05-18', '2022-05-18 00:00:00')").executeUpdate() + } + override def testUpdateColumnType(tbl: String): Unit = { sql(s"CREATE TABLE $tbl (ID INTEGER)") var t = spark.table(tbl) @@ -264,4 +276,77 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT ) } } + + override def testDatetime(tbl: String): Unit = { + val df1 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 AND dayofmonth(date1) > 10 ") + checkFilterPushed(df1) + val rows1 = df1.collect() + assert(rows1.length === 2) + assert(rows1(0).getString(0) === "amy") + assert(rows1(1).getString(0) === "alex") + + val df2 = sql(s"SELECT name FROM $tbl WHERE year(date1) = 2022 AND quarter(date1) = 2") + checkFilterPushed(df2) + val rows2 = df2.collect() + assert(rows2.length === 2) + assert(rows2(0).getString(0) === "amy") + assert(rows2(1).getString(0) === "alex") + + val df3 = sql(s"SELECT name FROM $tbl WHERE second(time1) = 0 AND month(date1) = 5") + checkFilterPushed(df3) + val rows3 = df3.collect() + assert(rows3.length === 2) + assert(rows3(0).getString(0) === "amy") + assert(rows3(1).getString(0) === "alex") + + val df4 = sql(s"SELECT name FROM $tbl WHERE hour(time1) = 0 AND minute(time1) = 0") + checkFilterPushed(df4) + val rows4 = df4.collect() + assert(rows4.length === 2) + assert(rows4(0).getString(0) === "amy") + assert(rows4(1).getString(0) === "alex") + + val df5 = sql(s"SELECT name FROM $tbl WHERE " + + "extract(WEEk from date1) > 10 AND extract(YEAROFWEEK from date1) = 2022") + checkFilterPushed(df5) + val rows5 = df5.collect() + assert(rows5.length === 2) + assert(rows5(0).getString(0) === "amy") + assert(rows5(1).getString(0) === "alex") + + val df6 = sql(s"SELECT name FROM $tbl WHERE date_add(date1, 1) = date'2022-05-20' " + + "AND datediff(date1, '2022-05-10') > 0") + checkFilterPushed(df6, false) + val rows6 = df6.collect() + assert(rows6.length === 1) + assert(rows6(0).getString(0) === "amy") + + val df7 = sql(s"SELECT name FROM $tbl WHERE weekday(date1) = 2") + checkFilterPushed(df7) + val rows7 = df7.collect() + assert(rows7.length === 1) + assert(rows7(0).getString(0) === "alex") + + val df8 = sql(s"SELECT name FROM $tbl WHERE dayofweek(date1) = 4") + checkFilterPushed(df8) + val rows8 = df8.collect() + assert(rows8.length === 1) + assert(rows8(0).getString(0) === "alex") + + val df9 = sql(s"SELECT name FROM $tbl WHERE " + + "dayofyear(date1) > 100 order by dayofyear(date1) limit 1") + checkFilterPushed(df9) + val rows9 = df9.collect() + assert(rows9.length === 1) + assert(rows9(0).getString(0) === "alex") + + // Postgres does not support + val df10 = sql(s"SELECT name FROM $tbl WHERE trunc(date1, 'week') = date'2022-05-16'") + checkFilterPushed(df10, false) + val rows10 = df10.collect() + assert(rows10.length === 2) + assert(rows10(0).getString(0) === "amy") + assert(rows10(1).getString(0) === "alex") + } } diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala index 13a68e72269f0..c4adb6b3f26e1 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaExceptions.scala @@ -184,5 +184,5 @@ private[kafka010] class KafkaIllegalStateException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java index 8ec5c2221b6e9..fa71eb066ff89 100644 --- a/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java +++ b/core/src/main/java/org/apache/spark/memory/SparkOutOfMemoryError.java @@ -52,7 +52,7 @@ public Map getMessageParameters() { } @Override - public String getErrorClass() { + public String getCondition() { return errorClass; } } diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3b7c7778e26ce..573608c4327e0 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -173,7 +173,7 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( } private def canShuffleMergeBeEnabled(): Boolean = { - val isPushShuffleEnabled = Utils.isPushBasedShuffleEnabled(rdd.sparkContext.getConf, + val isPushShuffleEnabled = Utils.isPushBasedShuffleEnabled(rdd.sparkContext.conf, // invoked at driver isDriver = true) if (isPushShuffleEnabled && rdd.isBarrier()) { diff --git a/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala b/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala index 0e578f045452e..82a0261f32ae7 100644 --- a/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala +++ b/core/src/main/scala/org/apache/spark/SparkFileAlreadyExistsException.scala @@ -33,5 +33,5 @@ private[spark] class SparkFileAlreadyExistsException( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass } diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index f0d6cba6ae734..3c3017a9a64c1 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -45,8 +45,8 @@ private[spark] abstract class StreamFileInputFormat[T] * which is set through setMaxSplitSize */ def setMinPartitions(sc: SparkContext, context: JobContext, minPartitions: Int): Unit = { - val defaultMaxSplitBytes = sc.getConf.get(config.FILES_MAX_PARTITION_BYTES) - val openCostInBytes = sc.getConf.get(config.FILES_OPEN_COST_IN_BYTES) + val defaultMaxSplitBytes = sc.conf.get(config.FILES_MAX_PARTITION_BYTES) + val openCostInBytes = sc.conf.get(config.FILES_OPEN_COST_IN_BYTES) val defaultParallelism = Math.max(sc.defaultParallelism, minPartitions) val files = listStatus(context).asScala val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 2c89fe7885d08..4f7338f74e298 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -174,7 +174,7 @@ private[spark] class DAGScheduler( // `NUM_CANCELLED_JOB_GROUPS_TO_TRACK` stored. On a new job submission, if its job group is in // this set, the job will be immediately cancelled. private[scheduler] val cancelledJobGroups = - new LimitedSizeFIFOSet[String](sc.getConf.get(config.NUM_CANCELLED_JOB_GROUPS_TO_TRACK)) + new LimitedSizeFIFOSet[String](sc.conf.get(config.NUM_CANCELLED_JOB_GROUPS_TO_TRACK)) /** * Contains the locations that each RDD's partitions are cached on. This map's keys are RDD ids @@ -224,9 +224,9 @@ private[spark] class DAGScheduler( private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ - private val disallowStageRetryForTest = sc.getConf.get(TEST_NO_STAGE_RETRY) + private val disallowStageRetryForTest = sc.conf.get(TEST_NO_STAGE_RETRY) - private val shouldMergeResourceProfiles = sc.getConf.get(config.RESOURCE_PROFILE_MERGE_CONFLICTS) + private val shouldMergeResourceProfiles = sc.conf.get(config.RESOURCE_PROFILE_MERGE_CONFLICTS) /** * Whether to unregister all the outputs on the host in condition that we receive a FetchFailure, @@ -234,19 +234,19 @@ private[spark] class DAGScheduler( * executor(instead of the host) on a FetchFailure. */ private[scheduler] val unRegisterOutputOnHostOnFetchFailure = - sc.getConf.get(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE) + sc.conf.get(config.UNREGISTER_OUTPUT_ON_HOST_ON_FETCH_FAILURE) /** * Number of consecutive stage attempts allowed before a stage is aborted. */ private[scheduler] val maxConsecutiveStageAttempts = - sc.getConf.get(config.STAGE_MAX_CONSECUTIVE_ATTEMPTS) + sc.conf.get(config.STAGE_MAX_CONSECUTIVE_ATTEMPTS) /** * Max stage attempts allowed before a stage is aborted. */ private[scheduler] val maxStageAttempts: Int = { - Math.max(maxConsecutiveStageAttempts, sc.getConf.get(config.STAGE_MAX_ATTEMPTS)) + Math.max(maxConsecutiveStageAttempts, sc.conf.get(config.STAGE_MAX_ATTEMPTS)) } /** @@ -254,7 +254,7 @@ private[spark] class DAGScheduler( * count spark.stage.maxConsecutiveAttempts */ private[scheduler] val ignoreDecommissionFetchFailure = - sc.getConf.get(config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE) + sc.conf.get(config.STAGE_IGNORE_DECOMMISSION_FETCH_FAILURE) /** * Number of max concurrent tasks check failures for each barrier job. @@ -264,14 +264,14 @@ private[spark] class DAGScheduler( /** * Time in seconds to wait between a max concurrent tasks check failure and the next check. */ - private val timeIntervalNumTasksCheck = sc.getConf + private val timeIntervalNumTasksCheck = sc.conf .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL) /** * Max number of max concurrent tasks check failures allowed for a job before fail the job * submission. */ - private val maxFailureNumTasksCheck = sc.getConf + private val maxFailureNumTasksCheck = sc.conf .get(config.BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES) private val messageScheduler = @@ -286,26 +286,26 @@ private[spark] class DAGScheduler( taskScheduler.setDAGScheduler(this) - private val pushBasedShuffleEnabled = Utils.isPushBasedShuffleEnabled(sc.getConf, isDriver = true) + private val pushBasedShuffleEnabled = Utils.isPushBasedShuffleEnabled(sc.conf, isDriver = true) private val blockManagerMasterDriverHeartbeatTimeout = - sc.getConf.get(config.STORAGE_BLOCKMANAGER_MASTER_DRIVER_HEARTBEAT_TIMEOUT).millis + sc.conf.get(config.STORAGE_BLOCKMANAGER_MASTER_DRIVER_HEARTBEAT_TIMEOUT).millis private val shuffleMergeResultsTimeoutSec = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_RESULTS_TIMEOUT) private val shuffleMergeFinalizeWaitSec = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_TIMEOUT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_TIMEOUT) private val shuffleMergeWaitMinSizeThreshold = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT) + sc.conf.get(config.PUSH_BASED_SHUFFLE_SIZE_MIN_SHUFFLE_SIZE_TO_WAIT) - private val shufflePushMinRatio = sc.getConf.get(config.PUSH_BASED_SHUFFLE_MIN_PUSH_RATIO) + private val shufflePushMinRatio = sc.conf.get(config.PUSH_BASED_SHUFFLE_MIN_PUSH_RATIO) private val shuffleMergeFinalizeNumThreads = - sc.getConf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS) + sc.conf.get(config.PUSH_BASED_SHUFFLE_MERGE_FINALIZE_THREADS) - private val shuffleFinalizeRpcThreads = sc.getConf.get(config.PUSH_SHUFFLE_FINALIZE_RPC_THREADS) + private val shuffleFinalizeRpcThreads = sc.conf.get(config.PUSH_SHUFFLE_FINALIZE_RPC_THREADS) // Since SparkEnv gets initialized after DAGScheduler, externalShuffleClient needs to be // initialized lazily @@ -328,11 +328,10 @@ private[spark] class DAGScheduler( ThreadUtils.newDaemonFixedThreadPool(shuffleFinalizeRpcThreads, "shuffle-merge-finalize-rpc") /** Whether rdd cache visibility tracking is enabled. */ - private val trackingCacheVisibility: Boolean = - sc.getConf.get(RDD_CACHE_VISIBILITY_TRACKING_ENABLED) + private val trackingCacheVisibility: Boolean = sc.conf.get(RDD_CACHE_VISIBILITY_TRACKING_ENABLED) /** Whether to abort a stage after canceling all of its tasks. */ - private val legacyAbortStageAfterKillTasks = sc.getConf.get(LEGACY_ABORT_STAGE_AFTER_KILL_TASKS) + private val legacyAbortStageAfterKillTasks = sc.conf.get(LEGACY_ABORT_STAGE_AFTER_KILL_TASKS) /** * Called by the TaskSetManager to report task's starting. @@ -557,7 +556,7 @@ private[spark] class DAGScheduler( * TODO SPARK-24942 Improve cluster resource management with jobs containing barrier stage */ private def checkBarrierStageWithDynamicAllocation(rdd: RDD[_]): Unit = { - if (rdd.isBarrier() && Utils.isDynamicAllocationEnabled(sc.getConf)) { + if (rdd.isBarrier() && Utils.isDynamicAllocationEnabled(sc.conf)) { throw SparkCoreErrors.barrierStageWithDynamicAllocationError() } } @@ -2163,7 +2162,7 @@ private[spark] class DAGScheduler( case mapStage: ShuffleMapStage => val numMissingPartitions = mapStage.findMissingPartitions().length if (numMissingPartitions < mapStage.numTasks) { - if (sc.getConf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { + if (sc.conf.get(config.SHUFFLE_USE_OLD_FETCH_PROTOCOL)) { val reason = "A shuffle map stage with indeterminate output was failed " + "and retried. However, Spark can only do this while using the new " + "shuffle block fetching protocol. Please check the config " + @@ -2893,8 +2892,8 @@ private[spark] class DAGScheduler( val finalException = exception.collect { // If the error is user-facing (defines error class and is not internal error), we don't // wrap it with "Job aborted" and expose this error to the end users directly. - case st: Exception with SparkThrowable if st.getErrorClass != null && - !SparkThrowableHelper.isInternalError(st.getErrorClass) => + case st: Exception with SparkThrowable if st.getCondition != null && + !SparkThrowableHelper.isInternalError(st.getCondition) => st }.getOrElse { new SparkException(s"Job aborted due to stage failure: $reason", cause = exception.orNull) diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 7a2b7d9caec42..fc7a4675429aa 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -35,7 +35,7 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { // Carriage return private val CR = '\r' // Update period of progress bar, in milliseconds - private val updatePeriodMSec = sc.getConf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) + private val updatePeriodMSec = sc.conf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) // Delay to show up a progress bar, in milliseconds private val firstDelayMSec = 500L diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 380231ce97c0b..ca51e61f5ed44 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -288,7 +288,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft sem.acquire(1) sc.cancelJobGroupAndFutureJobs(s"job-group-$idx") ThreadUtils.awaitReady(job, Duration.Inf).failed.foreach { case e: SparkException => - assert(e.getErrorClass == "SPARK_JOB_CANCELLED") + assert(e.getCondition == "SPARK_JOB_CANCELLED") } } // submit a job with the 0 job group that was evicted from cancelledJobGroups set, it should run diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 9f310c06ac5ae..e38efc27b78f9 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -343,7 +343,7 @@ abstract class SparkFunSuite parameters: Map[String, String] = Map.empty, matchPVals: Boolean = false, queryContext: Array[ExpectedContext] = Array.empty): Unit = { - assert(exception.getErrorClass === condition) + assert(exception.getCondition === condition) sqlState.foreach(state => assert(exception.getSqlState === state)) val expectedParameters = exception.getMessageParameters.asScala if (matchPVals) { diff --git a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala index 946ea75686e32..9f005e5757193 100644 --- a/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkThrowableSuite.scala @@ -199,7 +199,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { getMessage("UNRESOLVED_COLUMN.WITHOUT_SUGGESTION", Map.empty[String, String]) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessageParameters().get("message").contains("Undefined error message parameter")) } @@ -245,7 +245,7 @@ class SparkThrowableSuite extends SparkFunSuite { throw new SparkException("Arbitrary legacy message") } catch { case e: SparkThrowable => - assert(e.getErrorClass == null) + assert(e.getCondition == null) assert(!e.isInternalError) assert(e.getSqlState == null) case _: Throwable => @@ -262,7 +262,7 @@ class SparkThrowableSuite extends SparkFunSuite { cause = null) } catch { case e: SparkThrowable => - assert(e.getErrorClass == "CANNOT_PARSE_DECIMAL") + assert(e.getCondition == "CANNOT_PARSE_DECIMAL") assert(!e.isInternalError) assert(e.getSqlState == "22018") case _: Throwable => @@ -357,7 +357,7 @@ class SparkThrowableSuite extends SparkFunSuite { |}""".stripMargin) // Legacy mode when an exception does not have any error class class LegacyException extends Throwable with SparkThrowable { - override def getErrorClass: String = null + override def getCondition: String = null override def getMessage: String = "Test message" } val e3 = new LegacyException @@ -452,7 +452,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { new ErrorClassesJsonReader(Seq(errorJsonFilePath.toUri.toURL, json.toURI.toURL)) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessage.contains("DIVIDE.BY_ZERO")) } @@ -478,7 +478,7 @@ class SparkThrowableSuite extends SparkFunSuite { val e = intercept[SparkException] { new ErrorClassesJsonReader(Seq(errorJsonFilePath.toUri.toURL, json.toURI.toURL)) } - assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getCondition === "INTERNAL_ERROR") assert(e.getMessage.contains("BY.ZERO")) } } diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 1efef3383b821..b0f36b9744fa8 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -317,13 +317,13 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio // Instead, crash the driver by directly accessing the broadcast value. val e1 = intercept[SparkException] { broadcast.value } assert(e1.isInternalError) - assert(e1.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e1.getCondition == "INTERNAL_ERROR_BROADCAST") val e2 = intercept[SparkException] { broadcast.unpersist(blocking = true) } assert(e2.isInternalError) - assert(e2.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e2.getCondition == "INTERNAL_ERROR_BROADCAST") val e3 = intercept[SparkException] { broadcast.destroy(blocking = true) } assert(e3.isInternalError) - assert(e3.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(e3.getCondition == "INTERNAL_ERROR_BROADCAST") } else { val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) @@ -339,7 +339,7 @@ package object testPackage extends Assertions { val thrown = intercept[SparkException] { broadcast.value } assert(thrown.getMessage.contains("BroadcastSuite.scala")) assert(thrown.isInternalError) - assert(thrown.getErrorClass == "INTERNAL_ERROR_BROADCAST") + assert(thrown.getCondition == "INTERNAL_ERROR_BROADCAST") } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 30f3e4c4af021..5486c39034fd3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -204,7 +204,7 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) val inputType = try { SchemaUtils.getSchemaFieldType(schema, inputColName) } catch { - case e: SparkIllegalArgumentException if e.getErrorClass == "FIELD_NOT_FOUND" => + case e: SparkIllegalArgumentException if e.getCondition == "FIELD_NOT_FOUND" => throw new SparkException(s"Input column $inputColName does not exist.") case e: Exception => throw e diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 8e64f60427d90..20b03edf23c4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -127,7 +127,7 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi validateAndTransformField(schema, inputColName, dtype, outputColName) ) } catch { - case e: SparkIllegalArgumentException if e.getErrorClass == "FIELD_NOT_FOUND" => + case e: SparkIllegalArgumentException if e.getCondition == "FIELD_NOT_FOUND" => if (skipNonExistsCol) { None } else { diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2b3d76eb0c2c3..f31a29788aafe 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -189,6 +189,15 @@ object MimaExcludes { ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.javalang.typed"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.scalalang.typed"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.expressions.scalalang.typed$"), + + // SPARK-49418: Consolidate thread local handling in sql/api + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.SparkSession.setActiveSession"), + ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.sql.SparkSession.setDefaultSession"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.api.SparkSessionCompanion.clearActiveSession"), + ProblemFilters.exclude[DirectAbstractMethodProblem]("org.apache.spark.sql.api.SparkSessionCompanion.clearDefaultSession"), + + // SPARK-49748: Add getCondition and deprecate getErrorClass in SparkThrowable + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.SparkThrowable.getCondition"), ) ++ loggingExcludes("org.apache.spark.sql.DataFrameReader") ++ loggingExcludes("org.apache.spark.sql.streaming.DataStreamReader") ++ loggingExcludes("org.apache.spark.sql.SparkSession#Builder") diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index e412b98c47de5..91dec609e522a 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -360,8 +360,13 @@ def checkpoint(self, eager: bool = True) -> ParentDataFrame: jdf = self._jdf.checkpoint(eager) return DataFrame(jdf, self.sparkSession) - def localCheckpoint(self, eager: bool = True) -> ParentDataFrame: - jdf = self._jdf.localCheckpoint(eager) + def localCheckpoint( + self, eager: bool = True, storageLevel: Optional[StorageLevel] = None + ) -> ParentDataFrame: + if storageLevel is None: + jdf = self._jdf.localCheckpoint(eager) + else: + jdf = self._jdf.localCheckpoint(eager, self._sc._getJavaStorageLevel(storageLevel)) return DataFrame(jdf, self.sparkSession) def withWatermark(self, eventTime: str, delayThreshold: str) -> ParentDataFrame: diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index bb4dcb38c9e58..3d5b845fcd24c 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -2134,8 +2134,10 @@ def checkpoint(self, eager: bool = True) -> ParentDataFrame: assert isinstance(checkpointed._plan, plan.CachedRemoteRelation) return checkpointed - def localCheckpoint(self, eager: bool = True) -> ParentDataFrame: - cmd = plan.Checkpoint(child=self._plan, local=True, eager=eager) + def localCheckpoint( + self, eager: bool = True, storageLevel: Optional[StorageLevel] = None + ) -> ParentDataFrame: + cmd = plan.Checkpoint(child=self._plan, local=True, eager=eager, storage_level=storageLevel) _, properties, self._execution_info = self._session.client.execute_command( cmd.command(self._session.client) ) diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index fbed0eabc684f..b74f863db1e83 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -1868,21 +1868,29 @@ def command(self, session: "SparkConnectClient") -> proto.Command: class Checkpoint(LogicalPlan): - def __init__(self, child: Optional["LogicalPlan"], local: bool, eager: bool) -> None: + def __init__( + self, + child: Optional["LogicalPlan"], + local: bool, + eager: bool, + storage_level: Optional[StorageLevel] = None, + ) -> None: super().__init__(child) self._local = local self._eager = eager + self._storage_level = storage_level def command(self, session: "SparkConnectClient") -> proto.Command: cmd = proto.Command() assert self._child is not None - cmd.checkpoint_command.CopyFrom( - proto.CheckpointCommand( - relation=self._child.plan(session), - local=self._local, - eager=self._eager, - ) + checkpoint_command = proto.CheckpointCommand( + relation=self._child.plan(session), + local=self._local, + eager=self._eager, ) + if self._storage_level is not None: + checkpoint_command.storage_level.CopyFrom(storage_level_to_proto(self._storage_level)) + cmd.checkpoint_command.CopyFrom(checkpoint_command) return cmd diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 43390ffa36d33..562e9d817f5fe 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\x90\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"t\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\x90\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xd8\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x12\x36\n\x17\x63lustering_column_names\x18\x0f \x03(\tR\x15\x63lusteringColumnNames\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"\xcd\x01\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger\x12\x45\n\rstorage_level\x18\x04 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -71,8 +71,8 @@ _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_options = b"8\001" _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._options = None _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_options = b"8\001" - _STREAMINGQUERYEVENTTYPE._serialized_start = 11162 - _STREAMINGQUERYEVENTTYPE._serialized_end = 11295 + _STREAMINGQUERYEVENTTYPE._serialized_start = 11252 + _STREAMINGQUERYEVENTTYPE._serialized_end = 11385 _COMMAND._serialized_start = 167 _COMMAND._serialized_end = 1847 _SQLCOMMAND._serialized_start = 1850 @@ -167,8 +167,8 @@ _CREATERESOURCEPROFILECOMMANDRESULT._serialized_end = 10448 _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_start = 10450 _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_end = 10550 - _CHECKPOINTCOMMAND._serialized_start = 10552 - _CHECKPOINTCOMMAND._serialized_end = 10668 - _MERGEINTOTABLECOMMAND._serialized_start = 10671 - _MERGEINTOTABLECOMMAND._serialized_end = 11159 + _CHECKPOINTCOMMAND._serialized_start = 10553 + _CHECKPOINTCOMMAND._serialized_end = 10758 + _MERGEINTOTABLECOMMAND._serialized_start = 10761 + _MERGEINTOTABLECOMMAND._serialized_end = 11249 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 2dedcdfc8e3e4..6192a29607cbf 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -2188,6 +2188,7 @@ class CheckpointCommand(google.protobuf.message.Message): RELATION_FIELD_NUMBER: builtins.int LOCAL_FIELD_NUMBER: builtins.int EAGER_FIELD_NUMBER: builtins.int + STORAGE_LEVEL_FIELD_NUMBER: builtins.int @property def relation(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: """(Required) The logical plan to checkpoint.""" @@ -2197,22 +2198,46 @@ class CheckpointCommand(google.protobuf.message.Message): """ eager: builtins.bool """(Required) Whether to checkpoint this dataframe immediately.""" + @property + def storage_level(self) -> pyspark.sql.connect.proto.common_pb2.StorageLevel: + """(Optional) For local checkpoint, the storage level to use.""" def __init__( self, *, relation: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., local: builtins.bool = ..., eager: builtins.bool = ..., + storage_level: pyspark.sql.connect.proto.common_pb2.StorageLevel | None = ..., ) -> None: ... def HasField( - self, field_name: typing_extensions.Literal["relation", b"relation"] + self, + field_name: typing_extensions.Literal[ + "_storage_level", + b"_storage_level", + "relation", + b"relation", + "storage_level", + b"storage_level", + ], ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ - "eager", b"eager", "local", b"local", "relation", b"relation" + "_storage_level", + b"_storage_level", + "eager", + b"eager", + "local", + b"local", + "relation", + b"relation", + "storage_level", + b"storage_level", ], ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_storage_level", b"_storage_level"] + ) -> typing_extensions.Literal["storage_level"] | None: ... global___CheckpointCommand = CheckpointCommand diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c21e2271a64ac..62f2129e5be62 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -1017,7 +1017,9 @@ def checkpoint(self, eager: bool = True) -> "DataFrame": """ ... - def localCheckpoint(self, eager: bool = True) -> "DataFrame": + def localCheckpoint( + self, eager: bool = True, storageLevel: Optional[StorageLevel] = None + ) -> "DataFrame": """Returns a locally checkpointed version of this :class:`DataFrame`. Checkpointing can be used to truncate the logical plan of this :class:`DataFrame`, which is especially useful in iterative algorithms where the plan may grow exponentially. Local checkpoints @@ -1028,12 +1030,17 @@ def localCheckpoint(self, eager: bool = True) -> "DataFrame": .. versionchanged:: 4.0.0 Supports Spark Connect. + Added storageLevel parameter. Parameters ---------- eager : bool, optional, default True Whether to checkpoint this :class:`DataFrame` immediately. + storageLevel : :class:`StorageLevel`, optional, default None + The StorageLevel with which the checkpoint will be stored. + If not specified, default for RDD local checkpoints. + Returns ------- :class:`DataFrame` diff --git a/python/pyspark/sql/tests/connect/test_parity_readwriter.py b/python/pyspark/sql/tests/connect/test_parity_readwriter.py index 46333b555c351..f83f3edbfa787 100644 --- a/python/pyspark/sql/tests/connect/test_parity_readwriter.py +++ b/python/pyspark/sql/tests/connect/test_parity_readwriter.py @@ -33,6 +33,7 @@ def test_api(self): def test_partitioning_functions(self): self.check_partitioning_functions(DataFrameWriterV2) + self.partitioning_functions_user_error() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index b5af00a4e7b78..2f53ca38743c1 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -15,6 +15,8 @@ # limitations under the License. # +import glob +import os import pydoc import shutil import tempfile @@ -47,6 +49,7 @@ pandas_requirement_message, pyarrow_requirement_message, ) +from pyspark.testing.utils import SPARK_HOME class DataFrameTestsMixin: @@ -506,14 +509,16 @@ def test_toDF_with_schema_string(self): # number of fields must match. self.assertRaisesRegex( - Exception, "FIELD_STRUCT_LENGTH_MISMATCH", lambda: rdd.toDF("key: int").collect() + Exception, + "FIELD_STRUCT_LENGTH_MISMATCH", + lambda: rdd.coalesce(1).toDF("key: int").collect(), ) # field types mismatch will cause exception at runtime. self.assertRaisesRegex( Exception, "FIELD_DATA_TYPE_UNACCEPTABLE", - lambda: rdd.toDF("key: float, value: string").collect(), + lambda: rdd.coalesce(1).toDF("key: float, value: string").collect(), ) # flat schema values will be wrapped into row. @@ -777,6 +782,16 @@ def test_df_show(self): ) def test_df_merge_into(self): + filename_pattern = ( + "sql/catalyst/target/scala-*/test-classes/org/apache/spark/sql/connector/catalog/" + "InMemoryRowLevelOperationTableCatalog.class" + ) + if not bool(glob.glob(os.path.join(SPARK_HOME, filename_pattern))): + raise unittest.SkipTest( + "org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTableCatalog' " + "is not available. Will skip the related tests" + ) + try: # InMemoryRowLevelOperationTableCatalog is a test catalog that is included in the # catalyst-test package. If Spark complains that it can't find this class, make sure @@ -950,11 +965,17 @@ def test_union_classmethod_usage(self): def test_isinstance_dataframe(self): self.assertIsInstance(self.spark.range(1), DataFrame) - def test_checkpoint_dataframe(self): + def test_local_checkpoint_dataframe(self): with io.StringIO() as buf, redirect_stdout(buf): self.spark.range(1).localCheckpoint().explain() self.assertIn("ExistingRDD", buf.getvalue()) + def test_local_checkpoint_dataframe_with_storage_level(self): + # We don't have a way to reach into the server and assert the storage level server side, but + # this test should cover for unexpected errors in the API. + df = self.spark.range(10).localCheckpoint(eager=True, storageLevel=StorageLevel.DISK_ONLY) + df.collect() + def test_transpose(self): df = self.spark.createDataFrame([{"a": "x", "b": "y", "c": "z"}]) diff --git a/python/pyspark/sql/tests/test_readwriter.py b/python/pyspark/sql/tests/test_readwriter.py index f4f32dea9060a..2fca6b57decf9 100644 --- a/python/pyspark/sql/tests/test_readwriter.py +++ b/python/pyspark/sql/tests/test_readwriter.py @@ -255,6 +255,7 @@ def check_api(self, tpe): def test_partitioning_functions(self): self.check_partitioning_functions(DataFrameWriterV2) + self.partitioning_functions_user_error() def check_partitioning_functions(self, tpe): import datetime @@ -274,6 +275,35 @@ def check_partitioning_functions(self, tpe): self.assertIsInstance(writer.partitionedBy(bucket(11, col("id"))), tpe) self.assertIsInstance(writer.partitionedBy(bucket(3, "id"), hours(col("ts"))), tpe) + def partitioning_functions_user_error(self): + import datetime + from pyspark.sql.functions.partitioning import years, months, days, hours, bucket + + df = self.spark.createDataFrame( + [(1, datetime.datetime(2000, 1, 1), "foo")], ("id", "ts", "value") + ) + + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(years("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(months("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(days("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(hours("ts")).collect() + with self.assertRaisesRegex( + Exception, "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY" + ): + df.select(bucket(2, "ts")).collect() + def test_create(self): df = self.df with self.table("test_table"): diff --git a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala index a2c1f2cc41f8f..51825ee1a5bed 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -139,7 +139,7 @@ class AnalysisException protected ( override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava - override def getErrorClass: String = errorClass.orNull + override def getCondition: String = errorClass.orNull override def getQueryContext: Array[QueryContext] = context diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala index c277b4cab85c1..d6442930d1c5c 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/Dataset.scala @@ -314,7 +314,8 @@ abstract class Dataset[T] extends Serializable { * @group basic * @since 2.1.0 */ - def checkpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = true) + def checkpoint(): Dataset[T] = + checkpoint(eager = true, reliableCheckpoint = true, storageLevel = None) /** * Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the @@ -334,7 +335,7 @@ abstract class Dataset[T] extends Serializable { * @since 2.1.0 */ def checkpoint(eager: Boolean): Dataset[T] = - checkpoint(eager = eager, reliableCheckpoint = true) + checkpoint(eager = eager, reliableCheckpoint = true, storageLevel = None) /** * Eagerly locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used @@ -345,7 +346,8 @@ abstract class Dataset[T] extends Serializable { * @group basic * @since 2.3.0 */ - def localCheckpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = false) + def localCheckpoint(): Dataset[T] = + checkpoint(eager = true, reliableCheckpoint = false, storageLevel = None) /** * Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to @@ -365,7 +367,29 @@ abstract class Dataset[T] extends Serializable { * @since 2.3.0 */ def localCheckpoint(eager: Boolean): Dataset[T] = - checkpoint(eager = eager, reliableCheckpoint = false) + checkpoint(eager = eager, reliableCheckpoint = false, storageLevel = None) + + /** + * Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to + * truncate the logical plan of this Dataset, which is especially useful in iterative algorithms + * where the plan may grow exponentially. Local checkpoints are written to executor storage and + * despite potentially faster they are unreliable and may compromise job completion. + * + * @param eager + * Whether to checkpoint this dataframe immediately + * @param storageLevel + * StorageLevel with which to checkpoint the data. + * @note + * When checkpoint is used with eager = false, the final data that is checkpointed after the + * first action may be different from the data that was used during the job due to + * non-determinism of the underlying operation and retries. If checkpoint is used to achieve + * saving a deterministic snapshot of the data, eager = true should be used. Otherwise, it is + * only deterministic after the first execution, after the checkpoint was finalized. + * @group basic + * @since 4.0.0 + */ + def localCheckpoint(eager: Boolean, storageLevel: StorageLevel): Dataset[T] = + checkpoint(eager = eager, reliableCheckpoint = false, storageLevel = Some(storageLevel)) /** * Returns a checkpointed version of this Dataset. @@ -375,8 +399,14 @@ abstract class Dataset[T] extends Serializable { * @param reliableCheckpoint * Whether to create a reliable checkpoint saved to files inside the checkpoint directory. If * false creates a local checkpoint using the caching subsystem - */ - protected def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): Dataset[T] + * @param storageLevel + * Option. If defined, StorageLevel with which to checkpoint the data. Only with + * reliableCheckpoint = false. + */ + protected def checkpoint( + eager: Boolean, + reliableCheckpoint: Boolean, + storageLevel: Option[StorageLevel]): Dataset[T] /** * Defines an event time watermark for this [[Dataset]]. A watermark tracks a point in time diff --git a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala index b2e61df5937bd..31ceecb9e4ca5 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/api/SparkSession.scala @@ -24,8 +24,9 @@ import _root_.java.io.Closeable import _root_.java.lang import _root_.java.net.URI import _root_.java.util +import _root_.java.util.concurrent.atomic.AtomicReference -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD @@ -663,9 +664,19 @@ abstract class SparkSession extends Serializable with Closeable { * @since 2.0.0 */ def stop(): Unit = close() + + /** + * Check to see if the session is still usable. + * + * In Classic this means that the underlying `SparkContext` is still active. In Connect this + * means the connection to the server is usable. + */ + private[sql] def isUsable: Boolean } object SparkSession extends SparkSessionCompanion { + type Session = SparkSession + private[this] val companion: SparkSessionCompanion = { val cls = SparkClassUtils.classForName("org.apache.spark.sql.SparkSession") val mirror = scala.reflect.runtime.currentMirror @@ -675,12 +686,97 @@ object SparkSession extends SparkSessionCompanion { /** @inheritdoc */ override def builder(): SparkSessionBuilder = companion.builder() + + /** @inheritdoc */ + override def setActiveSession(session: SparkSession): Unit = + companion.setActiveSession(session.asInstanceOf[companion.Session]) + + /** @inheritdoc */ + override def clearActiveSession(): Unit = companion.clearActiveSession() + + /** @inheritdoc */ + override def setDefaultSession(session: SparkSession): Unit = + companion.setDefaultSession(session.asInstanceOf[companion.Session]) + + /** @inheritdoc */ + override def clearDefaultSession(): Unit = companion.clearDefaultSession() + + /** @inheritdoc */ + override def getActiveSession: Option[SparkSession] = companion.getActiveSession + + /** @inheritdoc */ + override def getDefaultSession: Option[SparkSession] = companion.getDefaultSession } /** - * Companion of a [[SparkSession]]. + * Interface for a [[SparkSession]] Companion. The companion is responsible for building the + * session, and managing the active (thread local) and default (global) SparkSessions. */ private[sql] abstract class SparkSessionCompanion { + private[sql] type Session <: SparkSession + + /** + * Changes the SparkSession that will be returned in this thread and its children when + * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives + * a SparkSession with an isolated session, instead of the global (first created) context. + * + * @since 2.0.0 + */ + def setActiveSession(session: Session): Unit + + /** + * Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will + * return the first created context instead of a thread-local override. + * + * @since 2.0.0 + */ + def clearActiveSession(): Unit + + /** + * Sets the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def setDefaultSession(session: Session): Unit + + /** + * Clears the default SparkSession that is returned by the builder. + * + * @since 2.0.0 + */ + def clearDefaultSession(): Unit + + /** + * Returns the active SparkSession for the current thread, returned by the builder. + * + * @note + * Return None, when calling this function on executors + * + * @since 2.2.0 + */ + def getActiveSession: Option[Session] + + /** + * Returns the default SparkSession that is returned by the builder. + * + * @note + * Return None, when calling this function on executors + * + * @since 2.2.0 + */ + def getDefaultSession: Option[Session] + + /** + * Returns the currently active SparkSession, otherwise the default one. If there is no default + * SparkSession, throws an exception. + * + * @since 2.4.0 + */ + def active: Session = { + getActiveSession.getOrElse( + getDefaultSession.getOrElse( + throw SparkException.internalError("No active or default Spark session found"))) + } /** * Creates a [[SparkSessionBuilder]] for constructing a [[SparkSession]]. @@ -690,6 +786,83 @@ private[sql] abstract class SparkSessionCompanion { def builder(): SparkSessionBuilder } +/** + * Abstract class for [[SparkSession]] companions. This implements active and default session + * management. + */ +private[sql] abstract class BaseSparkSessionCompanion extends SparkSessionCompanion { + + /** The active SparkSession for the current thread. */ + private val activeThreadSession = new InheritableThreadLocal[Session] + + /** Reference to the root SparkSession. */ + private val defaultSession = new AtomicReference[Session] + + /** @inheritdoc */ + def setActiveSession(session: Session): Unit = { + activeThreadSession.set(session) + } + + /** @inheritdoc */ + def clearActiveSession(): Unit = { + activeThreadSession.remove() + } + + /** @inheritdoc */ + def setDefaultSession(session: Session): Unit = { + defaultSession.set(session) + } + + /** @inheritdoc */ + def clearDefaultSession(): Unit = { + defaultSession.set(null.asInstanceOf[Session]) + } + + /** @inheritdoc */ + def getActiveSession: Option[Session] = usableSession(activeThreadSession.get()) + + /** @inheritdoc */ + def getDefaultSession: Option[Session] = usableSession(defaultSession.get()) + + private def usableSession(session: Session): Option[Session] = { + if ((session ne null) && canUseSession(session)) { + Some(session) + } else { + None + } + } + + protected def canUseSession(session: Session): Boolean = session.isUsable + + /** + * Set the (global) default [[SparkSession]], and (thread-local) active [[SparkSession]] when + * they are not set yet or they are not usable. + */ + protected def setDefaultAndActiveSession(session: Session): Unit = { + val currentDefault = defaultSession.getAcquire + if (currentDefault == null || !currentDefault.isUsable) { + // Update `defaultSession` if it is null or the contained session is not usable. There is a + // chance that the following `compareAndSet` fails if a new default session has just been set, + // but that does not matter since that event has happened after this method was invoked. + defaultSession.compareAndSet(currentDefault, session) + } + val active = getActiveSession + if (active.isEmpty || !active.get.isUsable) { + setActiveSession(session) + } + } + + /** + * When the session is closed remove it from active and default. + */ + private[sql] def onSessionClose(session: Session): Unit = { + defaultSession.compareAndSet(session, null.asInstanceOf[Session]) + if (getActiveSession.contains(session)) { + clearActiveSession() + } + } +} + /** * Builder for [[SparkSession]]. */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala index 10da24567545b..f2c7dd533af3a 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala @@ -100,7 +100,7 @@ abstract class AbstractParser extends DataTypeParserInterface with Logging { command = Option(command), start = e.origin, stop = e.origin, - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, queryContext = e.getQueryContext) } @@ -275,7 +275,7 @@ class ParseException private ( } def withCommand(cmd: String): ParseException = { - val cl = getErrorClass + val cl = getCondition val (newCl, params) = if (cl == "PARSE_SYNTAX_ERROR" && cmd.trim().isEmpty) { // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR when cmd is empty ("PARSE_EMPTY_STATEMENT", Map.empty[String, String]) @@ -287,7 +287,7 @@ class ParseException private ( override def getQueryContext: Array[QueryContext] = queryContext - override def getErrorClass: String = errorClass.getOrElse { + override def getCondition: String = errorClass.getOrElse { throw SparkException.internalError("ParseException shall have an error class.") } } diff --git a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala index b0743d6de4772..53cbf086c96e3 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala @@ -516,8 +516,11 @@ private[sql] object QueryParsingErrors extends DataTypeErrorsBase { new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0043", ctx) } - def intervalValueOutOfRangeError(ctx: IntervalContext): Throwable = { - new ParseException(errorClass = "_LEGACY_ERROR_TEMP_0044", ctx) + def intervalValueOutOfRangeError(input: String, ctx: IntervalContext): Throwable = { + new ParseException( + errorClass = "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + messageParameters = Map("input" -> input), + ctx) } def invalidTimeZoneDisplacementValueError(ctx: SetTimeZoneContext): Throwable = { diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java index ca4ea5114c26b..c0078872bd843 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/catalog/functions/ScalarFunction.java @@ -20,8 +20,11 @@ import org.apache.spark.SparkUnsupportedOperationException; import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.util.QuotingUtils; import org.apache.spark.sql.types.DataType; +import java.util.Map; + /** * Interface for a function that produces a result value for each input row. *

@@ -149,7 +152,10 @@ public interface ScalarFunction extends BoundFunction { * @return a result value */ default R produceResult(InternalRow input) { - throw new SparkUnsupportedOperationException("_LEGACY_ERROR_TEMP_3146"); + throw new SparkUnsupportedOperationException( + "SCALAR_FUNCTION_NOT_COMPATIBLE", + Map.of("scalarFunc", QuotingUtils.quoteIdentifier(name())) + ); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index a4f424ba4b421..4720b9dcdfa13 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -1621,7 +1621,7 @@ class PreemptedError() { // errors have the lowest priority. def set(error: Exception with SparkThrowable, priority: Option[Int] = None): Unit = { val calculatedPriority = priority.getOrElse { - error.getErrorClass match { + error.getCondition match { case c if c.startsWith("INTERNAL_ERROR") => 1 case _ => 2 } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala index e22a4b941b30c..8181078c519fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala @@ -24,20 +24,12 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ -/** - * A helper class used to detect duplicate relations fast in `DeduplicateRelations`. Two relations - * are duplicated if: - * 1. they are the same class. - * 2. they have the same output attribute IDs. - * - * The first condition is necessary because the CTE relation definition node and reference node have - * the same output attribute IDs but they are not duplicated. - */ -case class RelationWrapper(cls: Class[_], outputAttrIds: Seq[Long]) - object DeduplicateRelations extends Rule[LogicalPlan] { + + type ExprIdMap = mutable.HashMap[Class[_], mutable.HashSet[Long]] + override def apply(plan: LogicalPlan): LogicalPlan = { - val newPlan = renewDuplicatedRelations(mutable.HashSet.empty, plan)._1 + val newPlan = renewDuplicatedRelations(mutable.HashMap.empty, plan)._1 // Wait for `ResolveMissingReferences` to resolve missing attributes first def noMissingInput(p: LogicalPlan) = !p.exists(_.missingInput.nonEmpty) @@ -86,10 +78,10 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def existDuplicatedExprId( - existingRelations: mutable.HashSet[RelationWrapper], - plan: RelationWrapper): Boolean = { - existingRelations.filter(_.cls == plan.cls) - .exists(_.outputAttrIds.intersect(plan.outputAttrIds).nonEmpty) + existingRelations: ExprIdMap, + planClass: Class[_], exprIds: Seq[Long]): Boolean = { + val attrSet = existingRelations.getOrElse(planClass, mutable.HashSet.empty) + exprIds.exists(attrSet.contains) } /** @@ -100,20 +92,16 @@ object DeduplicateRelations extends Rule[LogicalPlan] { * whether the plan is changed or not) */ private def renewDuplicatedRelations( - existingRelations: mutable.HashSet[RelationWrapper], + existingRelations: ExprIdMap, plan: LogicalPlan): (LogicalPlan, Boolean) = plan match { case p: LogicalPlan if p.isStreaming => (plan, false) case m: MultiInstanceRelation => - val planWrapper = RelationWrapper(m.getClass, m.output.map(_.exprId.id)) - if (existingRelations.contains(planWrapper)) { - val newNode = m.newInstance() - newNode.copyTagsFrom(m) - (newNode, true) - } else { - existingRelations.add(planWrapper) - (m, false) - } + deduplicateAndRenew[LogicalPlan with MultiInstanceRelation]( + existingRelations, + m, + _.output.map(_.exprId.id), + node => node.newInstance().asInstanceOf[LogicalPlan with MultiInstanceRelation]) case p: Project => deduplicateAndRenew[Project]( @@ -207,7 +195,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def deduplicate( - existingRelations: mutable.HashSet[RelationWrapper], + existingRelations: ExprIdMap, plan: LogicalPlan): (LogicalPlan, Boolean) = { var planChanged = false val newPlan = if (plan.children.nonEmpty) { @@ -291,20 +279,21 @@ object DeduplicateRelations extends Rule[LogicalPlan] { } private def deduplicateAndRenew[T <: LogicalPlan]( - existingRelations: mutable.HashSet[RelationWrapper], plan: T, + existingRelations: ExprIdMap, plan: T, getExprIds: T => Seq[Long], copyNewPlan: T => T): (LogicalPlan, Boolean) = { var (newPlan, planChanged) = deduplicate(existingRelations, plan) if (newPlan.resolved) { val exprIds = getExprIds(newPlan.asInstanceOf[T]) if (exprIds.nonEmpty) { - val planWrapper = RelationWrapper(newPlan.getClass, exprIds) - if (existDuplicatedExprId(existingRelations, planWrapper)) { + if (existDuplicatedExprId(existingRelations, newPlan.getClass, exprIds)) { newPlan = copyNewPlan(newPlan.asInstanceOf[T]) newPlan.copyTagsFrom(plan) (newPlan, true) } else { - existingRelations.add(planWrapper) + val attrSet = existingRelations.getOrElseUpdate(newPlan.getClass, mutable.HashSet.empty) + exprIds.foreach(attrSet.add) + existingRelations.put(newPlan.getClass, attrSet) (newPlan, planChanged) } } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala index 2642b4a1c5daa..0f9b93cc2986d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala @@ -36,7 +36,7 @@ class ResolveDataFrameDropColumns(val catalogManager: CatalogManager) // df.drop(col("non-existing-column")) val dropped = d.dropList.map { case u: UnresolvedAttribute => - resolveExpressionByPlanChildren(u, d.child) + resolveExpressionByPlanChildren(u, d) case e => e } val remaining = d.child.output.filterNot(attr => dropped.exists(_.semanticEquals(attr))) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index d7d53230470d9..f2f86a90d5172 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -87,7 +87,7 @@ object ExpressionEncoder { } constructProjection(row).get(0, anyObjectType).asInstanceOf[T] } catch { - case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => throw e case e: Exception => throw QueryExecutionErrors.expressionDecodingError(e, expressions) @@ -115,7 +115,7 @@ object ExpressionEncoder { inputRow(0) = t extractProjection(inputRow) } catch { - case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => throw e case e: Exception => throw QueryExecutionErrors.expressionEncodingError(e, expressions) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index de15ec43c4f31..6a57ba2aaa569 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -383,10 +383,10 @@ abstract class Expression extends TreeNode[Expression] { trait FoldableUnevaluable extends Expression { override def foldable: Boolean = true - final override def eval(input: InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw QueryExecutionErrors.cannotEvaluateExpressionError(this) - final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = throw QueryExecutionErrors.cannotGenerateCodeForExpressionError(this) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala index 433f8500fab1f..04d31b5797819 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala @@ -17,7 +17,11 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types.{DataType, IntegerType} @@ -37,8 +41,21 @@ import org.apache.spark.sql.types.{DataType, IntegerType} abstract class PartitionTransformExpression extends Expression with Unevaluable with UnaryLike[Expression] { override def nullable: Boolean = true -} + override def eval(input: InternalRow): Any = + throw new SparkException( + errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + messageParameters = Map("expression" -> toSQLExpr(this)), + cause = null + ) + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new SparkException( + errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + messageParameters = Map("expression" -> toSQLExpr(this)), + cause = null + ) +} /** * Expression for the v2 partition transform years. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala index 220920a5a3198..d14c8cb675387 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.connector.catalog.{FunctionCatalog, Identifier} import org.apache.spark.sql.connector.catalog.functions._ import org.apache.spark.sql.connector.catalog.functions.ScalarFunction.MAGIC_METHOD_NAME import org.apache.spark.sql.connector.expressions.{BucketTransform, Expression => V2Expression, FieldReference, IdentityTransform, Literal => V2Literal, NamedReference, NamedTransform, NullOrdering => V2NullOrdering, SortDirection => V2SortDirection, SortOrder => V2SortOrder, SortValue, Transform} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.types._ import org.apache.spark.util.ArrayImplicits._ @@ -182,8 +183,8 @@ object V2ExpressionUtils extends SQLConfHelper with Logging { ApplyFunctionExpression(scalarFunc, arguments) case _ => throw new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_3055", - messageParameters = Map("scalarFunc" -> scalarFunc.name())) + errorClass = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + messageParameters = Map("scalarFunc" -> toSQLId(scalarFunc.name()))) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c9150b8a26100..3ecb680cf6427 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -3256,7 +3256,7 @@ class AstBuilder extends DataTypeAstBuilder } catch { case e: SparkArithmeticException => throw new ParseException( - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, ctx) } @@ -3552,7 +3552,7 @@ class AstBuilder extends DataTypeAstBuilder // Keep error class of SparkIllegalArgumentExceptions and enrich it with query context case se: SparkIllegalArgumentException => val pe = new ParseException( - errorClass = se.getErrorClass, + errorClass = se.getCondition, messageParameters = se.getMessageParameters.asScala.toMap, ctx) pe.setStackTrace(se.getStackTrace) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala index 46f14876be363..8d88b05546ed2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/GeneratedColumn.scala @@ -127,7 +127,7 @@ object GeneratedColumn { } catch { case ex: AnalysisException => // Improve error message if possible - if (ex.getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { + if (ex.getCondition == "UNRESOLVED_COLUMN.WITH_SUGGESTION") { ex.messageParameters.get("objectName").foreach { unresolvedCol => val resolver = SQLConf.get.resolver // Whether `col` = `unresolvedCol` taking into account case-sensitivity @@ -144,7 +144,7 @@ object GeneratedColumn { } } } - if (ex.getErrorClass == "UNRESOLVED_ROUTINE") { + if (ex.getCondition == "UNRESOLVED_ROUTINE") { // Cannot resolve function using built-in catalog ex.messageParameters.get("routineName").foreach { fnName => throw unsupportedExpressionError(s"failed to resolve $fnName to a built-in function") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala index 1f43b3dfa4a16..0e02e4249addd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala @@ -3388,8 +3388,9 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat def cannotModifyValueOfStaticConfigError(key: String): Throwable = { new AnalysisException( - errorClass = "_LEGACY_ERROR_TEMP_1325", - messageParameters = Map("key" -> key)) + errorClass = "CANNOT_MODIFY_CONFIG", + messageParameters = Map("key" -> toSQLConf(key), "docroot" -> SPARK_DOC_ROOT) + ) } def cannotModifyValueOfSparkConfigError(key: String, docroot: String): Throwable = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala index f0c28c95046eb..7602366c71a65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/exceptions/SqlScriptingException.scala @@ -33,7 +33,7 @@ class SqlScriptingException ( cause) with SparkThrowable { - override def getErrorClass: String = errorClass + override def getCondition: String = errorClass override def getMessageParameters: java.util.Map[String, String] = messageParameters.asJava } 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 969eee4d912e4..08002887135ce 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 @@ -782,7 +782,7 @@ object SQLConf { CollationFactory.fetchCollation(collationName) true } catch { - case e: SparkException if e.getErrorClass == "COLLATION_INVALID_NAME" => false + case e: SparkException if e.getCondition == "COLLATION_INVALID_NAME" => false } }, "DEFAULT_COLLATION", diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java index 0db155e88aea5..339f16407ae60 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java @@ -80,7 +80,7 @@ public void testLoadWithoutConfig() { SparkException exc = Assertions.assertThrows(CatalogNotFoundException.class, () -> Catalogs.load("missing", conf)); - Assertions.assertEquals(exc.getErrorClass(), "CATALOG_NOT_FOUND"); + Assertions.assertEquals(exc.getCondition(), "CATALOG_NOT_FOUND"); Assertions.assertEquals(exc.getMessageParameters().get("catalogName"), "`missing`"); } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index e23a753dafe8c..8409f454bfb88 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -1832,4 +1832,14 @@ class AnalysisSuite extends AnalysisTest with Matchers { preemptedError.clear() assert(preemptedError.getErrorOpt().isEmpty) } + + test("SPARK-49782: ResolveDataFrameDropColumns rule resolves complex UnresolvedAttribute") { + val function = UnresolvedFunction("trim", Seq(UnresolvedAttribute("i")), isDistinct = false) + val addColumnF = Project(Seq(UnresolvedAttribute("i"), Alias(function, "f")()), testRelation5) + // Drop column "f" via ResolveDataFrameDropColumns rule. + val inputPlan = DataFrameDropColumns(Seq(UnresolvedAttribute("f")), addColumnF) + // The expected Project (root node) should only have column "i". + val expectedPlan = Project(Seq(UnresolvedAttribute("i")), addColumnF).analyze + checkAnalysis(inputPlan, expectedPlan) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 33b9fb488c94f..71744f4d15105 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -205,7 +205,7 @@ trait AnalysisTest extends PlanTest { assert(e.message.contains(message)) } if (condition.isDefined) { - assert(e.getErrorClass == condition.get) + assert(e.getCondition == condition.get) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 3e9a93dc743df..6ee19bab5180a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -1133,7 +1133,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper { } } if (!condition.isEmpty) { - assert(e.getErrorClass == condition) + assert(e.getCondition == condition) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index e8239c7523948..f3817e4dd1a8b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -106,7 +106,7 @@ class CSVExprUtilsSuite extends SparkFunSuite { } catch { case e: SparkIllegalArgumentException => assert(separatorStr.isEmpty) - assert(e.getErrorClass === expectedErrorClass.get) + assert(e.getCondition === expectedErrorClass.get) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index 35a27f41da80a..6bd5b457ea24e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -173,7 +173,7 @@ class EncoderResolutionSuite extends PlanTest { val exception = intercept[SparkRuntimeException] { fromRow(InternalRow(new GenericArrayData(Array(1, null)))) } - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("the real number of fields doesn't match encoder schema: tuple encoder") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index f73911d344d96..79c6d07d6d218 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -279,7 +279,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { // Check the error class only since the parameters may change depending on how we are running // this test case. val exception = intercept[SparkRuntimeException](toRow(encoder, null)) - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("RowEncoder should validate external type") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index 3aeb0c882ac3c..891e2d048b7a8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -64,7 +64,7 @@ object BufferHolderSparkSubmitSuite extends Assertions { val e1 = intercept[SparkIllegalArgumentException] { holder.grow(-1) } - assert(e1.getErrorClass === "_LEGACY_ERROR_TEMP_3198") + assert(e1.getCondition === "_LEGACY_ERROR_TEMP_3198") // while to reuse a buffer may happen, this test checks whether the buffer can be grown holder.grow(ARRAY_MAX / 2) @@ -82,6 +82,6 @@ object BufferHolderSparkSubmitSuite extends Assertions { val e2 = intercept[SparkIllegalArgumentException] { holder.grow(ARRAY_MAX + 1 - holder.totalSize()) } - assert(e2.getErrorClass === "_LEGACY_ERROR_TEMP_3199") + assert(e2.getCondition === "_LEGACY_ERROR_TEMP_3199") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala index b7e2490b552cc..926beacc592a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -3065,7 +3065,7 @@ class DDLParserSuite extends AnalysisTest { s"(id BIGINT GENERATED ALWAYS AS IDENTITY $identitySpecStr, val INT) USING foo" ) } - assert(exception.getErrorClass === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION") + assert(exception.getCondition === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala index 2972ba2db21de..2e702e5642a92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala @@ -50,7 +50,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("SELECT")) } @@ -90,7 +90,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("at or near ';'")) } @@ -105,7 +105,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) assert(e.getMessage.contains("at or near end of input")) } @@ -367,7 +367,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { val e = intercept[ParseException] { parseScript(sqlScriptText) } - assert(e.getErrorClass === "PARSE_SYNTAX_ERROR") + assert(e.getCondition === "PARSE_SYNTAX_ERROR") assert(e.getMessage.contains("Syntax error")) } diff --git a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto index 71189a3c43a19..a01d4369a7aed 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -507,6 +507,9 @@ message CheckpointCommand { // (Required) Whether to checkpoint this dataframe immediately. bool eager = 3; + + // (Optional) For local checkpoint, the storage level to use. + optional StorageLevel storage_level = 4; } message MergeIntoTableCommand { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 231e54ff77d29..4e6994f9c2f8b 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -3118,7 +3118,7 @@ class SparkConnectPlanner( .newBuilder() exception_builder .setExceptionMessage(e.toString()) - .setErrorClass(e.getErrorClass) + .setErrorClass(e.getCondition) val stackTrace = Option(ExceptionUtils.getStackTrace(e)) stackTrace.foreach { s => @@ -3354,9 +3354,18 @@ class SparkConnectPlanner( responseObserver: StreamObserver[proto.ExecutePlanResponse]): Unit = { val target = Dataset .ofRows(session, transformRelation(checkpointCommand.getRelation)) - val checkpointed = target.checkpoint( - eager = checkpointCommand.getEager, - reliableCheckpoint = !checkpointCommand.getLocal) + val checkpointed = if (checkpointCommand.getLocal) { + if (checkpointCommand.hasStorageLevel) { + target.localCheckpoint( + eager = checkpointCommand.getEager, + storageLevel = + StorageLevelProtoConverter.toStorageLevel(checkpointCommand.getStorageLevel)) + } else { + target.localCheckpoint(eager = checkpointCommand.getEager) + } + } else { + target.checkpoint(eager = checkpointCommand.getEager) + } val dfId = UUID.randomUUID().toString logInfo(log"Caching DataFrame with id ${MDC(DATAFRAME_ID, dfId)}") diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala index 0468a55e23027..e62c19b66c8e5 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala @@ -345,7 +345,7 @@ object SparkConnectService extends Logging { val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore] listener = new SparkConnectServerListener(kvStore, sc.conf) sc.listenerBus.addToStatusQueue(listener) - uiTab = if (sc.getConf.get(UI_ENABLED)) { + uiTab = if (sc.conf.get(UI_ENABLED)) { Some( new SparkConnectServerTab( new SparkConnectServerAppStatusStore(kvStore), diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala index f1636ed1ef092..837d4a4d3ee78 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala @@ -114,8 +114,8 @@ private[connect] object ErrorUtils extends Logging { case sparkThrowable: SparkThrowable => val sparkThrowableBuilder = FetchErrorDetailsResponse.SparkThrowable .newBuilder() - if (sparkThrowable.getErrorClass != null) { - sparkThrowableBuilder.setErrorClass(sparkThrowable.getErrorClass) + if (sparkThrowable.getCondition != null) { + sparkThrowableBuilder.setErrorClass(sparkThrowable.getCondition) } for (queryCtx <- sparkThrowable.getQueryContext) { val builder = FetchErrorDetailsResponse.QueryContext @@ -193,7 +193,7 @@ private[connect] object ErrorUtils extends Logging { if (state != null && state.nonEmpty) { errorInfo.putMetadata("sqlState", state) } - val errorClass = e.getErrorClass + val errorClass = e.getCondition if (errorClass != null && errorClass.nonEmpty) { val messageParameters = JsonMethods.compact( JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap))) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala index 42bb93de05e26..1f522ea28b761 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala @@ -37,7 +37,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGetOrCreate = intercept[SparkSQLException] { SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None) } - assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.FORMAT") + assert(exGetOrCreate.getCondition == "INVALID_HANDLE.FORMAT") } test( @@ -72,7 +72,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA key, Some(sessionHolder.session.sessionUUID + "invalid")) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CHANGED") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CHANGED") } test( @@ -85,12 +85,12 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGetOrCreate = intercept[SparkSQLException] { SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None) } - assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED") + assert(exGetOrCreate.getCondition == "INVALID_HANDLE.SESSION_CLOSED") val exGet = intercept[SparkSQLException] { SparkConnectService.sessionManager.getIsolatedSession(key, None) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CLOSED") val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key) assert(sessionGetIfPresent.isEmpty) @@ -102,7 +102,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA val exGet = intercept[SparkSQLException] { SparkConnectService.sessionManager.getIsolatedSession(key, None) } - assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_NOT_FOUND") + assert(exGet.getCondition == "INVALID_HANDLE.SESSION_NOT_FOUND") val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key) assert(sessionGetIfPresent.isEmpty) diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml index 6bb12a927738c..d177b4a9971f5 100644 --- a/sql/connect/shims/pom.xml +++ b/sql/connect/shims/pom.xml @@ -34,6 +34,13 @@ connect-shims + + + org.scala-lang + scala-library + + + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala index 8ffdbb952b082..3b64cb97e10b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataSourceRegistration.scala @@ -68,7 +68,7 @@ class DataSourceRegistration private[sql] (dataSourceManager: DataSourceManager) DataSource.lookupDataSource(name, SQLConf.get) throw QueryCompilationErrors.dataSourceAlreadyExists(name) } catch { - case e: SparkClassNotFoundException if e.getErrorClass == "DATA_SOURCE_NOT_FOUND" => // OK + case e: SparkClassNotFoundException if e.getCondition == "DATA_SOURCE_NOT_FOUND" => // OK case _: Throwable => // If there are other errors when resolving the data source, it's unclear whether // it's safe to proceed. To prevent potential lookup errors, treat it as an existing 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 1c5df1163eb78..b7b96f0c98274 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 @@ -540,13 +540,18 @@ class Dataset[T] private[sql]( def isStreaming: Boolean = logicalPlan.isStreaming /** @inheritdoc */ - protected[sql] def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): Dataset[T] = { + protected[sql] def checkpoint( + eager: Boolean, + reliableCheckpoint: Boolean, + storageLevel: Option[StorageLevel]): Dataset[T] = { val actionName = if (reliableCheckpoint) "checkpoint" else "localCheckpoint" withAction(actionName, queryExecution) { physicalPlan => val internalRdd = physicalPlan.execute().map(_.copy()) if (reliableCheckpoint) { + assert(storageLevel.isEmpty, "StorageLevel should not be defined for reliableCheckpoint") internalRdd.checkpoint() } else { + storageLevel.foreach(storageLevel => internalRdd.persist(storageLevel)) internalRdd.localCheckpoint() } @@ -1794,6 +1799,10 @@ class Dataset[T] private[sql]( /** @inheritdoc */ override def localCheckpoint(eager: Boolean): Dataset[T] = super.localCheckpoint(eager) + /** @inheritdoc */ + override def localCheckpoint(eager: Boolean, storageLevel: StorageLevel): Dataset[T] = + super.localCheckpoint(eager, storageLevel) + /** @inheritdoc */ override def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = super.joinWith(other, condition) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 2d485c4ef321d..99ab3ca69fb20 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -21,7 +21,7 @@ import java.net.URI import java.nio.file.Paths import java.util.{ServiceLoader, UUID} import java.util.concurrent.ConcurrentHashMap -import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} +import java.util.concurrent.atomic.AtomicBoolean import scala.concurrent.duration.DurationInt import scala.jdk.CollectionConverters._ @@ -743,7 +743,7 @@ class SparkSession private( // Use the active session thread local directly to make sure we get the session that is actually // set and not the default session. This to prevent that we promote the default session to the // active session once we are done. - val old = SparkSession.activeThreadSession.get() + val old = SparkSession.getActiveSession.orNull SparkSession.setActiveSession(this) try block finally { SparkSession.setActiveSession(old) @@ -774,11 +774,14 @@ class SparkSession private( } private[sql] lazy val observationManager = new ObservationManager(this) + + override private[sql] def isUsable: Boolean = !sparkContext.isStopped } @Stable -object SparkSession extends api.SparkSessionCompanion with Logging { +object SparkSession extends api.BaseSparkSessionCompanion with Logging { + override private[sql] type Session = SparkSession /** * Builder for [[SparkSession]]. @@ -862,28 +865,22 @@ object SparkSession extends api.SparkSessionCompanion with Logging { assertOnDriver() } - def clearSessionIfDead(session: SparkSession): SparkSession = { - if ((session ne null) && !session.sparkContext.isStopped) { - session - } else { - null - } - } - // Get the session from current thread's active session. - val active = clearSessionIfDead(activeThreadSession.get()) - if (!forceCreate && (active ne null)) { - applyModifiableSettings(active, new java.util.HashMap[String, String](options.asJava)) - return active + val active = getActiveSession + if (!forceCreate && active.isDefined) { + val session = active.get + applyModifiableSettings(session, new java.util.HashMap[String, String](options.asJava)) + return session } // Global synchronization so we will only set the default session once. SparkSession.synchronized { // If the current thread does not have an active session, get it from the global session. - val default = clearSessionIfDead(defaultSession.get()) - if (!forceCreate && (default ne null)) { - applyModifiableSettings(default, new java.util.HashMap[String, String](options.asJava)) - return default + val default = getDefaultSession + if (!forceCreate && default.isDefined) { + val session = default.get + applyModifiableSettings(session, new java.util.HashMap[String, String](options.asJava)) + return session } // No active nor global default session. Create a new one. @@ -906,12 +903,7 @@ object SparkSession extends api.SparkSessionCompanion with Logging { extensions, initialSessionOptions = options.toMap, parentManagedJobTags = Map.empty) - if (default eq null) { - setDefaultSession(session) - } - if (active eq null) { - setActiveSession(session) - } + setDefaultAndActiveSession(session) registerContextListener(sparkContext) session } @@ -931,87 +923,17 @@ object SparkSession extends api.SparkSessionCompanion with Logging { */ def builder(): Builder = new Builder - /** - * Changes the SparkSession that will be returned in this thread and its children when - * SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives - * a SparkSession with an isolated session, instead of the global (first created) context. - * - * @since 2.0.0 - */ - def setActiveSession(session: SparkSession): Unit = { - activeThreadSession.set(session) - } - - /** - * Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will - * return the first created context instead of a thread-local override. - * - * @since 2.0.0 - */ - def clearActiveSession(): Unit = { - activeThreadSession.remove() - } - - /** - * Sets the default SparkSession that is returned by the builder. - * - * @since 2.0.0 - */ - def setDefaultSession(session: SparkSession): Unit = { - defaultSession.set(session) - } - - /** - * Clears the default SparkSession that is returned by the builder. - * - * @since 2.0.0 - */ - def clearDefaultSession(): Unit = { - defaultSession.set(null) - } + /** @inheritdoc */ + override def getActiveSession: Option[SparkSession] = super.getActiveSession - /** - * Returns the active SparkSession for the current thread, returned by the builder. - * - * @note Return None, when calling this function on executors - * - * @since 2.2.0 - */ - def getActiveSession: Option[SparkSession] = { - if (Utils.isInRunningSparkTask) { - // Return None when running on executors. - None - } else { - Option(activeThreadSession.get) - } - } + /** @inheritdoc */ + override def getDefaultSession: Option[SparkSession] = super.getDefaultSession - /** - * Returns the default SparkSession that is returned by the builder. - * - * @note Return None, when calling this function on executors - * - * @since 2.2.0 - */ - def getDefaultSession: Option[SparkSession] = { - if (Utils.isInRunningSparkTask) { - // Return None when running on executors. - None - } else { - Option(defaultSession.get) - } - } + /** @inheritdoc */ + override def active: SparkSession = super.active - /** - * Returns the currently active SparkSession, otherwise the default one. If there is no default - * SparkSession, throws an exception. - * - * @since 2.4.0 - */ - def active: SparkSession = { - getActiveSession.getOrElse(getDefaultSession.getOrElse( - throw SparkException.internalError("No active or default Spark session found"))) - } + override protected def canUseSession(session: SparkSession): Boolean = + session.isUsable && !Utils.isInRunningSparkTask /** * Apply modifiable settings to an existing [[SparkSession]]. This method are used @@ -1082,7 +1004,8 @@ object SparkSession extends api.SparkSessionCompanion with Logging { if (!listenerRegistered.get()) { sparkContext.addSparkListener(new SparkListener { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = { - defaultSession.set(null) + clearDefaultSession() + clearActiveSession() listenerRegistered.set(false) } }) @@ -1090,12 +1013,6 @@ object SparkSession extends api.SparkSessionCompanion with Logging { } } - /** The active SparkSession for the current thread. */ - private val activeThreadSession = new InheritableThreadLocal[SparkSession] - - /** Reference to the root SparkSession. */ - private val defaultSession = new AtomicReference[SparkSession] - private val HIVE_SESSION_STATE_BUILDER_CLASS_NAME = "org.apache.spark.sql.hive.HiveSessionStateBuilder" @@ -1183,7 +1100,7 @@ object SparkSession extends api.SparkSessionCompanion with Logging { private def applyExtensions( sparkContext: SparkContext, extensions: SparkSessionExtensions): SparkSessionExtensions = { - val extensionConfClassNames = sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) + val extensionConfClassNames = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) .getOrElse(Seq.empty) extensionConfClassNames.foreach { extensionConfClassName => try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8fc860c503c96..9fbe400a555fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -189,10 +189,29 @@ class SparkSqlAstBuilder extends AstBuilder { val key = SQLConf.SESSION_LOCAL_TIMEZONE.key if (ctx.interval != null) { val interval = parseIntervalLiteral(ctx.interval) - if (interval.months != 0 || interval.days != 0 || - math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR || - interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { - throw QueryParsingErrors.intervalValueOutOfRangeError(ctx.interval()) + if (interval.months != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue(interval.months), + ctx.interval() + ) + } + else if (interval.days != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue(interval.days), + ctx.interval() + ) + } + else if (math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue((math.abs(interval.microseconds) / DateTimeConstants.MICROS_PER_HOUR).toInt), + ctx.interval() + ) + } + else if (interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) { + throw QueryParsingErrors.intervalValueOutOfRangeError( + toSQLValue((interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt), + ctx.interval() + ) } else { val seconds = (interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt SetCommand(Some(key -> Some(ZoneOffset.ofTotalSeconds(seconds).toString))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 29385904a7525..cbbf9f88f89d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -89,9 +89,9 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { LogicalRelation(ds.resolveRelation()) } catch { case _: ClassNotFoundException => u - case e: SparkIllegalArgumentException if e.getErrorClass != null => + case e: SparkIllegalArgumentException if e.getCondition != null => u.failAnalysis( - errorClass = e.getErrorClass, + errorClass = e.getCondition, messageParameters = e.getMessageParameters.asScala.toMap, cause = e) case e: Exception if !e.isInstanceOf[AnalysisException] => @@ -469,8 +469,8 @@ object PreprocessTableInsertion extends ResolveInsertionBase { supportColDefaultValue = true) } catch { case e: AnalysisException if staticPartCols.nonEmpty && - (e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" || - e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") => + (e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" || + e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") => val newException = e.copy( errorClass = Some("INSERT_PARTITION_COLUMN_ARITY_MISMATCH"), messageParameters = e.messageParameters ++ Map( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala index 168aea5b041f8..4242fc5d8510a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala @@ -131,7 +131,7 @@ object FileDataSourceV2 { // The error is already FAILED_READ_FILE, throw it directly. To be consistent, schema // inference code path throws `FAILED_READ_FILE`, but the file reading code path can reach // that code path as well and we should not double-wrap the error. - case e: SparkException if e.getErrorClass == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" => + case e: SparkException if e.getCondition == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" => throw e case e: SchemaColumnConvertNotSupportedException => throw QueryExecutionErrors.parquetColumnDataTypeMismatchError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala index d890107277d6c..5c0f8c0a4afd9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -164,7 +164,7 @@ trait FileScan extends Scan if (splitFiles.length == 1) { val path = splitFiles(0).toPath if (!isSplitable(path) && splitFiles(0).length > - sparkSession.sparkContext.getConf.get(IO_WARNING_LARGEFILETHRESHOLD)) { + sparkSession.sparkContext.conf.get(IO_WARNING_LARGEFILETHRESHOLD)) { logWarning(log"Loading one large unsplittable file ${MDC(PATH, path.toString)} with only " + log"one partition, the reason is: ${MDC(REASON, getFileUnSplittableReason(path))}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index e669165f4f2f8..8ec903f8e61da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -70,7 +70,16 @@ case class EnsureRequirements( case (child, distribution) => val numPartitions = distribution.requiredNumPartitions .getOrElse(conf.numShufflePartitions) - ShuffleExchangeExec(distribution.createPartitioning(numPartitions), child, shuffleOrigin) + distribution match { + case _: StatefulOpClusteredDistribution => + ShuffleExchangeExec( + distribution.createPartitioning(numPartitions), child, + REQUIRED_BY_STATEFUL_OPERATOR) + + case _ => + ShuffleExchangeExec( + distribution.createPartitioning(numPartitions), child, shuffleOrigin) + } } // Get the indexes of children which have specified distribution requirements and need to be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index ae11229cd516e..31a3f53eb7191 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -177,6 +177,11 @@ case object REBALANCE_PARTITIONS_BY_NONE extends ShuffleOrigin // the output needs to be partitioned by the given columns. case object REBALANCE_PARTITIONS_BY_COL extends ShuffleOrigin +// Indicates that the shuffle operator was added by the internal `EnsureRequirements` rule, but +// was required by a stateful operator. The physical partitioning is static and Spark shouldn't +// change it. +case object REQUIRED_BY_STATEFUL_OPERATOR extends ShuffleOrigin + /** * Performs a shuffle that will result in the desired partitioning. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index 8f030884ad33b..14adf951f07e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -374,7 +374,7 @@ abstract class StreamExecution( "message" -> message)) errorClassOpt = e match { - case t: SparkThrowable => Option(t.getErrorClass) + case t: SparkThrowable => Option(t.getCondition) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 884b8aa3853cb..3df63c41dbf97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -282,7 +282,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with newMap } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 6ab634668bc2a..870ed79ec1747 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -389,7 +389,7 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( @@ -409,7 +409,7 @@ private[sql] class RocksDBStateStoreProvider new RocksDBStateStore(version) } catch { - case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") => + case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") => throw e case e: OutOfMemoryError => throw QueryExecutionErrors.notEnoughMemoryToLoadStore( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 52b8d35e2fbf8..64689e75e2e5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -177,7 +177,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { try { Some(makeTable(catalogName +: ns :+ tableName)) } catch { - case e: AnalysisException if e.getErrorClass == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" => + case e: AnalysisException if e.getCondition == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" => Some(new Table( name = tableName, catalog = catalogName, @@ -189,7 +189,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { } } } catch { - case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => None + case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => None } } @@ -203,7 +203,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { case _ => false } } catch { - case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => false + case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => false } } @@ -323,7 +323,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { case _ => false } } catch { - case e: AnalysisException if e.getErrorClass == "UNRESOLVED_ROUTINE" => false + case e: AnalysisException if e.getCondition == "UNRESOLVED_ROUTINE" => false } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index 60258ecbb0d61..8341063e09890 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -23,6 +23,7 @@ import java.util import java.util.Locale import scala.util.Using +import scala.util.control.NonFatal import org.apache.spark.SparkThrowable import org.apache.spark.internal.LogKeys.COLUMN_NAME @@ -30,7 +31,7 @@ import org.apache.spark.internal.MDC import org.apache.spark.sql.catalyst.SQLConfHelper import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NonEmptyNamespaceException, NoSuchIndexException} import org.apache.spark.sql.connector.catalog.Identifier -import org.apache.spark.sql.connector.expressions.NamedReference +import org.apache.spark.sql.connector.expressions.{Expression, NamedReference} import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils} import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo @@ -300,6 +301,28 @@ private case class PostgresDialect() } } + class PostgresSQLBuilder extends JDBCSQLBuilder { + override def visitExtract(field: String, source: String): String = { + field match { + case "DAY_OF_YEAR" => s"EXTRACT(DOY FROM $source)" + case "YEAR_OF_WEEK" => s"EXTRACT(YEAR FROM $source)" + case "DAY_OF_WEEK" => s"EXTRACT(DOW FROM $source)" + case _ => super.visitExtract(field, source) + } + } + } + + override def compileExpression(expr: Expression): Option[String] = { + val postgresSQLBuilder = new PostgresSQLBuilder() + try { + Some(postgresSQLBuilder.build(expr)) + } catch { + case NonFatal(e) => + logWarning("Error occurs while compiling V2 expression", e) + None + } + } + override def supportsLimit: Boolean = true override def supportsOffset: Boolean = true diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java index 9fbd1919a2668..9988d04220f0f 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java @@ -85,7 +85,7 @@ public void isInCollectionCheckExceptionMessage() { Dataset df = spark.createDataFrame(rows, schema); AnalysisException e = Assertions.assertThrows(AnalysisException.class, () -> df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b"))))); - Assertions.assertTrue(e.getErrorClass().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES")); + Assertions.assertTrue(e.getCondition().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES")); Map messageParameters = new HashMap<>(); messageParameters.put("functionName", "`in`"); messageParameters.put("dataType", "[\"INT\", \"ARRAY\"]"); diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out index 9059f37f3607b..5b55a0c218934 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/timezone.sql.out @@ -64,7 +64,11 @@ SET TIME ZONE INTERVAL 3 DAYS -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "3" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -80,7 +84,11 @@ SET TIME ZONE INTERVAL 24 HOURS -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "24" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -96,7 +104,11 @@ SET TIME ZONE INTERVAL '19:40:32' HOUR TO SECOND -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "19" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -128,7 +140,11 @@ SET TIME ZONE INTERVAL 10 HOURS 1 MILLISECOND -- !query analysis org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "36000" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out index d34599a49c5ff..5f0fdef50e3db 100644 --- a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out @@ -80,7 +80,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "3" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -98,7 +102,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "24" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -116,7 +124,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "19" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", @@ -152,7 +164,11 @@ struct<> -- !query output org.apache.spark.sql.catalyst.parser.ParseException { - "errorClass" : "_LEGACY_ERROR_TEMP_0044", + "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + "sqlState" : "22006", + "messageParameters" : { + "input" : "36000" + }, "queryContext" : [ { "objectType" : "", "objectName" : "", diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt index 96bed479d2e06..4bf7de791b279 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt @@ -175,125 +175,125 @@ Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#21, cou Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#12 AS i_product_name#24, i_brand#9 AS i_brand#25, i_class#10 AS i_class#26, i_category#11 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (27) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (28) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (29) HashAggregate [codegen id : 16] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (30) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (32) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (33) HashAggregate [codegen id : 25] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (34) HashAggregate [codegen id : 25] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (35) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 26] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (37) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (38) HashAggregate [codegen id : 34] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (39) HashAggregate [codegen id : 34] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (40) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 35] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (42) ReusedExchange [Reuses operator id: 23] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (43) HashAggregate [codegen id : 43] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (44) HashAggregate [codegen id : 43] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (45) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 44] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (47) Union (48) TakeOrderedAndProject -Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -306,22 +306,22 @@ BroadcastExchange (53) (49) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#7, d_month_seq#91] +Output [2]: [d_date_sk#7, d_month_seq#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (50) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (51) Filter [codegen id : 1] -Input [2]: [d_date_sk#7, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [2]: [d_date_sk#7, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#7)) (52) Project [codegen id : 1] Output [1]: [d_date_sk#7] -Input [2]: [d_date_sk#7, d_month_seq#91] +Input [2]: [d_date_sk#7, d_month_seq#95] (53) BroadcastExchange Input [1]: [d_date_sk#7] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt index 0c4267b3ca513..042f946b8fca4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (8) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt index 4b8993f370f4d..8aab8e91acfc8 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt @@ -160,125 +160,125 @@ Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, coun Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10] Functions [1]: [avg(qoh#18)] Aggregate Attributes [1]: [avg(qoh#18)#23] -Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24] +Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28] (24) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] +Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] (25) HashAggregate [codegen id : 10] -Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30] -Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28] -Functions [1]: [avg(inv_quantity_on_hand#31)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17] -Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32] +Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34] +Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32] +Functions [1]: [avg(inv_quantity_on_hand#35)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17] +Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36] (26) HashAggregate [codegen id : 10] -Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [partial_avg(qoh#32)] -Aggregate Attributes [2]: [sum#33, count#34] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] +Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [partial_avg(qoh#36)] +Aggregate Attributes [2]: [sum#37, count#38] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] (27) Exchange -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4] (28) HashAggregate [codegen id : 11] -Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36] -Keys [3]: [i_product_name#25, i_brand#26, i_class#27] -Functions [1]: [avg(qoh#32)] -Aggregate Attributes [1]: [avg(qoh#32)#37] -Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39] +Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40] +Keys [3]: [i_product_name#29, i_brand#30, i_class#31] +Functions [1]: [avg(qoh#36)] +Aggregate Attributes [1]: [avg(qoh#36)#41] +Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43] (29) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] +Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] (30) HashAggregate [codegen id : 16] -Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45] -Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43] -Functions [1]: [avg(inv_quantity_on_hand#46)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17] -Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47] +Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49] +Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47] +Functions [1]: [avg(inv_quantity_on_hand#50)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17] +Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51] (31) HashAggregate [codegen id : 16] -Input [3]: [i_product_name#40, i_brand#41, qoh#47] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [partial_avg(qoh#47)] -Aggregate Attributes [2]: [sum#48, count#49] -Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51] +Input [3]: [i_product_name#44, i_brand#45, qoh#51] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [partial_avg(qoh#51)] +Aggregate Attributes [2]: [sum#52, count#53] +Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55] (32) Exchange -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5] (33) HashAggregate [codegen id : 17] -Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51] -Keys [2]: [i_product_name#40, i_brand#41] -Functions [1]: [avg(qoh#47)] -Aggregate Attributes [1]: [avg(qoh#47)#52] -Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55] +Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55] +Keys [2]: [i_product_name#44, i_brand#45] +Functions [1]: [avg(qoh#51)] +Aggregate Attributes [1]: [avg(qoh#51)#56] +Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59] (34) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] +Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] (35) HashAggregate [codegen id : 22] -Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61] -Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59] -Functions [1]: [avg(inv_quantity_on_hand#62)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17] -Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63] +Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65] +Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63] +Functions [1]: [avg(inv_quantity_on_hand#66)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17] +Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67] (36) HashAggregate [codegen id : 22] -Input [2]: [i_product_name#56, qoh#63] -Keys [1]: [i_product_name#56] -Functions [1]: [partial_avg(qoh#63)] -Aggregate Attributes [2]: [sum#64, count#65] -Results [3]: [i_product_name#56, sum#66, count#67] +Input [2]: [i_product_name#60, qoh#67] +Keys [1]: [i_product_name#60] +Functions [1]: [partial_avg(qoh#67)] +Aggregate Attributes [2]: [sum#68, count#69] +Results [3]: [i_product_name#60, sum#70, count#71] (37) Exchange -Input [3]: [i_product_name#56, sum#66, count#67] -Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [3]: [i_product_name#60, sum#70, count#71] +Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6] (38) HashAggregate [codegen id : 23] -Input [3]: [i_product_name#56, sum#66, count#67] -Keys [1]: [i_product_name#56] -Functions [1]: [avg(qoh#63)] -Aggregate Attributes [1]: [avg(qoh#63)#68] -Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72] +Input [3]: [i_product_name#60, sum#70, count#71] +Keys [1]: [i_product_name#60] +Functions [1]: [avg(qoh#67)] +Aggregate Attributes [1]: [avg(qoh#67)#72] +Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76] (39) ReusedExchange [Reuses operator id: 20] -Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] +Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] (40) HashAggregate [codegen id : 28] -Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78] -Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76] -Functions [1]: [avg(inv_quantity_on_hand#79)] -Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17] -Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80] +Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82] +Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80] +Functions [1]: [avg(inv_quantity_on_hand#83)] +Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17] +Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84] (41) HashAggregate [codegen id : 28] -Input [1]: [qoh#80] +Input [1]: [qoh#84] Keys: [] -Functions [1]: [partial_avg(qoh#80)] -Aggregate Attributes [2]: [sum#81, count#82] -Results [2]: [sum#83, count#84] +Functions [1]: [partial_avg(qoh#84)] +Aggregate Attributes [2]: [sum#85, count#86] +Results [2]: [sum#87, count#88] (42) Exchange -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7] (43) HashAggregate [codegen id : 29] -Input [2]: [sum#83, count#84] +Input [2]: [sum#87, count#88] Keys: [] -Functions [1]: [avg(qoh#80)] -Aggregate Attributes [1]: [avg(qoh#80)#85] -Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90] +Functions [1]: [avg(qoh#84)] +Aggregate Attributes [1]: [avg(qoh#84)#89] +Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94] (44) Union (45) TakeOrderedAndProject -Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] -Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24] +Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] +Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28] ===== Subqueries ===== @@ -291,22 +291,22 @@ BroadcastExchange (50) (46) Scan parquet spark_catalog.default.date_dim -Output [2]: [d_date_sk#6, d_month_seq#91] +Output [2]: [d_date_sk#6, d_month_seq#95] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (47) ColumnarToRow [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (48) Filter [codegen id : 1] -Input [2]: [d_date_sk#6, d_month_seq#91] -Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#6)) +Input [2]: [d_date_sk#6, d_month_seq#95] +Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6)) (49) Project [codegen id : 1] Output [1]: [d_date_sk#6] -Input [2]: [d_date_sk#6, d_month_seq#91] +Input [2]: [d_date_sk#6, d_month_seq#95] (50) BroadcastExchange Input [1]: [d_date_sk#6] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt index 22f73cc9b9db5..d747066f5945b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt @@ -1,7 +1,7 @@ TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category] Union WholeStageCodegen (5) - HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count] + HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count] HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count] InputAdapter diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt index 9c28ff9f351d8..a4c009f8219b4 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt @@ -186,265 +186,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (25) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (26) HashAggregate [codegen id : 16] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (27) HashAggregate [codegen id : 16] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (28) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=5] (29) HashAggregate [codegen id : 17] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (30) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (31) HashAggregate [codegen id : 25] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (32) HashAggregate [codegen id : 25] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (33) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=6] (34) HashAggregate [codegen id : 26] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (35) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (36) HashAggregate [codegen id : 34] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (37) HashAggregate [codegen id : 34] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (38) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=7] (39) HashAggregate [codegen id : 35] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (40) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (41) HashAggregate [codegen id : 43] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (42) HashAggregate [codegen id : 43] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (43) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=8] (44) HashAggregate [codegen id : 44] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (45) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (46) HashAggregate [codegen id : 52] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (47) HashAggregate [codegen id : 52] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (48) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=9] (49) HashAggregate [codegen id : 53] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (50) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (51) HashAggregate [codegen id : 61] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (52) HashAggregate [codegen id : 61] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (53) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=10] (54) HashAggregate [codegen id : 62] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (55) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (56) HashAggregate [codegen id : 70] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (57) HashAggregate [codegen id : 70] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (58) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=11] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=11] (59) HashAggregate [codegen id : 71] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (60) ReusedExchange [Reuses operator id: 23] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (61) HashAggregate [codegen id : 79] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (62) HashAggregate [codegen id : 79] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (63) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12] (64) HashAggregate [codegen id : 80] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (65) Union (66) Sort [codegen id : 81] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (68) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=13] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=13] (69) Sort [codegen id : 82] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (70) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (71) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (72) Filter [codegen id : 83] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (73) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -457,22 +457,22 @@ BroadcastExchange (78) (74) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (75) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (76) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (77) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (78) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt index 795fa297b9bad..b6a4358c4d43b 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (8) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (7) diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt index 75d526da4ba71..417af4fe924ee 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt @@ -171,265 +171,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12] Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))] Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22] -Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23] +Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31] (22) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] +Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] (23) HashAggregate [codegen id : 10] -Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33] -Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31] -Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22] -Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36] +Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41] +Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39] +Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22] +Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44] (24) HashAggregate [codegen id : 10] -Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [partial_sum(sumsales#36)] -Aggregate Attributes [2]: [sum#37, isEmpty#38] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] +Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [partial_sum(sumsales#44)] +Aggregate Attributes [2]: [sum#45, isEmpty#46] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] (25) Exchange -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=4] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4] (26) HashAggregate [codegen id : 11] -Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40] -Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30] -Functions [1]: [sum(sumsales#36)] -Aggregate Attributes [1]: [sum(sumsales#36)#41] -Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43] +Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48] +Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38] +Functions [1]: [sum(sumsales#44)] +Aggregate Attributes [1]: [sum(sumsales#44)#49] +Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51] (27) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] +Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] (28) HashAggregate [codegen id : 16] -Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53] -Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51] -Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22] -Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56] +Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61] +Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59] +Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22] +Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64] (29) HashAggregate [codegen id : 16] -Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [partial_sum(sumsales#56)] -Aggregate Attributes [2]: [sum#57, isEmpty#58] -Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] +Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [partial_sum(sumsales#64)] +Aggregate Attributes [2]: [sum#65, isEmpty#66] +Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] (30) Exchange -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=5] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5] (31) HashAggregate [codegen id : 17] -Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60] -Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49] -Functions [1]: [sum(sumsales#56)] -Aggregate Attributes [1]: [sum(sumsales#56)#61] -Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64] +Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68] +Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57] +Functions [1]: [sum(sumsales#64)] +Aggregate Attributes [1]: [sum(sumsales#64)#69] +Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72] (32) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] +Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] (33) HashAggregate [codegen id : 22] -Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74] -Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72] -Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22] -Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77] +Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82] +Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80] +Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22] +Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85] (34) HashAggregate [codegen id : 22] -Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [partial_sum(sumsales#77)] -Aggregate Attributes [2]: [sum#78, isEmpty#79] -Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] +Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [partial_sum(sumsales#85)] +Aggregate Attributes [2]: [sum#86, isEmpty#87] +Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] (35) Exchange -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=6] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6] (36) HashAggregate [codegen id : 23] -Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81] -Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69] -Functions [1]: [sum(sumsales#77)] -Aggregate Attributes [1]: [sum(sumsales#77)#82] -Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86] +Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89] +Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77] +Functions [1]: [sum(sumsales#85)] +Aggregate Attributes [1]: [sum(sumsales#85)#90] +Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94] (37) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] +Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] (38) HashAggregate [codegen id : 28] -Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96] -Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94] -Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22] -Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99] +Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104] +Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102] +Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22] +Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107] (39) HashAggregate [codegen id : 28] -Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [partial_sum(sumsales#99)] -Aggregate Attributes [2]: [sum#100, isEmpty#101] -Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] +Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [partial_sum(sumsales#107)] +Aggregate Attributes [2]: [sum#108, isEmpty#109] +Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] (40) Exchange -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=7] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7] (41) HashAggregate [codegen id : 29] -Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103] -Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90] -Functions [1]: [sum(sumsales#99)] -Aggregate Attributes [1]: [sum(sumsales#99)#104] -Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109] +Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111] +Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98] +Functions [1]: [sum(sumsales#107)] +Aggregate Attributes [1]: [sum(sumsales#107)#112] +Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117] (42) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] +Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] (43) HashAggregate [codegen id : 34] -Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119] -Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117] -Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22] -Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122] +Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127] +Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125] +Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22] +Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130] (44) HashAggregate [codegen id : 34] -Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [partial_sum(sumsales#122)] -Aggregate Attributes [2]: [sum#123, isEmpty#124] -Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] +Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [partial_sum(sumsales#130)] +Aggregate Attributes [2]: [sum#131, isEmpty#132] +Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] (45) Exchange -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=8] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8] (46) HashAggregate [codegen id : 35] -Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126] -Keys [3]: [i_category#110, i_class#111, i_brand#112] -Functions [1]: [sum(sumsales#122)] -Aggregate Attributes [1]: [sum(sumsales#122)#127] -Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133] +Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134] +Keys [3]: [i_category#118, i_class#119, i_brand#120] +Functions [1]: [sum(sumsales#130)] +Aggregate Attributes [1]: [sum(sumsales#130)#135] +Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141] (47) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] +Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] (48) HashAggregate [codegen id : 40] -Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143] -Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141] -Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22] -Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146] +Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151] +Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149] +Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22] +Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154] (49) HashAggregate [codegen id : 40] -Input [3]: [i_category#134, i_class#135, sumsales#146] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [partial_sum(sumsales#146)] -Aggregate Attributes [2]: [sum#147, isEmpty#148] -Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] +Input [3]: [i_category#142, i_class#143, sumsales#154] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [partial_sum(sumsales#154)] +Aggregate Attributes [2]: [sum#155, isEmpty#156] +Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] (50) Exchange -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=9] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9] (51) HashAggregate [codegen id : 41] -Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150] -Keys [2]: [i_category#134, i_class#135] -Functions [1]: [sum(sumsales#146)] -Aggregate Attributes [1]: [sum(sumsales#146)#151] -Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158] +Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158] +Keys [2]: [i_category#142, i_class#143] +Functions [1]: [sum(sumsales#154)] +Aggregate Attributes [1]: [sum(sumsales#154)#159] +Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166] (52) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] +Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] (53) HashAggregate [codegen id : 46] -Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168] -Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166] -Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22] -Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171] +Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176] +Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174] +Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22] +Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179] (54) HashAggregate [codegen id : 46] -Input [2]: [i_category#159, sumsales#171] -Keys [1]: [i_category#159] -Functions [1]: [partial_sum(sumsales#171)] -Aggregate Attributes [2]: [sum#172, isEmpty#173] -Results [3]: [i_category#159, sum#174, isEmpty#175] +Input [2]: [i_category#167, sumsales#179] +Keys [1]: [i_category#167] +Functions [1]: [partial_sum(sumsales#179)] +Aggregate Attributes [2]: [sum#180, isEmpty#181] +Results [3]: [i_category#167, sum#182, isEmpty#183] (55) Exchange -Input [3]: [i_category#159, sum#174, isEmpty#175] -Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=10] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10] (56) HashAggregate [codegen id : 47] -Input [3]: [i_category#159, sum#174, isEmpty#175] -Keys [1]: [i_category#159] -Functions [1]: [sum(sumsales#171)] -Aggregate Attributes [1]: [sum(sumsales#171)#176] -Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184] +Input [3]: [i_category#167, sum#182, isEmpty#183] +Keys [1]: [i_category#167] +Functions [1]: [sum(sumsales#179)] +Aggregate Attributes [1]: [sum(sumsales#179)#184] +Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192] (57) ReusedExchange [Reuses operator id: 20] -Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] +Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] (58) HashAggregate [codegen id : 52] -Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194] -Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192] -Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))] -Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22] -Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197] +Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202] +Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200] +Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))] +Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22] +Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205] (59) HashAggregate [codegen id : 52] -Input [1]: [sumsales#197] +Input [1]: [sumsales#205] Keys: [] -Functions [1]: [partial_sum(sumsales#197)] -Aggregate Attributes [2]: [sum#198, isEmpty#199] -Results [2]: [sum#200, isEmpty#201] +Functions [1]: [partial_sum(sumsales#205)] +Aggregate Attributes [2]: [sum#206, isEmpty#207] +Results [2]: [sum#208, isEmpty#209] (60) Exchange -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11] (61) HashAggregate [codegen id : 53] -Input [2]: [sum#200, isEmpty#201] +Input [2]: [sum#208, isEmpty#209] Keys: [] -Functions [1]: [sum(sumsales#197)] -Aggregate Attributes [1]: [sum(sumsales#197)#202] -Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211] +Functions [1]: [sum(sumsales#205)] +Aggregate Attributes [1]: [sum(sumsales#205)#210] +Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219] (62) Union (63) Sort [codegen id : 54] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (64) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial (65) Exchange -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12] (66) Sort [codegen id : 55] -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0 +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0 (67) WindowGroupLimit -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final (68) Window -Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23] -Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST] +Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31] +Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST] (69) Filter [codegen id : 56] -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Condition : (rk#212 <= 100) +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Condition : (rk#220 <= 100) (70) TakeOrderedAndProject -Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] -Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212] +Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] +Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220] ===== Subqueries ===== @@ -442,22 +442,22 @@ BroadcastExchange (75) (71) Scan parquet spark_catalog.default.date_dim -Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] Batched: true Location [not included in comparison]/{warehouse_dir}/date_dim] PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)] ReadSchema: struct (72) ColumnarToRow [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (73) Filter [codegen id : 1] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] -Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7)) +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] +Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7)) (74) Project [codegen id : 1] Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] -Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10] +Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10] (75) BroadcastExchange Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10] diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt index 89393f265a49f..5a43dced056bd 100644 --- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt +++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt @@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_ InputAdapter Union WholeStageCodegen (5) - HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty] + HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty] InputAdapter Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2 WholeStageCodegen (4) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala index 879c0c480943d..8600ec4f8787f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala @@ -741,7 +741,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi assert(resultUTF8.collect() === resultUTF8Lcase.collect()) } } catch { - case e: SparkRuntimeException => assert(e.getErrorClass == "USER_RAISED_EXCEPTION") + case e: SparkRuntimeException => assert(e.getCondition == "USER_RAISED_EXCEPTION") case other: Throwable => throw other } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 089ce79201dd8..45c34d9c73367 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1273,7 +1273,7 @@ class DatasetSuite extends QueryTest // Just check the error class here to avoid flakiness due to different parameters. assert(intercept[SparkRuntimeException] { buildDataset(Row(Row("hello", null))).collect() - }.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + }.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("SPARK-12478: top level null field") { @@ -1416,7 +1416,7 @@ class DatasetSuite extends QueryTest val ex = intercept[SparkRuntimeException] { spark.createDataFrame(rdd, schema).collect() } - assert(ex.getErrorClass == "EXPRESSION_ENCODING_FAILED") + assert(ex.getCondition == "EXPRESSION_ENCODING_FAILED") assert(ex.getCause.getMessage.contains("The 1th field 'b' of input row cannot be null")) } @@ -1612,7 +1612,7 @@ class DatasetSuite extends QueryTest test("Dataset should throw RuntimeException if top-level product input object is null") { val e = intercept[SparkRuntimeException](Seq(ClassData("a", 1), null).toDS()) - assert(e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("dropDuplicates") { @@ -1849,6 +1849,26 @@ class DatasetSuite extends QueryTest } } + test("Dataset().localCheckpoint() lazy with StorageLevel") { + val df = spark.range(10).repartition($"id" % 2) + val checkpointedDf = df.localCheckpoint(eager = false, StorageLevel.DISK_ONLY) + val checkpointedPlan = checkpointedDf.queryExecution.analyzed + val rdd = checkpointedPlan.asInstanceOf[LogicalRDD].rdd + assert(rdd.getStorageLevel == StorageLevel.DISK_ONLY) + assert(!rdd.isCheckpointed) + checkpointedDf.collect() + assert(rdd.isCheckpointed) + } + + test("Dataset().localCheckpoint() eager with StorageLevel") { + val df = spark.range(10).repartition($"id" % 2) + val checkpointedDf = df.localCheckpoint(eager = true, StorageLevel.DISK_ONLY) + val checkpointedPlan = checkpointedDf.queryExecution.analyzed + val rdd = checkpointedPlan.asInstanceOf[LogicalRDD].rdd + assert(rdd.isCheckpointed) + assert(rdd.getStorageLevel == StorageLevel.DISK_ONLY) + } + test("identity map for primitive arrays") { val arrayByte = Array(1.toByte, 2.toByte, 3.toByte) val arrayInt = Array(1, 2, 3) @@ -2101,7 +2121,7 @@ class DatasetSuite extends QueryTest test("SPARK-23835: null primitive data type should throw NullPointerException") { val ds = Seq[(Option[Int], Option[Int])]((Some(1), None)).toDS() val exception = intercept[SparkRuntimeException](ds.as[(Int, Int)].collect()) - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } test("SPARK-24569: Option of primitive types are mistakenly mapped to struct type") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala index a892cd4db02b0..3f921618297d0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/LateralColumnAliasSuite.scala @@ -205,7 +205,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { withLCAOn { checkAnswer(sql(query), expectedAnswerLCAOn) } withLCAOff { assert(intercept[AnalysisException]{ sql(query) } - .getErrorClass == "UNRESOLVED_COLUMN.WITH_SUGGESTION") + .getCondition == "UNRESOLVED_COLUMN.WITH_SUGGESTION") } } @@ -216,8 +216,8 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { errorParams: Map[String, String]): Unit = { val e1 = intercept[AnalysisException] { sql(q1) } val e2 = intercept[AnalysisException] { sql(q2) } - assert(e1.getErrorClass == condition) - assert(e2.getErrorClass == condition) + assert(e1.getCondition == condition) + assert(e2.getCondition == condition) errorParams.foreach { case (k, v) => assert(e1.messageParameters.get(k).exists(_ == v)) assert(e2.messageParameters.get(k).exists(_ == v)) @@ -1187,7 +1187,7 @@ class LateralColumnAliasSuite extends LateralColumnAliasSuiteBase { "sum_avg * 1.0 as sum_avg1, sum_avg1 + dept " + s"from $testTable group by dept, properties.joinYear $havingSuffix" ).foreach { query => - assert(intercept[AnalysisException](sql(query)).getErrorClass == + assert(intercept[AnalysisException](sql(query)).getCondition == "UNSUPPORTED_FEATURE.LATERAL_COLUMN_ALIAS_IN_AGGREGATE_WITH_WINDOW_AND_HAVING") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala index 754c46cc5cd3e..b48ff7121c767 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeNullChecksV2Writes.scala @@ -64,7 +64,7 @@ class RuntimeNullChecksV2Writes extends QueryTest with SQLTestUtils with SharedS sql("INSERT INTO t VALUES ('txt', null)") } } - assert(e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } @@ -404,7 +404,7 @@ class RuntimeNullChecksV2Writes extends QueryTest with SQLTestUtils with SharedS private def assertNotNullException(e: SparkRuntimeException, colPath: Seq[String]): Unit = { e.getCause match { - case _ if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" => + case _ if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" => case other => fail(s"Unexpected exception cause: $other") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala index 38e004e0b7209..4bd20bc245613 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestHelper.scala @@ -148,7 +148,7 @@ trait SQLQueryTestHelper extends Logging { try { result } catch { - case e: SparkThrowable with Throwable if e.getErrorClass != null => + case e: SparkThrowable with Throwable if e.getCondition != null => (emptySchema, Seq(e.getClass.getName, getMessage(e, format))) case a: AnalysisException => // Do not output the logical plan tree which contains expression IDs. @@ -160,7 +160,7 @@ trait SQLQueryTestHelper extends Logging { // information of stage, task ID, etc. // To make result matching simpler, here we match the cause of the exception if it exists. s.getCause match { - case e: SparkThrowable with Throwable if e.getErrorClass != null => + case e: SparkThrowable with Throwable if e.getCondition != null => (emptySchema, Seq(e.getClass.getName, getMessage(e, format))) case cause => (emptySchema, Seq(cause.getClass.getName, cause.getMessage)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index f17cf25565145..f8f7fd246832f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -925,12 +925,12 @@ class SubquerySuite extends QueryTest withSQLConf(SQLConf.DECORRELATE_INNER_QUERY_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } withSQLConf(SQLConf.DECORRELATE_SET_OPS_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } @@ -1004,12 +1004,12 @@ class SubquerySuite extends QueryTest withSQLConf(SQLConf.DECORRELATE_INNER_QUERY_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } withSQLConf(SQLConf.DECORRELATE_SET_OPS_ENABLED.key -> "false") { val error = intercept[AnalysisException] { sql(query) } - assert(error.getErrorClass == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + + assert(error.getCondition == "UNSUPPORTED_SUBQUERY_EXPRESSION_CATEGORY." + "ACCESSING_OUTER_QUERY_COLUMN_IS_NOT_ALLOWED") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 2e072e5afc926..d550d0f94f236 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -821,14 +821,14 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e1 = intercept[SparkException] { Seq("20").toDF("col").select(udf(f1).apply(Column("col"))).collect() } - assert(e1.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e1.getCondition == "FAILED_EXECUTE_UDF") assert(e1.getCause.getStackTrace.head.toString.contains( "UDFSuite$MalformedClassObject$MalformedNonPrimitiveFunction")) val e2 = intercept[SparkException] { Seq(20).toDF("col").select(udf(f2).apply(Column("col"))).collect() } - assert(e2.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e2.getCondition == "FAILED_EXECUTE_UDF") assert(e2.getCause.getStackTrace.head.toString.contains( "UDFSuite$MalformedClassObject$MalformedPrimitiveFunction")) } @@ -938,7 +938,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"dateTime")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } @@ -1053,7 +1053,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"d")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } @@ -1101,7 +1101,7 @@ class UDFSuite extends QueryTest with SharedSparkSession { val e = intercept[SparkException] { input.select(overflowFunc($"p")).collect() } - assert(e.getErrorClass == "FAILED_EXECUTE_UDF") + assert(e.getCondition == "FAILED_EXECUTE_UDF") assert(e.getCause.isInstanceOf[java.lang.ArithmeticException]) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala index d6599debd3b11..6b0fd6084099c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala @@ -414,8 +414,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { new JavaStrLen(new JavaStrLenNoImpl)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('abc')").collect()), - condition = "_LEGACY_ERROR_TEMP_3055", - parameters = Map("scalarFunc" -> "strlen"), + condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + parameters = Map("scalarFunc" -> "`strlen`"), context = ExpectedContext( fragment = "testcat.ns.strlen('abc')", start = 7, @@ -448,8 +448,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddMismatchMagic)) checkError( exception = intercept[AnalysisException](sql("SELECT testcat.ns.add(1L, 2L)").collect()), - condition = "_LEGACY_ERROR_TEMP_3055", - parameters = Map("scalarFunc" -> "long_add_mismatch_magic"), + condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED", + parameters = Map("scalarFunc" -> "`long_add_mismatch_magic`"), context = ExpectedContext( fragment = "testcat.ns.add(1L, 2L)", start = 7, @@ -458,6 +458,23 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase { ) } + test("SPARK-49549: scalar function w/ mismatch a compatible ScalarFunction#produceResult") { + case object CharLength extends ScalarFunction[Int] { + override def inputTypes(): Array[DataType] = Array(StringType) + override def resultType(): DataType = IntegerType + override def name(): String = "CHAR_LENGTH" + } + + catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps) + addFunction(Identifier.of(Array("ns"), "my_strlen"), StrLen(CharLength)) + checkError( + exception = intercept[SparkUnsupportedOperationException] + (sql("SELECT testcat.ns.my_strlen('abc')").collect()), + condition = "SCALAR_FUNCTION_NOT_COMPATIBLE", + parameters = Map("scalarFunc" -> "`CHAR_LENGTH`") + ) + } + test("SPARK-35390: scalar function w/ type coercion") { catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps) addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddDefault(false))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala index 6b58d23e92603..52ae1bf5d9d3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala @@ -840,7 +840,7 @@ class DataSourceV2SQLSuiteV1Filter val exception = intercept[SparkRuntimeException] { insertNullValueAndCheck() } - assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala index 9d4e4fc016722..053616c88d638 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala @@ -1326,7 +1326,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | UPDATE SET s = named_struct('n_i', null, 'n_l', -1L) |""".stripMargin) } - assert(e1.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e1.getCondition == "NOT_NULL_ASSERT_VIOLATION") val e2 = intercept[SparkRuntimeException] { sql( @@ -1337,7 +1337,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | UPDATE SET s = named_struct('n_i', null, 'n_l', -1L) |""".stripMargin) } - assert(e2.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e2.getCondition == "NOT_NULL_ASSERT_VIOLATION") val e3 = intercept[SparkRuntimeException] { sql( @@ -1348,7 +1348,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase { | INSERT (pk, s, dep) VALUES (s.pk, named_struct('n_i', null, 'n_l', -1L), 'invalid') |""".stripMargin) } - assert(e3.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(e3.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala index 9d1448d0ac09d..1adb1fdf05032 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala @@ -35,11 +35,12 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, Encoder, Kry import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator} import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression, Years} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean import org.apache.spark.sql.catalyst.rules.RuleIdCollection +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions} import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider import org.apache.spark.sql.execution.datasources.orc.OrcTest @@ -292,7 +293,7 @@ class QueryExecutionErrorsSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val format = "Parquet" val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key + "\"" @@ -311,7 +312,7 @@ class QueryExecutionErrorsSuite val ex = intercept[SparkException] { spark.read.schema("time timestamp_ntz").orc(file.getCanonicalPath).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", @@ -333,7 +334,7 @@ class QueryExecutionErrorsSuite val ex = intercept[SparkException] { spark.read.schema("time timestamp_ltz").orc(file.getCanonicalPath).collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) checkError( exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException], condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST", @@ -381,7 +382,7 @@ class QueryExecutionErrorsSuite } val e2 = e1.getCause.asInstanceOf[SparkException] - assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") + assert(e2.getCondition == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], @@ -920,7 +921,7 @@ class QueryExecutionErrorsSuite val e = intercept[StreamingQueryException] { query.awaitTermination() } - assert(e.getErrorClass === "STREAM_FAILED") + assert(e.getCondition === "STREAM_FAILED") assert(e.getCause.isInstanceOf[NullPointerException]) } @@ -1006,6 +1007,17 @@ class QueryExecutionErrorsSuite sqlState = "XX000") } + test("PartitionTransformExpression error on eval") { + val expr = Years(Literal("foo")) + val e = intercept[SparkException] { + expr.eval() + } + checkError( + exception = e, + condition = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY", + parameters = Map("expression" -> toSQLExpr(expr))) + } + test("INTERNAL_ERROR: Calling doGenCode on unresolved") { val e = intercept[SparkException] { val ctx = new CodegenContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index 75f016d050de9..c5e64c96b2c8a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -904,7 +904,7 @@ class AdaptiveQueryExecSuite val error = intercept[SparkException] { aggregated.count() } - assert(error.getErrorClass === "INVALID_BUCKET_FILE") + assert(error.getCondition === "INVALID_BUCKET_FILE") assert(error.getMessage contains "Invalid bucket file") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala index deb62eb3ac234..387a2baa256bf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -368,7 +368,7 @@ class BinaryFileFormatSuite extends QueryTest with SharedSparkSession { checkAnswer(readContent(), expected) } } - assert(caught.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(caught.getCondition.startsWith("FAILED_READ_FILE")) assert(caught.getCause.getMessage.contains("exceeds the max length allowed")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 023f401516dc3..422ae02a18322 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -392,7 +392,7 @@ abstract class CSVSuite condition = "FAILED_READ_FILE.NO_HINT", parameters = Map("path" -> s".*$carsFile.*")) val e2 = e1.getCause.asInstanceOf[SparkException] - assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") + assert(e2.getCondition == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION") checkError( exception = e2.getCause.asInstanceOf[SparkRuntimeException], condition = "MALFORMED_CSV_RECORD", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index f13d66b76838f..500c0647bcb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -708,7 +708,7 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { val ex = intercept[SparkException] { sql(s"select A from $tableName where A < 0").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) assert(ex.getCause.isInstanceOf[SparkRuntimeException]) assert(ex.getCause.getMessage.contains( """Found duplicate field(s) "A": [A, a] in case-insensitive mode""")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index 2e6413d998d12..ab0d4d9bc53b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -604,14 +604,14 @@ abstract class OrcQueryTest extends OrcTest { val e1 = intercept[SparkException] { testIgnoreCorruptFiles() } - assert(e1.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e1.getCondition.startsWith("FAILED_READ_FILE")) assert(e1.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e1.getCause.getCause.getMessage.contains("Malformed ORC file")) val e2 = intercept[SparkException] { testIgnoreCorruptFilesWithoutSchemaInfer() } - assert(e2.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e2.getCondition.startsWith("FAILED_READ_FILE")) assert(e2.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e2.getCause.getCause.getMessage.contains("Malformed ORC file")) @@ -625,7 +625,7 @@ abstract class OrcQueryTest extends OrcTest { val e4 = intercept[SparkException] { testAllCorruptFilesWithoutSchemaInfer() } - assert(e4.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e4.getCondition.startsWith("FAILED_READ_FILE")) assert(e4.getCause.getMessage.contains("Malformed ORC file") || // Hive ORC table scan uses a different code path and has one more error stack e4.getCause.getCause.getMessage.contains("Malformed ORC file")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 9348d10711b35..040999476ece1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -450,8 +450,8 @@ abstract class OrcSuite val ex = intercept[SparkException] { spark.read.orc(basePath).columns.length } - assert(ex.getErrorClass == "CANNOT_MERGE_SCHEMAS") - assert(ex.getCause.asInstanceOf[SparkException].getErrorClass === + assert(ex.getCondition == "CANNOT_MERGE_SCHEMAS") + assert(ex.getCause.asInstanceOf[SparkException].getCondition === "CANNOT_MERGE_INCOMPATIBLE_DATA_TYPE") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index 5c382b1858716..903dda7f41c0d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -1958,7 +1958,7 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared val ex = intercept[SparkException] { sql(s"select a from $tableName where b > 0").collect() } - assert(ex.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(ex.getCondition.startsWith("FAILED_READ_FILE")) assert(ex.getCause.isInstanceOf[SparkRuntimeException]) assert(ex.getCause.getMessage.contains( """Found duplicate field(s) "B": [B, b] in case-insensitive mode""")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala index 0afa545595c77..95fb178154929 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala @@ -1223,7 +1223,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession val m1 = intercept[SparkException] { spark.range(1).coalesce(1).write.options(extraOptions).parquet(dir.getCanonicalPath) } - assert(m1.getErrorClass == "TASK_WRITE_FAILED") + assert(m1.getCondition == "TASK_WRITE_FAILED") assert(m1.getCause.getMessage.contains("Intentional exception for testing purposes")) } @@ -1233,8 +1233,8 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSparkSession .coalesce(1) df.write.partitionBy("a").options(extraOptions).parquet(dir.getCanonicalPath) } - if (m2.getErrorClass != null) { - assert(m2.getErrorClass == "TASK_WRITE_FAILED") + if (m2.getCondition != null) { + assert(m2.getCondition == "TASK_WRITE_FAILED") assert(m2.getCause.getMessage.contains("Intentional exception for testing purposes")) } else { assert(m2.getMessage.contains("TASK_WRITE_FAILED")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index eb4618834504c..87a2843f34de1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -1181,7 +1181,7 @@ abstract class ParquetPartitionDiscoverySuite spark.read.parquet(dir.toString) } val msg = exception.getMessage - assert(exception.getErrorClass === "CONFLICTING_PARTITION_COLUMN_NAMES") + assert(exception.getCondition === "CONFLICTING_PARTITION_COLUMN_NAMES") // Partitions inside the error message can be presented in any order assert("Partition column name list #[0-1]: col1".r.findFirstIn(msg).isDefined) assert("Partition column name list #[0-1]: col1, col2".r.findFirstIn(msg).isDefined) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 4d413efe50430..22a02447e720f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -1075,7 +1075,7 @@ abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedS val e = intercept[SparkException] { readParquet("d DECIMAL(3, 2)", path).collect() } - assert(e.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(e.getCondition.startsWith("FAILED_READ_FILE")) assert(e.getCause.getMessage.contains("Please read this column/field as Spark BINARY type")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala index 6d9092391a98e..30503af0fab6f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRebaseDatetimeSuite.scala @@ -414,7 +414,7 @@ abstract class ParquetRebaseDatetimeSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } @@ -431,7 +431,7 @@ abstract class ParquetRebaseDatetimeSuite val e = intercept[SparkException] { df.write.parquet(dir.getCanonicalPath) } - assert(e.getErrorClass == "TASK_WRITE_FAILED") + assert(e.getCondition == "TASK_WRITE_FAILED") val errMsg = e.getCause.asInstanceOf[SparkUpgradeException].getMessage assert(errMsg.contains("You may get a different result due to the upgrading")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala index 95378d9467478..08fd8a9ecb53e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowIndexSuite.scala @@ -319,7 +319,7 @@ class ParquetRowIndexSuite extends QueryTest with SharedSparkSession { .load(path.getAbsolutePath) val exception = intercept[SparkException](dfRead.collect()) - assert(exception.getErrorClass.startsWith("FAILED_READ_FILE")) + assert(exception.getCondition.startsWith("FAILED_READ_FILE")) assert(exception.getCause.getMessage.contains( ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala index 4833b8630134c..59c0af8afd198 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala @@ -90,7 +90,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") } } @@ -103,7 +103,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE") } } @@ -116,7 +116,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + assert(exc.getCondition === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") } } @@ -130,7 +130,7 @@ abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestB .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) .load(tempDir.getAbsolutePath) } - assert(exc.getErrorClass === "STDS_CONFLICT_OPTIONS") + assert(exc.getCondition === "STDS_CONFLICT_OPTIONS") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala index 5f55848d540df..300da03f73e1f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceReadSuite.scala @@ -1137,7 +1137,7 @@ abstract class StateDataSourceReadSuite extends StateDataSourceTestBase with Ass val exc = intercept[StateStoreSnapshotPartitionNotFound] { stateDfError.show() } - assert(exc.getErrorClass === "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND") + assert(exc.getCondition === "CANNOT_LOAD_STATE_STORE.SNAPSHOT_PARTITION_ID_NOT_FOUND") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala index dcebece29037f..1f2be12058eb7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonDataSourceSuite.scala @@ -330,7 +330,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("PySparkNotImplementedError")) } @@ -350,7 +350,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("error creating reader")) } @@ -369,7 +369,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { val err = intercept[AnalysisException] { spark.read.format(dataSourceName).schema(schema).load().collect() } - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("DATA_SOURCE_TYPE_MISMATCH")) assert(err.getMessage.contains("PySparkAssertionError")) } @@ -480,7 +480,7 @@ class PythonDataSourceSuite extends PythonDataSourceSuiteBase { spark.dataSource.registerPython(dataSourceName, dataSource) val err = intercept[AnalysisException]( spark.read.format(dataSourceName).load().collect()) - assert(err.getErrorClass == "PYTHON_DATA_SOURCE_ERROR") + assert(err.getCondition == "PYTHON_DATA_SOURCE_ERROR") assert(err.getMessage.contains("partitions")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala index 8d0e1c5f578fa..3d91a045907fc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonStreamingDataSourceSuite.scala @@ -574,7 +574,7 @@ class PythonStreamingDataSourceSuite extends PythonDataSourceSuiteBase { val q = spark.readStream.format(dataSourceName).load().writeStream.format("console").start() q.awaitTermination() } - assert(err.getErrorClass == "STREAM_FAILED") + assert(err.getCondition == "STREAM_FAILED") assert(err.getMessage.contains("error creating stream reader")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala index 38533825ece90..99483bc0ee8dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityCheckerSuite.scala @@ -423,14 +423,14 @@ class StateSchemaCompatibilityCheckerSuite extends SharedSparkSession { // collation checks are also performed in this path. so we need to check for them explicitly. if (keyCollationChecks) { assert(ex.getMessage.contains("Binary inequality column is not supported")) - assert(ex.getErrorClass === "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY") + assert(ex.getCondition === "STATE_STORE_UNSUPPORTED_OPERATION_BINARY_INEQUALITY") } else { if (ignoreValueSchema) { // if value schema is ignored, the mismatch has to be on the key schema - assert(ex.getErrorClass === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE") + assert(ex.getCondition === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE") } else { - assert(ex.getErrorClass === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE" || - ex.getErrorClass === "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE") + assert(ex.getCondition === "STATE_STORE_KEY_SCHEMA_NOT_COMPATIBLE" || + ex.getCondition === "STATE_STORE_VALUE_SCHEMA_NOT_COMPATIBLE") } assert(ex.getMessage.contains("does not match existing")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 8bbc7a31760d9..2a9944a81cb2a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -1373,7 +1373,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] put(store, "a", 0, 0) val e = intercept[SparkException](quietly { store.commit() } ) - assert(e.getErrorClass == "CANNOT_WRITE_STATE_STORE.CANNOT_COMMIT") + assert(e.getCondition == "CANNOT_WRITE_STATE_STORE.CANNOT_COMMIT") if (store.getClass.getName contains ROCKSDB_STATE_STORE) { assert(e.getMessage contains "RocksDBStateStore[id=(op=0,part=0)") } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 6611ecce0ad8e..2b58440baf852 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala @@ -233,8 +233,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { // static sql configs checkError( exception = intercept[AnalysisException](sql(s"RESET ${StaticSQLConf.WAREHOUSE_PATH.key}")), - condition = "_LEGACY_ERROR_TEMP_1325", - parameters = Map("key" -> "spark.sql.warehouse.dir")) + condition = "CANNOT_MODIFY_CONFIG", + parameters = Map("key" -> "\"spark.sql.warehouse.dir\"", "docroot" -> SPARK_DOC_ROOT)) } @@ -315,10 +315,16 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { } test("cannot set/unset static SQL conf") { - val e1 = intercept[AnalysisException](sql(s"SET ${GLOBAL_TEMP_DATABASE.key}=10")) - assert(e1.message.contains("Cannot modify the value of a static config")) - val e2 = intercept[AnalysisException](spark.conf.unset(GLOBAL_TEMP_DATABASE.key)) - assert(e2.message.contains("Cannot modify the value of a static config")) + checkError( + exception = intercept[AnalysisException](sql(s"SET ${GLOBAL_TEMP_DATABASE.key}=10")), + condition = "CANNOT_MODIFY_CONFIG", + parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"", "docroot" -> SPARK_DOC_ROOT) + ) + checkError( + exception = intercept[AnalysisException](spark.conf.unset(GLOBAL_TEMP_DATABASE.key)), + condition = "CANNOT_MODIFY_CONFIG", + parameters = Map("key" -> "\"spark.sql.globalTempDatabase\"", "docroot" -> SPARK_DOC_ROOT) + ) } test("SPARK-36643: Show migration guide when attempting SparkConf") { @@ -486,8 +492,8 @@ class SQLConfSuite extends QueryTest with SharedSparkSession { val sqlText = "set time zone interval 19 hours" checkError( exception = intercept[ParseException](sql(sqlText)), - condition = "_LEGACY_ERROR_TEMP_0044", - parameters = Map.empty, + condition = "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE", + parameters = Map("input" -> "19"), context = ExpectedContext(sqlText, 0, 30)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala index 41447d8af5740..baf99798965da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/InsertSuite.scala @@ -956,7 +956,7 @@ class InsertSuite extends DataSourceTest with SharedSparkSession { val msg = intercept[SparkRuntimeException] { sql("INSERT INTO TABLE test_table SELECT 2, null") } - assert(msg.getErrorClass == "NOT_NULL_ASSERT_VIOLATION") + assert(msg.getCondition == "NOT_NULL_ASSERT_VIOLATION") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index d9ce8002d285b..a0eea14e54eed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -296,7 +296,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val exception = SparkException.internalError("testpurpose") testSerialization( new QueryTerminatedEvent(UUID.randomUUID, UUID.randomUUID, - Some(exception.getMessage), Some(exception.getErrorClass))) + Some(exception.getMessage), Some(exception.getCondition))) } test("only one progress event per interval when no data") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 8471995cb1e50..c12846d7512d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit} -import org.apache.spark.sql.execution.exchange.ReusedExchangeExec +import org.apache.spark.sql.execution.exchange.{REQUIRED_BY_STATEFUL_OPERATOR, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ @@ -1448,6 +1448,28 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } } + test("SPARK-49905 shuffle added by stateful operator should use the shuffle origin " + + "`REQUIRED_BY_STATEFUL_OPERATOR`") { + val inputData = MemoryStream[Int] + + // Use the streaming aggregation as an example - all stateful operators are using the same + // distribution, named `StatefulOpClusteredDistribution`. + val df = inputData.toDF().groupBy("value").count() + + testStream(df, OutputMode.Update())( + AddData(inputData, 1, 2, 3, 1, 2, 3), + CheckAnswer((1, 2), (2, 2), (3, 2)), + Execute { qe => + val shuffleOpt = qe.lastExecution.executedPlan.collect { + case s: ShuffleExchangeExec => s + } + + assert(shuffleOpt.nonEmpty, "No shuffle exchange found in the query plan") + assert(shuffleOpt.head.shuffleOrigin === REQUIRED_BY_STATEFUL_OPERATOR) + } + ) + } + private def checkAppendOutputModeException(df: DataFrame): Unit = { withTempDir { outputDir => withTempDir { checkpointDir => diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala index 8a8bdd4d38ee3..59d1b61f2f8e7 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala @@ -38,7 +38,7 @@ object HiveThriftServerErrors { def runningQueryError(e: Throwable, format: ErrorMessageFormat.Value): Throwable = e match { case st: SparkThrowable if format == ErrorMessageFormat.PRETTY => - val errorClassPrefix = Option(st.getErrorClass).map(e => s"[$e] ").getOrElse("") + val errorClassPrefix = Option(st.getCondition).map(e => s"[$e] ").getOrElse("") new HiveSQLException( s"Error running query: $errorClassPrefix${st.toString}", st.getSqlState, st) case st: SparkThrowable with Throwable => 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 f1f0befcb0d30..43030f68e5dac 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 @@ -1062,7 +1062,7 @@ class SingleSessionSuite extends HiveThriftServer2TestBase { statement.executeQuery("SET spark.sql.hive.thriftServer.singleSession=false") }.getMessage assert(e.contains( - "Cannot modify the value of a static config: spark.sql.hive.thriftServer.singleSession")) + "CANNOT_MODIFY_CONFIG")) } } 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 1c45b02375b30..83d70b2e19109 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 @@ -26,7 +26,7 @@ import java.util.{Locale, Set} import com.google.common.io.{Files, FileWriteMode} import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.{SparkException, TestUtils} +import org.apache.spark.{SPARK_DOC_ROOT, SparkException, TestUtils} import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLConf import org.apache.spark.sql.connector.catalog.CatalogManager.SESSION_CATALOG_NAME import org.apache.spark.sql.execution.{SparkPlanInfo, TestUncaughtExceptionHandler} import org.apache.spark.sql.execution.adaptive.{DisableAdaptiveExecutionSuite, EnableAdaptiveExecutionSuite} @@ -2461,8 +2462,12 @@ abstract class SQLQuerySuiteBase extends QueryTest with SQLTestUtils with TestHi "spark.sql.hive.metastore.jars", "spark.sql.hive.metastore.sharedPrefixes", "spark.sql.hive.metastore.barrierPrefixes").foreach { key => - val e = intercept[AnalysisException](sql(s"set $key=abc")) - assert(e.getMessage.contains("Cannot modify the value of a static config")) + checkError( + exception = intercept[AnalysisException](sql(s"set $key=abc")), + condition = "CANNOT_MODIFY_CONFIG", + parameters = Map( + "key" -> toSQLConf(key), "docroot" -> SPARK_DOC_ROOT) + ) } }