From b565a8df9fef4344f18ad103df69178586ea099d Mon Sep 17 00:00:00 2001 From: Herman van Hovell Date: Wed, 9 Oct 2024 08:58:06 -0400 Subject: [PATCH 01/16] [SPARK-49418][CONNECT][SQL] Shared Session Thread Locals ### What changes were proposed in this pull request? This PR adds interfaces for SparkSession Thread Locals. ### Why are the changes needed? We are creating a unified Spark SQL Scala interface. This is part of that effort. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48374 from hvanhovell/SPARK-49418. Authored-by: Herman van Hovell Signed-off-by: Herman van Hovell --- .../org/apache/spark/sql/SparkSession.scala | 113 ++--------- .../apache/spark/sql/SparkSessionSuite.scala | 3 +- .../CheckConnectJvmClientCompatibility.scala | 2 + project/MimaExcludes.scala | 6 + .../apache/spark/sql/api/SparkSession.scala | 177 +++++++++++++++++- .../org/apache/spark/sql/SparkSession.scala | 137 +++----------- 6 files changed, 224 insertions(+), 214 deletions(-) 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/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/project/MimaExcludes.scala b/project/MimaExcludes.scala index 2b3d76eb0c2c3..3ccb0bddfb0eb 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -189,6 +189,12 @@ 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"), ) ++ loggingExcludes("org.apache.spark.sql.DataFrameReader") ++ loggingExcludes("org.apache.spark.sql.streaming.DataStreamReader") ++ loggingExcludes("org.apache.spark.sql.SparkSession#Builder") 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/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 2d485c4ef321d..55525380aee55 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" From b1ff7672cba12750d41d803f0faeb3487d934601 Mon Sep 17 00:00:00 2001 From: Julek Sompolski Date: Wed, 9 Oct 2024 21:34:16 +0800 Subject: [PATCH 02/16] [SPARK-49857][SQL] Add storageLevel to Dataset localCheckpoint API ### What changes were proposed in this pull request? Currently, when running `Dataset.localCheckpoint(eager = true)`, it is impossible to specify a non-default StorageLevel for the checkpoint. On the other hand it is possible with Dataset cache by using `Dataset.persist(newLevel: StorageLevel)`. If one wants to specify a non-default StorageLevel for localCheckpoint, it currently needs accessing the plan, changing the level, and then triggering an action to materialize checkpoint: ``` // start lazy val checkpointDf = df.localCheckpoint(eager = false) // fish out the RDD val checkpointPlan = checkpointedSourcePlanDF.queryExecution.analyzed val rdd = checkpointedPlan.asInstanceOf[LogicalRDD].rdd // change the StorageLevel rdd.persist(StorageLevel.DISK_ONLY) // force materialization checkpointDf .mapPartitions(_ => Iterator.empty.asInstanceOf[Iterator[Row]]) .foreach((_: Row) => ()) ``` There are several issues with this: 1. Won't work with Connect as we don't have access to RDD internals 2. Lazy checkpoint is not in fact lazy when AQE is involved. In order to get the RDD of a lazy checkpoint, AQE will actually trigger execution of all the query stages except the result stage in order to get the final plan. So the `start lazy` phase will already execute everything except the final stage, and then `force materialization` will only execute result stage. This is "unexpected" and makes it more difficult to debug, first showing a query with missing metrics for the final stage, and then another query that skipped everything and only ran final stage. Having an API to specify storageLevel for localCheckpoint will help avoid such hacks. As a precedent, it is already possible to specify StorageLevel for Dataset cache by using `Dataset.persist(newLevel: StorageLevel)`. In this PR, I implement this API for scala and python, and classic and connect. ### Why are the changes needed? https://github.com/delta-io/delta/blob/master/spark/src/main/scala/org/apache/spark/sql/delta/commands/merge/MergeIntoMaterializeSource.scala in `prepareMergeSource` has to do hacks as described above to use localCheckpoint with non-default StorageLevel. It is hacky, and confusing that it then records two separate executions as described above. ### Does this PR introduce _any_ user-facing change? Yes. Adds API to pass `storageLevel` to Dataset `localCheckpoint`. ### How was this patch tested? Tests added. ### Was this patch authored or co-authored using generative AI tooling? Generated-by: Github Copilot (trivial code completions) Closes #48324 from juliuszsompolski/SPARK-49857. Authored-by: Julek Sompolski Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/sql/Dataset.scala | 15 ++++++- .../apache/spark/sql/CheckpointSuite.scala | 15 +++++-- python/pyspark/sql/classic/dataframe.py | 9 +++- python/pyspark/sql/connect/dataframe.py | 6 ++- python/pyspark/sql/connect/plan.py | 22 ++++++---- .../pyspark/sql/connect/proto/commands_pb2.py | 14 +++---- .../sql/connect/proto/commands_pb2.pyi | 29 ++++++++++++- python/pyspark/sql/dataframe.py | 9 +++- python/pyspark/sql/tests/test_dataframe.py | 8 +++- .../org/apache/spark/sql/api/Dataset.scala | 42 ++++++++++++++++--- .../protobuf/spark/connect/commands.proto | 3 ++ .../connect/planner/SparkConnectPlanner.scala | 15 +++++-- .../scala/org/apache/spark/sql/Dataset.scala | 11 ++++- .../org/apache/spark/sql/DatasetSuite.scala | 20 +++++++++ 14 files changed, 181 insertions(+), 37 deletions(-) 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/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/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/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index b5af00a4e7b78..cc43804949e84 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -950,11 +950,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/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/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..25fd7d13b7d48 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 @@ -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/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/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 089ce79201dd8..85f296665b6e0 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 @@ -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) From 97a5aa6ef7cab46564325dafed9f8266ac7f8999 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Wed, 9 Oct 2024 15:48:38 +0200 Subject: [PATCH 03/16] [SPARK-49873][SQL] Assign proper error class for _LEGACY_ERROR_TEMP_1325 ### What changes were proposed in this pull request? This PR proposes to assign proper error class for _LEGACY_ERROR_TEMP_1325 ### Why are the changes needed? To improve user facing error message by providing proper error condition and sql state ### Does this PR introduce _any_ user-facing change? Improve user-facing error message ### How was this patch tested? Updated the existing UT ### Was this patch authored or co-authored using generative AI tooling? No Closes #48346 from itholic/legacy_1325. Authored-by: Haejoon Lee Signed-off-by: Max Gekk --- .../main/resources/error/error-conditions.json | 5 ----- .../sql/errors/QueryCompilationErrors.scala | 5 +++-- .../spark/sql/internal/SQLConfSuite.scala | 18 ++++++++++++------ .../thriftserver/HiveThriftServer2Suites.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 11 ++++++++--- 5 files changed, 24 insertions(+), 17 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 1b7f42e105077..f6317d731c77b 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -6696,11 +6696,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 ." 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/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..1a6cdd1258cc3 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") { 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) + ) } } From fed9a8da3d4187794161e0be325aa96be8487783 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Thu, 10 Oct 2024 01:49:24 +0800 Subject: [PATCH 04/16] [SPARK-49569][BUILD][FOLLOWUP] Adds `scala-library` maven dependency to the `spark-connect-shims` module to fix Maven build errors ### What changes were proposed in this pull request? This PR adds `scala-library` maven dependency to the `spark-connect-shims` module to fix Maven build errors. ### Why are the changes needed? Maven daily test pipeline build failed: - https://github.com/apache/spark/actions/runs/11255598249 - https://github.com/apache/spark/actions/runs/11256610976 ``` scaladoc error: fatal error: object scala in compiler mirror not found. Error: Failed to execute goal net.alchim31.maven:scala-maven-plugin:4.9.1:doc-jar (attach-scaladocs) on project spark-connect-shims_2.13: MavenReportException: Error while creating archive: wrap: Process exited with an error: 1 (Exit value: 1) -> [Help 1] Error: Error: To see the full stack trace of the errors, re-run Maven with the -e switch. Error: Re-run Maven using the -X switch to enable full debug logging. Error: Error: For more information about the errors and possible solutions, please read the following articles: Error: [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException Error: Error: After correcting the problems, you can resume the build with the command Error: mvn -rf :spark-connect-shims_2.13 Error: Process completed with exit code 1. ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - locally test: ``` build/mvn clean install -DskipTests -Phive ``` **Before** ``` [INFO] --- scala:4.9.1:doc-jar (attach-scaladocs) spark-connect-shims_2.13 --- scaladoc error: fatal error: object scala in compiler mirror not found. [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 4.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 2.833 s] [INFO] Spark Project Tags ................................. SUCCESS [ 5.292 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 5.675 s] [INFO] Spark Project Common Utils ......................... SUCCESS [ 16.762 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 7.735 s] [INFO] Spark Project Networking ........................... SUCCESS [ 11.389 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 9.159 s] [INFO] Spark Project Variant .............................. SUCCESS [ 3.618 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 9.692 s] [INFO] Spark Project Connect Shims ........................ FAILURE [ 2.478 s] [INFO] Spark Project Launcher ............................. SKIPPED [INFO] Spark Project Core ................................. SKIPPED [INFO] Spark Project ML Local Library ..................... SKIPPED [INFO] Spark Project GraphX ............................... SKIPPED [INFO] Spark Project Streaming ............................ SKIPPED [INFO] Spark Project SQL API .............................. SKIPPED [INFO] Spark Project Catalyst ............................. SKIPPED [INFO] Spark Project SQL .................................. SKIPPED [INFO] Spark Project ML Library ........................... SKIPPED [INFO] Spark Project Tools ................................ SKIPPED [INFO] Spark Project Hive ................................. SKIPPED [INFO] Spark Project Connect Common ....................... SKIPPED [INFO] Spark Avro ......................................... SKIPPED [INFO] Spark Protobuf ..................................... SKIPPED [INFO] Spark Project REPL ................................. SKIPPED [INFO] Spark Project Connect Server ....................... SKIPPED [INFO] Spark Project Connect Client ....................... SKIPPED [INFO] Spark Project Assembly ............................. SKIPPED [INFO] Kafka 0.10+ Token Provider for Streaming ........... SKIPPED [INFO] Spark Integration for Kafka 0.10 ................... SKIPPED [INFO] Kafka 0.10+ Source for Structured Streaming ........ SKIPPED [INFO] Spark Project Examples ............................. SKIPPED [INFO] Spark Integration for Kafka 0.10 Assembly .......... SKIPPED [INFO] ------------------------------------------------------------------------ [INFO] BUILD FAILURE [INFO] ------------------------------------------------------------------------ [INFO] Total time: 01:15 min [INFO] Finished at: 2024-10-09T23:43:58+08:00 [INFO] ------------------------------------------------------------------------ [ERROR] Failed to execute goal net.alchim31.maven:scala-maven-plugin:4.9.1:doc-jar (attach-scaladocs) on project spark-connect-shims_2.13: MavenReportException: Error while creating archive: wrap: Process exited with an error: 1 (Exit value: 1) -> [Help 1] [ERROR] [ERROR] To see the full stack trace of the errors, re-run Maven with the -e switch. [ERROR] Re-run Maven using the -X switch to enable full debug logging. [ERROR] [ERROR] For more information about the errors and possible solutions, please read the following articles: [ERROR] [Help 1] http://cwiki.apache.org/confluence/display/MAVEN/MojoExecutionException [ERROR] [ERROR] After correcting the problems, you can resume the build with the command [ERROR] mvn -rf :spark-connect-shims_2.13 ``` **After** ``` [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 4.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 2.766 s] [INFO] Spark Project Tags ................................. SUCCESS [ 5.398 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 6.361 s] [INFO] Spark Project Common Utils ......................... SUCCESS [ 16.919 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 8.083 s] [INFO] Spark Project Networking ........................... SUCCESS [ 11.240 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 9.438 s] [INFO] Spark Project Variant .............................. SUCCESS [ 3.697 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 9.939 s] [INFO] Spark Project Connect Shims ........................ SUCCESS [ 2.938 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 6.502 s] [INFO] Spark Project Core ................................. SUCCESS [01:33 min] [INFO] Spark Project ML Local Library ..................... SUCCESS [ 18.220 s] [INFO] Spark Project GraphX ............................... SUCCESS [ 20.923 s] [INFO] Spark Project Streaming ............................ SUCCESS [ 29.949 s] [INFO] Spark Project SQL API .............................. SUCCESS [ 25.842 s] [INFO] Spark Project Catalyst ............................. SUCCESS [02:02 min] [INFO] Spark Project SQL .................................. SUCCESS [02:18 min] [INFO] Spark Project ML Library ........................... SUCCESS [01:38 min] [INFO] Spark Project Tools ................................ SUCCESS [ 3.365 s] [INFO] Spark Project Hive ................................. SUCCESS [ 45.357 s] [INFO] Spark Project Connect Common ....................... SUCCESS [ 33.636 s] [INFO] Spark Avro ......................................... SUCCESS [ 22.040 s] [INFO] Spark Protobuf ..................................... SUCCESS [ 24.557 s] [INFO] Spark Project REPL ................................. SUCCESS [ 13.843 s] [INFO] Spark Project Connect Server ....................... SUCCESS [ 35.587 s] [INFO] Spark Project Connect Client ....................... SUCCESS [ 33.929 s] [INFO] Spark Project Assembly ............................. SUCCESS [ 5.121 s] [INFO] Kafka 0.10+ Token Provider for Streaming ........... SUCCESS [ 12.623 s] [INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 16.908 s] [INFO] Kafka 0.10+ Source for Structured Streaming ........ SUCCESS [ 23.664 s] [INFO] Spark Project Examples ............................. SUCCESS [ 30.777 s] [INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [ 6.997 s] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS [INFO] ------------------------------------------------------------------------ [INFO] Total time: 15:40 min [INFO] Finished at: 2024-10-09T23:27:20+08:00 [INFO] ------------------------------------------------------------------------ ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #48399 from LuciferYang/SPARK-49569-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: yangjie01 --- sql/connect/shims/pom.xml | 7 +++++++ 1 file changed, 7 insertions(+) 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 From f69d03e0cf45ae15fde770cba0340a9bd2e05c28 Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Thu, 10 Oct 2024 08:17:39 +0900 Subject: [PATCH 05/16] [SPARK-43838][SQL][FOLLOWUP] Replace `HashSet` with `HashMap` to improve performance of `DeduplicateRelations` ### What changes were proposed in this pull request? This PR replaces `HashSet` that is currently used with a `HashMap` to improve `DeduplicateRelations` performance. Additionally, this PR reverts #48053 as that change is no longer needed ### Why are the changes needed? Current implementation doesn't utilize `HashSet` properly, but instead performs multiple linear searches on the set creating a O(n^2) complexity ### Does this PR introduce _any_ user-facing change? ### How was this patch tested? Existing tests ### Was this patch authored or co-authored using generative AI tooling? Closes #48392 from mihailotim-db/mihailotim-db/master. Authored-by: Mihailo Timotic Signed-off-by: Hyukjin Kwon --- .../analysis/DeduplicateRelations.scala | 51 ++- .../q22a.sf100/explain.txt | 154 ++++----- .../q22a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q22a/explain.txt | 154 ++++----- .../approved-plans-v2_7/q22a/simplified.txt | 2 +- .../q67a.sf100/explain.txt | 326 +++++++++--------- .../q67a.sf100/simplified.txt | 2 +- .../approved-plans-v2_7/q67a/explain.txt | 326 +++++++++--------- .../approved-plans-v2_7/q67a/simplified.txt | 2 +- 9 files changed, 504 insertions(+), 515 deletions(-) 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/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) From 6ed4bdf7fdbaa4699ce75de95dc94afe61483b77 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 10 Oct 2024 08:23:55 +0900 Subject: [PATCH 06/16] [MINOR][PYTHON][TESTS] Reduce the python worker error log of `test_toDF_with_schema_string` ### What changes were proposed in this pull request? Reduce the python worker error log of `test_toDF_with_schema_string` ### Why are the changes needed? When I run the test locally ```python python/run-tests -k --python-executables python3 --testnames 'pyspark.sql.tests.test_dataframe' ``` Two assertions in `test_toDF_with_schema_string` generate too many python worker error logs (~1k lines), which easily exceed the limitation of terminal and make it hard to debug. So I want to reduce the number of python workers in the two assertions. ### Does this PR introduce _any_ user-facing change? no, test only ### How was this patch tested? manually test, the logs will be reduced to ~200 lines ### Was this patch authored or co-authored using generative AI tooling? no Closes #48388 from zhengruifeng/test_to_df_error. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_dataframe.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index cc43804949e84..4fb3e7a9192c2 100644 --- a/python/pyspark/sql/tests/test_dataframe.py +++ b/python/pyspark/sql/tests/test_dataframe.py @@ -506,14 +506,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. From 7e82e290fb051afd47a542eb6c3509d9bb654ab9 Mon Sep 17 00:00:00 2001 From: Jungtaek Lim Date: Thu, 10 Oct 2024 09:52:02 +0900 Subject: [PATCH 07/16] [SPARK-49905][SQL][SS] Use different ShuffleOrigin for the shuffle required from stateful operators ### What changes were proposed in this pull request? This PR proposes to use different ShuffleOrigin for the shuffle required from stateful operators. Spark has been using ENSURE_REQUIREMENTS as ShuffleOrigin which is open for optimization e.g. AQE can adjust the shuffle spec. Quoting the code of ENSURE_REQUIREMENTS: ``` // Indicates that the shuffle operator was added by the internal `EnsureRequirements` rule. It // means that the shuffle operator is used to ensure internal data partitioning requirements and // Spark is free to optimize it as long as the requirements are still ensured. case object ENSURE_REQUIREMENTS extends ShuffleOrigin ``` But the distribution requirement for stateful operators is lot more strict - it has to use the all expressions to calculate the hash (for partitioning) and the number of shuffle partitions must be the same with the spec. This is because stateful operator assumes that there is 1:1 mapping between the partition for the operator and the "physical" partition for checkpointed state. That said, it is fragile if we allow any optimization to be made against shuffle for stateful operator. To prevent this, this PR introduces a new ShuffleOrigin with note that the shuffle is not expected to be "modified". ### Why are the changes needed? This exposes a possibility of broken state based on the contract. We introduced StatefulOpClusteredDistribution in similar reason. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New UT added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48382 from HeartSaVioR/SPARK-49905. Authored-by: Jungtaek Lim Signed-off-by: Jungtaek Lim --- .../exchange/EnsureRequirements.scala | 11 ++++++++- .../exchange/ShuffleExchangeExec.scala | 5 ++++ .../sql/streaming/StreamingQuerySuite.scala | 24 ++++++++++++++++++- 3 files changed, 38 insertions(+), 2 deletions(-) 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/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 => From ea60e935fb11cb17acd3c1883ed57c52da0a1f7c Mon Sep 17 00:00:00 2001 From: Prashanth Menon Date: Thu, 10 Oct 2024 09:28:20 +0800 Subject: [PATCH 08/16] [SPARK-49918][CORE] Use read-only access to conf in `SparkContext` where appropriate ### What changes were proposed in this pull request? This PR switches all calls to `SparkContext.getConf` that are read-only to use `SparkContext.conf` instead. The former method clones the conf, which is unnecessary when the caller only reads the conf. `SparkContext.conf` provides read-only access to the conf. ### Why are the changes needed? Cloning the entire conf adds unnecessary CPU overhead due to copying, and GC overhead due to cleanup, and both affect tail latencies on certain workloads. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #48402 from pmenon/getconf-optimizations. Authored-by: Prashanth Menon Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/Dependency.scala | 2 +- .../spark/input/PortableDataStream.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 43 +++++++++---------- .../apache/spark/ui/ConsoleProgressBar.scala | 2 +- .../connect/service/SparkConnectService.scala | 2 +- .../org/apache/spark/sql/SparkSession.scala | 2 +- .../execution/datasources/v2/FileScan.scala | 2 +- 7 files changed, 28 insertions(+), 29 deletions(-) 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/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..f41888320a29b 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 " + 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/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/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 55525380aee55..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 @@ -1100,7 +1100,7 @@ object SparkSession extends api.BaseSparkSessionCompanion 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/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))}") } From 0912217b38f474e758f5d0161a24d732a4d58db5 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 10 Oct 2024 11:28:21 +0900 Subject: [PATCH 09/16] [SPARK-48714][PYTHON][FOLLOW-UP] Skip tests if test class is not available ### What changes were proposed in this pull request? This PR is a followup of https://github.com/apache/spark/pull/47086 that makes PySpark tests skipped if tests classes are unavailable. ### Why are the changes needed? `./build/sbt package` should be able to run PySpark tests according to https://spark.apache.org/developer-tools.html and https://spark.apache.org/docs/latest/api/python/development/testing.html ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Manually tested: ```bash build/sbt -Phive clean package python/run-tests -k --python-executables python3 --testnames 'pyspark.sql.tests.test_dataframe' ``` ``` ... Starting test(python3): pyspark.sql.tests.test_dataframe (temp output: /.../spark/python/target/33eca5b9-23e8-4e95-9eca-9f09ce333336/python3__pyspark.sql.tests.test_dataframe__3saz2ymf.log) Finished test(python3): pyspark.sql.tests.test_dataframe (21s) ... 1 tests were skipped Tests passed in 21 seconds Skipped tests in pyspark.sql.tests.test_dataframe with python3: test_df_merge_into (pyspark.sql.tests.test_dataframe.DataFrameTests.test_df_merge_into) ... skip (0.001s) ``` ```bash build/sbt -Phive clean test:package python/run-tests -k --python-executables python3 --testnames 'pyspark.sql.tests.test_dataframe' ``` ``` Starting test(python3): pyspark.sql.tests.test_dataframe (temp output: /.../spark/python/target/710cf488-f39d-49b4-8b04-70044318ea02/python3__pyspark.sql.tests.test_dataframe___95hp4wt.log) Finished test(python3): pyspark.sql.tests.test_dataframe (23s) Tests passed in 23 seconds ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48404 from HyukjinKwon/SPARK-48714-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/tests/test_dataframe.py | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/python/pyspark/sql/tests/test_dataframe.py b/python/pyspark/sql/tests/test_dataframe.py index 4fb3e7a9192c2..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: @@ -779,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 From d7772f27b8a851c5007d7d9c891bb862233bf7fa Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 10 Oct 2024 12:20:13 +0800 Subject: [PATCH 10/16] [SPARK-49782][SQL] ResolveDataFrameDropColumns rule resolves UnresolvedAttribute with child output ### What changes were proposed in this pull request? When the drop list of `DataFrameDropColumns` contains an UnresolvedAttribute. Current rule mistakenly resolve the column with its grand-children's output attributes. In dataframe/dataset API application, issue cannot be encountered since the `dropList` are all AttributeReferences. But when we use Spark LogicalPlan, the bug will be encountered, the UnresolvedAttribute in dropList cannot work. ### Why are the changes needed? In `ResolveDataFrameDropColumns` ```scala val dropped = d.dropList.map { case u: UnresolvedAttribute => resolveExpressionByPlanChildren(u, d.child) //mistakenly resolve the column with its grand-children's output attributes case e => e } ``` To fix it, change to `resolveExpressionByPlanChildren(u, d)` or `resolveExpressionByPlanOutput(u, d.child)` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? UT added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48240 from LantaoJin/SPARK-49782. Authored-by: LantaoJin Signed-off-by: Wenchen Fan --- .../analysis/ResolveDataFrameDropColumns.scala | 2 +- .../spark/sql/catalyst/analysis/AnalysisSuite.scala | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) 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/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) + } } From f0498f056ddfcf82012ccc27b2b86e4ca9af79ba Mon Sep 17 00:00:00 2001 From: Mark Andreev Date: Thu, 10 Oct 2024 08:59:24 +0200 Subject: [PATCH 11/16] [SPARK-49549][SQL] Assign a name to the error conditions _LEGACY_ERROR_TEMP_3055, 3146 ### What changes were proposed in this pull request? Choose a proper name for the error conditions _LEGACY_ERROR_TEMP_3055 and _LEGACY_ERROR_TEMP_3146 defined in core/src/main/resources/error/error-conditions.json. The name should be short but complete (look at the example in error-conditions.json). Add a test which triggers the error from user code if such test still doesn't exist. Check exception fields by using checkError(). The last function checks valuable error fields only, and avoids dependencies from error text message. In this way, tech editors can modify error format in error-conditions.json, and don't worry of Spark's internal tests. Migrate other tests that might trigger the error onto checkError(). ### Why are the changes needed? This changes needed because spark 4 introduce new approach with user friendly error messages. ### Does this PR introduce _any_ user-facing change? Yes, if user's code depends on error condition names. ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48288 from mrk-andreev/SPARK-49549. Authored-by: Mark Andreev Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 22 ++++++++-------- .../main/resources/error/error-states.json | 12 +++++++++ .../catalog/functions/ScalarFunction.java | 8 +++++- .../expressions/V2ExpressionUtils.scala | 5 ++-- .../connector/DataSourceV2FunctionSuite.scala | 25 ++++++++++++++++--- 5 files changed, 55 insertions(+), 17 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index f6317d731c77b..99403e12e62c6 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3996,6 +3996,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.", @@ -7946,11 +7958,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 = ): " @@ -8309,11 +8316,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..9be97556c1076 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", 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/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/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))) From f003638f10157191b2501a547af62ab4e7d8aa43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Du=C5=A1an=20Ti=C5=A1ma?= Date: Thu, 10 Oct 2024 09:11:46 +0200 Subject: [PATCH 12/16] [SPARK-49542][SQL] Partition transform exception evaluate error MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add user facing error for improper use of partition transform expressions. ### Why are the changes needed? Replace internal error with user facing one. ### Does this PR introduce _any_ user-facing change? Yes, new error condition. ### How was this patch tested? Added tests to QueryExecutionErrorsSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #48387 from dusantism-db/partition-transform-exception-evaluate-error. Authored-by: Dušan Tišma Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 6 ++++ .../main/resources/error/error-states.json | 6 ++++ .../tests/connect/test_parity_readwriter.py | 1 + python/pyspark/sql/tests/test_readwriter.py | 30 +++++++++++++++++++ .../sql/catalyst/expressions/Expression.scala | 4 +-- .../expressions/PartitionTransforms.scala | 19 +++++++++++- .../errors/QueryExecutionErrorsSuite.scala | 14 ++++++++- 7 files changed, 76 insertions(+), 4 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 99403e12e62c6..14b228daf3c1a 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3824,6 +3824,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." diff --git a/common/utils/src/main/resources/error/error-states.json b/common/utils/src/main/resources/error/error-states.json index 9be97556c1076..fb899e4eb207e 100644 --- a/common/utils/src/main/resources/error/error-states.json +++ b/common/utils/src/main/resources/error/error-states.json @@ -4913,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/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_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/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/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..86c1f17b4dbb9 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 @@ -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 From 38d66fde2eec7e0d7bc9dda4d87cdee03b1ee9e8 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Thu, 10 Oct 2024 09:15:50 +0200 Subject: [PATCH 13/16] [SPARK-49908][SQL] Assign proper error condition for _LEGACY_ERROR_TEMP_0044 ### What changes were proposed in this pull request? This PR proposes to assign proper error condition & sqlstate for _LEGACY_ERROR_TEMP_0044 ### Why are the changes needed? To improve the error message by assigning proper error condition and SQLSTATE ### Does this PR introduce _any_ user-facing change? No, only user-facing error message improved ### How was this patch tested? Updated the existing tests ### Was this patch authored or co-authored using generative AI tooling? No Closes #48384 from itholic/legacy_0044. Authored-by: Haejoon Lee Signed-off-by: Max Gekk --- .../resources/error/error-conditions.json | 10 +++---- .../spark/sql/errors/QueryParsingErrors.scala | 7 +++-- .../spark/sql/execution/SparkSqlParser.scala | 27 ++++++++++++++++--- .../analyzer-results/timezone.sql.out | 24 ++++++++++++++--- .../sql-tests/results/timezone.sql.out | 24 ++++++++++++++--- .../spark/sql/internal/SQLConfSuite.scala | 4 +-- 6 files changed, 75 insertions(+), 21 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 14b228daf3c1a..4ceef4b2d8b92 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2537,6 +2537,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 ." @@ -5703,11 +5708,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." 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/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/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/scala/org/apache/spark/sql/internal/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala index 1a6cdd1258cc3..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 @@ -492,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)) } From e693af0a0adafe845c90579c02810b4c7d0c2b7d Mon Sep 17 00:00:00 2001 From: Max Gekk Date: Thu, 10 Oct 2024 10:55:35 +0200 Subject: [PATCH 14/16] [SPARK-49748][CORE] Add `getCondition` and deprecate `getErrorClass` in `SparkThrowable` ### What changes were proposed in this pull request? 1. Deprecate the `getErrorClass` method in `SparkThrowable` 2. Add new method `getCondition` as the replacement of `getErrorClass` to the `SparkThrowable` interface 3. Use `getCondition` instead of `getErrorClass` in implementations of `SparkThrowable` to avoid warnings. ### Why are the changes needed? To follow new naming convention proposed by SPARK-46810. ### Does this PR introduce _any_ user-facing change? Yes, it extends existing APIs. ### How was this patch tested? By running the existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48196 from MaxGekk/deprecate-getErrorClass. Authored-by: Max Gekk Signed-off-by: Max Gekk --- .../unsafe/types/CollationFactorySuite.scala | 2 +- .../java/org/apache/spark/SparkThrowable.java | 20 ++++++++--- .../org/apache/spark/SparkException.scala | 34 +++++++++---------- .../apache/spark/SparkThrowableHelper.scala | 4 +-- .../streaming/StreamingQueryException.scala | 2 +- .../spark/sql/avro/AvroLogicalTypeSuite.scala | 2 +- .../org/apache/spark/sql/avro/AvroSuite.scala | 14 ++++---- .../org/apache/spark/sql/CatalogSuite.scala | 2 +- .../spark/sql/ClientDataFrameStatSuite.scala | 6 ++-- .../apache/spark/sql/ClientE2ETestSuite.scala | 20 +++++------ .../client/SparkConnectClientSuite.scala | 2 +- .../client/arrow/ArrowEncoderSuite.scala | 2 +- .../streaming/ClientStreamingQuerySuite.scala | 4 +-- .../spark/sql/kafka010/KafkaExceptions.scala | 2 +- .../spark/memory/SparkOutOfMemoryError.java | 2 +- .../SparkFileAlreadyExistsException.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 4 +-- .../apache/spark/JobCancellationSuite.scala | 2 +- .../org/apache/spark/SparkFunSuite.scala | 2 +- .../apache/spark/SparkThrowableSuite.scala | 12 +++---- .../spark/broadcast/BroadcastSuite.scala | 8 ++--- .../apache/spark/ml/feature/Binarizer.scala | 2 +- .../spark/ml/feature/StringIndexer.scala | 2 +- project/MimaExcludes.scala | 3 ++ .../apache/spark/sql/AnalysisException.scala | 2 +- .../spark/sql/catalyst/parser/parsers.scala | 6 ++-- .../sql/catalyst/analysis/CheckAnalysis.scala | 2 +- .../catalyst/encoders/ExpressionEncoder.scala | 4 +-- .../sql/catalyst/parser/AstBuilder.scala | 4 +-- .../sql/catalyst/util/GeneratedColumn.scala | 4 +-- .../exceptions/SqlScriptingException.scala | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 2 +- .../catalog/CatalogLoadingSuite.java | 2 +- .../sql/catalyst/analysis/AnalysisTest.scala | 2 +- .../analysis/UnsupportedOperationsSuite.scala | 2 +- .../sql/catalyst/csv/CSVExprUtilsSuite.scala | 2 +- .../encoders/EncoderResolutionSuite.scala | 2 +- .../catalyst/encoders/RowEncoderSuite.scala | 2 +- .../BufferHolderSparkSubmitSuite.scala | 4 +-- .../sql/catalyst/parser/DDLParserSuite.scala | 2 +- .../parser/SqlScriptingParserSuite.scala | 8 ++--- .../connect/planner/SparkConnectPlanner.scala | 2 +- .../spark/sql/connect/utils/ErrorUtils.scala | 6 ++-- .../SparkConnectSessionManagerSuite.scala | 10 +++--- .../spark/sql/DataSourceRegistration.scala | 2 +- .../sql/execution/datasources/rules.scala | 8 ++--- .../datasources/v2/FileDataSourceV2.scala | 2 +- .../execution/streaming/StreamExecution.scala | 2 +- .../state/HDFSBackedStateStoreProvider.scala | 2 +- .../state/RocksDBStateStoreProvider.scala | 4 +-- .../spark/sql/internal/CatalogImpl.scala | 8 ++--- .../spark/sql/JavaColumnExpressionSuite.java | 2 +- .../sql/CollationExpressionWalkerSuite.scala | 2 +- .../org/apache/spark/sql/DatasetSuite.scala | 8 ++--- .../spark/sql/LateralColumnAliasSuite.scala | 8 ++--- .../spark/sql/RuntimeNullChecksV2Writes.scala | 4 +-- .../apache/spark/sql/SQLQueryTestHelper.scala | 4 +-- .../org/apache/spark/sql/SubquerySuite.scala | 8 ++--- .../scala/org/apache/spark/sql/UDFSuite.scala | 10 +++--- .../sql/connector/DataSourceV2SQLSuite.scala | 2 +- .../connector/MergeIntoTableSuiteBase.scala | 6 ++-- .../errors/QueryExecutionErrorsSuite.scala | 10 +++--- .../adaptive/AdaptiveQueryExecSuite.scala | 2 +- .../binaryfile/BinaryFileFormatSuite.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 2 +- .../datasources/orc/OrcFilterSuite.scala | 2 +- .../datasources/orc/OrcQuerySuite.scala | 6 ++-- .../datasources/orc/OrcSourceSuite.scala | 4 +-- .../parquet/ParquetFilterSuite.scala | 2 +- .../datasources/parquet/ParquetIOSuite.scala | 6 ++-- .../ParquetPartitionDiscoverySuite.scala | 2 +- .../parquet/ParquetQuerySuite.scala | 2 +- .../parquet/ParquetRebaseDatetimeSuite.scala | 4 +-- .../parquet/ParquetRowIndexSuite.scala | 2 +- .../StateDataSourceChangeDataReadSuite.scala | 8 ++--- .../v2/state/StateDataSourceReadSuite.scala | 2 +- .../python/PythonDataSourceSuite.scala | 8 ++--- .../PythonStreamingDataSourceSuite.scala | 2 +- ...StateSchemaCompatibilityCheckerSuite.scala | 8 ++--- .../streaming/state/StateStoreSuite.scala | 2 +- .../spark/sql/sources/InsertSuite.scala | 2 +- .../StreamingQueryListenerSuite.scala | 2 +- .../thriftserver/HiveThriftServerErrors.scala | 2 +- 83 files changed, 201 insertions(+), 188 deletions(-) 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/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/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/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/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/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/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/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f41888320a29b..4f7338f74e298 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -2892,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/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 3ccb0bddfb0eb..f31a29788aafe 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -195,6 +195,9 @@ object MimaExcludes { 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/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/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/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/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/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/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/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/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 25fd7d13b7d48..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 => 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/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/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/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/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/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 85f296665b6e0..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") { @@ -2121,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/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 86c1f17b4dbb9..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 @@ -293,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 + "\"" @@ -312,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", @@ -334,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", @@ -382,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], @@ -921,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]) } 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/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/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 => From ab1315b95b8e4058d3e97946afc772f05dc519bb Mon Sep 17 00:00:00 2001 From: beliefer Date: Thu, 10 Oct 2024 19:57:18 +0800 Subject: [PATCH 15/16] [SPARK-49756][SQL] Postgres dialect supports pushdown datetime functions ### What changes were proposed in this pull request? This PR propose to make Postgres dialect supports pushdown datetime functions. ### Why are the changes needed? Currently, DS V2 pushdown framework pushed the datetime functions with in a common way. But Postgres doesn't support some datetime functions. ### Does this PR introduce _any_ user-facing change? 'No'. This is a new feature for Postgres dialect. ### How was this patch tested? GA. ### Was this patch authored or co-authored using generative AI tooling? 'No'. Closes #48210 from beliefer/SPARK-49756. Authored-by: beliefer Signed-off-by: Wenchen Fan --- .../jdbc/v2/PostgresIntegrationSuite.scala | 84 +++++++++++++++++++ .../spark/sql/jdbc/PostgresDialect.scala | 25 +++++- 2 files changed, 108 insertions(+), 1 deletion(-) 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 6bb415a928837..05f02a402353b 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 @@ -65,6 +65,17 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCT |) """.stripMargin ).executeUpdate() + connection.prepareStatement( + "CREATE TABLE datetime (name VARCHAR(32), date1 DATE, time1 TIMESTAMP)") + .executeUpdate() + } + + 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 = { @@ -123,4 +134,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/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 From e589ccd9c18e8159ea05471434834246f494a476 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 10 Oct 2024 20:55:34 +0800 Subject: [PATCH 16/16] [SPARK-49920][INFRA] Install `R` for `ubuntu 24.04` when GA run `k8s-integration-tests` ### What changes were proposed in this pull request? The pr aims to install `R` for `ubuntu 24.04` when GA run `k8s-integration-tests`. ### Why are the changes needed? - As the GitHub community switches the default version of `ubuntu-latest` from `ubuntu-22.04` to `ubuntu-24.04`. https://github.com/actions/runner-images/issues/10636 - In `ubuntu-24.04`, `R` is `not installed` by default A.`ubuntu-24.04`(`R` is `not installed` by default) https://github.com/actions/runner-images/blob/main/images/ubuntu/Ubuntu2404-Readme.md#tools image B.`ubuntu-22.04`(`R` is `installed` by default) https://github.com/actions/runner-images/blob/main/images/ubuntu/Ubuntu2204-Readme.md#tools image - Fix the failure issue of GA https://github.com/LuciferYang/spark/actions/runs/11268158324/job/31334445659 image ### Does this PR introduce _any_ user-facing change? No, only for tests. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48406 from panbingkun/install_R. Authored-by: panbingkun Signed-off-by: yangjie01 --- .github/workflows/build_and_test.yml | 4 ++++ 1 file changed, 4 insertions(+) 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: