arrayBufferOfKeys = new ArrayList<>();
+
+ // traverse until the end of input and ensure it returns valid key
+ while (jsonParser.nextValue() != null && jsonParser.currentName() != null) {
+ // add current fieldName to the ArrayBuffer
+ arrayBufferOfKeys.add(UTF8String.fromString(jsonParser.currentName()));
+
+ // skip all the children of inner object or array
+ jsonParser.skipChildren();
+ }
+ return new GenericArrayData(arrayBufferOfKeys.toArray());
+ } catch (IOException e) {
+ return null;
+ }
+ }
+}
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/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
index 721e6a60befe2..12a2879794b10 100644
--- a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
+++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java
@@ -58,7 +58,7 @@ public int numElements() {
private UnsafeArrayData setNullBits(UnsafeArrayData arrayData) {
if (data.hasNull()) {
for (int i = 0; i < length; i++) {
- if (data.isNullAt(i)) {
+ if (data.isNullAt(offset + i)) {
arrayData.setNullAt(i);
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 5d41c07b47842..49f3092390536 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -39,7 +39,6 @@ import org.apache.spark.sql.catalyst.optimizer.OptimizeUpdateFields
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
-import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
import org.apache.spark.sql.catalyst.trees.AlwaysProcess
import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
import org.apache.spark.sql.catalyst.trees.TreePattern._
@@ -203,6 +202,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
with CheckAnalysis with SQLConfHelper with ColumnResolutionHelper {
private val v1SessionCatalog: SessionCatalog = catalogManager.v1SessionCatalog
+ private val relationResolution = new RelationResolution(catalogManager)
override protected def validatePlanChanges(
previousPlan: LogicalPlan,
@@ -972,30 +972,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}
}
- private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty
- private def isReferredTempViewName(nameParts: Seq[String]): Boolean = {
- AnalysisContext.get.referredTempViewNames.exists { n =>
- (n.length == nameParts.length) && n.zip(nameParts).forall {
- case (a, b) => resolver(a, b)
- }
- }
- }
-
- // If we are resolving database objects (relations, functions, etc.) insides views, we may need to
- // expand single or multi-part identifiers with the current catalog and namespace of when the
- // view was created.
- private def expandIdentifier(nameParts: Seq[String]): Seq[String] = {
- if (!isResolvingView || isReferredTempViewName(nameParts)) return nameParts
-
- if (nameParts.length == 1) {
- AnalysisContext.get.catalogAndNamespace :+ nameParts.head
- } else if (catalogManager.isCatalogRegistered(nameParts.head)) {
- nameParts
- } else {
- AnalysisContext.get.catalogAndNamespace.head +: nameParts
- }
- }
-
/**
* Adds metadata columns to output for child relations when nodes are missing resolved attributes.
*
@@ -1122,7 +1098,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
case i @ InsertIntoStatement(table, _, _, _, _, _, _) =>
val relation = table match {
case u: UnresolvedRelation if !u.isStreaming =>
- resolveRelation(u).getOrElse(u)
+ relationResolution.resolveRelation(u).getOrElse(u)
case other => other
}
@@ -1139,7 +1115,7 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
case write: V2WriteCommand =>
write.table match {
case u: UnresolvedRelation if !u.isStreaming =>
- resolveRelation(u).map(unwrapRelationPlan).map {
+ relationResolution.resolveRelation(u).map(unwrapRelationPlan).map {
case v: View => throw QueryCompilationErrors.writeIntoViewNotAllowedError(
v.desc.identifier, write)
case r: DataSourceV2Relation => write.withNewTable(r)
@@ -1154,12 +1130,12 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}
case u: UnresolvedRelation =>
- resolveRelation(u).map(resolveViews).getOrElse(u)
+ relationResolution.resolveRelation(u).map(resolveViews).getOrElse(u)
case r @ RelationTimeTravel(u: UnresolvedRelation, timestamp, version)
if timestamp.forall(ts => ts.resolved && !SubqueryExpression.hasSubquery(ts)) =>
val timeTravelSpec = TimeTravelSpec.create(timestamp, version, conf.sessionLocalTimeZone)
- resolveRelation(u, timeTravelSpec).getOrElse(r)
+ relationResolution.resolveRelation(u, timeTravelSpec).getOrElse(r)
case u @ UnresolvedTable(identifier, cmd, suggestAlternative) =>
lookupTableOrView(identifier).map {
@@ -1194,29 +1170,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}.getOrElse(u)
}
- private def lookupTempView(identifier: Seq[String]): Option[TemporaryViewRelation] = {
- // We are resolving a view and this name is not a temp view when that view was created. We
- // return None earlier here.
- if (isResolvingView && !isReferredTempViewName(identifier)) return None
- v1SessionCatalog.getRawLocalOrGlobalTempView(identifier)
- }
-
- private def resolveTempView(
- identifier: Seq[String],
- isStreaming: Boolean = false,
- isTimeTravel: Boolean = false): Option[LogicalPlan] = {
- lookupTempView(identifier).map { v =>
- val tempViewPlan = v1SessionCatalog.getTempViewRelation(v)
- if (isStreaming && !tempViewPlan.isStreaming) {
- throw QueryCompilationErrors.readNonStreamingTempViewError(identifier.quoted)
- }
- if (isTimeTravel) {
- throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(identifier))
- }
- tempViewPlan
- }
- }
-
/**
* Resolves relations to `ResolvedTable` or `Resolved[Temp/Persistent]View`. This is
* for resolving DDL and misc commands.
@@ -1224,10 +1177,10 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
private def lookupTableOrView(
identifier: Seq[String],
viewOnly: Boolean = false): Option[LogicalPlan] = {
- lookupTempView(identifier).map { tempView =>
+ relationResolution.lookupTempView(identifier).map { tempView =>
ResolvedTempView(identifier.asIdentifier, tempView.tableMeta)
}.orElse {
- expandIdentifier(identifier) match {
+ relationResolution.expandIdentifier(identifier) match {
case CatalogAndIdentifier(catalog, ident) =>
if (viewOnly && !CatalogV2Util.isSessionCatalog(catalog)) {
throw QueryCompilationErrors.catalogOperationNotSupported(catalog, "views")
@@ -1246,113 +1199,6 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
}
}
}
-
- private def createRelation(
- catalog: CatalogPlugin,
- ident: Identifier,
- table: Option[Table],
- options: CaseInsensitiveStringMap,
- isStreaming: Boolean): Option[LogicalPlan] = {
- table.map {
- // To utilize this code path to execute V1 commands, e.g. INSERT,
- // either it must be session catalog, or tracksPartitionsInCatalog
- // must be false so it does not require use catalog to manage partitions.
- // Obviously we cannot execute V1Table by V1 code path if the table
- // is not from session catalog and the table still requires its catalog
- // to manage partitions.
- case v1Table: V1Table if CatalogV2Util.isSessionCatalog(catalog)
- || !v1Table.catalogTable.tracksPartitionsInCatalog =>
- if (isStreaming) {
- if (v1Table.v1Table.tableType == CatalogTableType.VIEW) {
- throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError(
- ident.quoted)
- }
- SubqueryAlias(
- catalog.name +: ident.asMultipartIdentifier,
- UnresolvedCatalogRelation(v1Table.v1Table, options, isStreaming = true))
- } else {
- v1SessionCatalog.getRelation(v1Table.v1Table, options)
- }
-
- case table =>
- if (isStreaming) {
- val v1Fallback = table match {
- case withFallback: V2TableWithV1Fallback =>
- Some(UnresolvedCatalogRelation(withFallback.v1Table, isStreaming = true))
- case _ => None
- }
- SubqueryAlias(
- catalog.name +: ident.asMultipartIdentifier,
- StreamingRelationV2(None, table.name, table, options, table.columns.toAttributes,
- Some(catalog), Some(ident), v1Fallback))
- } else {
- SubqueryAlias(
- catalog.name +: ident.asMultipartIdentifier,
- DataSourceV2Relation.create(table, Some(catalog), Some(ident), options))
- }
- }
- }
-
- /**
- * Resolves relations to v1 relation if it's a v1 table from the session catalog, or to v2
- * relation. This is for resolving DML commands and SELECT queries.
- */
- private def resolveRelation(
- u: UnresolvedRelation,
- timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = {
- val timeTravelSpecFromOptions = TimeTravelSpec.fromOptions(
- u.options,
- conf.getConf(SQLConf.TIME_TRAVEL_TIMESTAMP_KEY),
- conf.getConf(SQLConf.TIME_TRAVEL_VERSION_KEY),
- conf.sessionLocalTimeZone
- )
- if (timeTravelSpec.nonEmpty && timeTravelSpecFromOptions.nonEmpty) {
- throw new AnalysisException("MULTIPLE_TIME_TRAVEL_SPEC", Map.empty[String, String])
- }
- val finalTimeTravelSpec = timeTravelSpec.orElse(timeTravelSpecFromOptions)
- resolveTempView(u.multipartIdentifier, u.isStreaming, finalTimeTravelSpec.isDefined).orElse {
- expandIdentifier(u.multipartIdentifier) match {
- case CatalogAndIdentifier(catalog, ident) =>
- val key =
- ((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq,
- finalTimeTravelSpec)
- AnalysisContext.get.relationCache.get(key).map { cache =>
- val cachedRelation = cache.transform {
- case multi: MultiInstanceRelation =>
- val newRelation = multi.newInstance()
- newRelation.copyTagsFrom(multi)
- newRelation
- }
- u.getTagValue(LogicalPlan.PLAN_ID_TAG).map { planId =>
- val cachedConnectRelation = cachedRelation.clone()
- cachedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId)
- cachedConnectRelation
- }.getOrElse(cachedRelation)
- }.orElse {
- val writePrivilegesString =
- Option(u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES))
- val table = CatalogV2Util.loadTable(
- catalog, ident, finalTimeTravelSpec, writePrivilegesString)
- val loaded = createRelation(
- catalog, ident, table, u.clearWritePrivileges.options, u.isStreaming)
- loaded.foreach(AnalysisContext.get.relationCache.update(key, _))
- u.getTagValue(LogicalPlan.PLAN_ID_TAG).map { planId =>
- loaded.map { loadedRelation =>
- val loadedConnectRelation = loadedRelation.clone()
- loadedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId)
- loadedConnectRelation
- }
- }.getOrElse(loaded)
- }
- case _ => None
- }
- }
- }
-
- /** Consumes an unresolved relation and resolves it to a v1 or v2 relation or temporary view. */
- def resolveRelationOrTempView(u: UnresolvedRelation): LogicalPlan = {
- EliminateSubqueryAliases(resolveRelation(u).getOrElse(u))
- }
}
/** Handle INSERT INTO for DSv2 */
@@ -2135,7 +1981,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
if (ResolveFunctions.lookupBuiltinOrTempFunction(nameParts, Some(f)).isDefined) {
f
} else {
- val CatalogAndIdentifier(catalog, ident) = expandIdentifier(nameParts)
+ val CatalogAndIdentifier(catalog, ident) =
+ relationResolution.expandIdentifier(nameParts)
val fullName =
normalizeFuncName((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq)
if (externalFunctionNameSet.contains(fullName)) {
@@ -2186,7 +2033,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
ResolvedNonPersistentFunc(nameParts.head, V1Function(info))
}
}.getOrElse {
- val CatalogAndIdentifier(catalog, ident) = expandIdentifier(nameParts)
+ val CatalogAndIdentifier(catalog, ident) =
+ relationResolution.expandIdentifier(nameParts)
val fullName = catalog.name +: ident.namespace :+ ident.name
CatalogV2Util.loadFunction(catalog, ident).map { func =>
ResolvedPersistentFunc(catalog.asFunctionCatalog, ident, func)
@@ -2198,7 +2046,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
withPosition(u) {
try {
val resolvedFunc = resolveBuiltinOrTempTableFunction(u.name, u.functionArgs).getOrElse {
- val CatalogAndIdentifier(catalog, ident) = expandIdentifier(u.name)
+ val CatalogAndIdentifier(catalog, ident) =
+ relationResolution.expandIdentifier(u.name)
if (CatalogV2Util.isSessionCatalog(catalog)) {
v1SessionCatalog.resolvePersistentTableFunction(
ident.asFunctionIdentifier, u.functionArgs)
@@ -2355,7 +2204,8 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor
private[analysis] def resolveFunction(u: UnresolvedFunction): Expression = {
withPosition(u) {
resolveBuiltinOrTempFunction(u.nameParts, u.arguments, u).getOrElse {
- val CatalogAndIdentifier(catalog, ident) = expandIdentifier(u.nameParts)
+ val CatalogAndIdentifier(catalog, ident) =
+ relationResolution.expandIdentifier(u.nameParts)
if (CatalogV2Util.isSessionCatalog(catalog)) {
resolveV1Function(ident.asFunctionIdentifier, u.arguments, u)
} else {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index a4f424ba4b421..4720b9dcdfa13 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -1621,7 +1621,7 @@ class PreemptedError() {
// errors have the lowest priority.
def set(error: Exception with SparkThrowable, priority: Option[Int] = None): Unit = {
val calculatedPriority = priority.getOrElse {
- error.getErrorClass match {
+ error.getCondition match {
case c if c.startsWith("INTERNAL_ERROR") => 1
case _ => 2
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
index e22a4b941b30c..8181078c519fc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DeduplicateRelations.scala
@@ -24,20 +24,12 @@ import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.TreePattern._
-/**
- * A helper class used to detect duplicate relations fast in `DeduplicateRelations`. Two relations
- * are duplicated if:
- * 1. they are the same class.
- * 2. they have the same output attribute IDs.
- *
- * The first condition is necessary because the CTE relation definition node and reference node have
- * the same output attribute IDs but they are not duplicated.
- */
-case class RelationWrapper(cls: Class[_], outputAttrIds: Seq[Long])
-
object DeduplicateRelations extends Rule[LogicalPlan] {
+
+ type ExprIdMap = mutable.HashMap[Class[_], mutable.HashSet[Long]]
+
override def apply(plan: LogicalPlan): LogicalPlan = {
- val newPlan = renewDuplicatedRelations(mutable.HashSet.empty, plan)._1
+ val newPlan = renewDuplicatedRelations(mutable.HashMap.empty, plan)._1
// Wait for `ResolveMissingReferences` to resolve missing attributes first
def noMissingInput(p: LogicalPlan) = !p.exists(_.missingInput.nonEmpty)
@@ -86,10 +78,10 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
}
private def existDuplicatedExprId(
- existingRelations: mutable.HashSet[RelationWrapper],
- plan: RelationWrapper): Boolean = {
- existingRelations.filter(_.cls == plan.cls)
- .exists(_.outputAttrIds.intersect(plan.outputAttrIds).nonEmpty)
+ existingRelations: ExprIdMap,
+ planClass: Class[_], exprIds: Seq[Long]): Boolean = {
+ val attrSet = existingRelations.getOrElse(planClass, mutable.HashSet.empty)
+ exprIds.exists(attrSet.contains)
}
/**
@@ -100,20 +92,16 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
* whether the plan is changed or not)
*/
private def renewDuplicatedRelations(
- existingRelations: mutable.HashSet[RelationWrapper],
+ existingRelations: ExprIdMap,
plan: LogicalPlan): (LogicalPlan, Boolean) = plan match {
case p: LogicalPlan if p.isStreaming => (plan, false)
case m: MultiInstanceRelation =>
- val planWrapper = RelationWrapper(m.getClass, m.output.map(_.exprId.id))
- if (existingRelations.contains(planWrapper)) {
- val newNode = m.newInstance()
- newNode.copyTagsFrom(m)
- (newNode, true)
- } else {
- existingRelations.add(planWrapper)
- (m, false)
- }
+ deduplicateAndRenew[LogicalPlan with MultiInstanceRelation](
+ existingRelations,
+ m,
+ _.output.map(_.exprId.id),
+ node => node.newInstance().asInstanceOf[LogicalPlan with MultiInstanceRelation])
case p: Project =>
deduplicateAndRenew[Project](
@@ -207,7 +195,7 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
}
private def deduplicate(
- existingRelations: mutable.HashSet[RelationWrapper],
+ existingRelations: ExprIdMap,
plan: LogicalPlan): (LogicalPlan, Boolean) = {
var planChanged = false
val newPlan = if (plan.children.nonEmpty) {
@@ -291,20 +279,21 @@ object DeduplicateRelations extends Rule[LogicalPlan] {
}
private def deduplicateAndRenew[T <: LogicalPlan](
- existingRelations: mutable.HashSet[RelationWrapper], plan: T,
+ existingRelations: ExprIdMap, plan: T,
getExprIds: T => Seq[Long],
copyNewPlan: T => T): (LogicalPlan, Boolean) = {
var (newPlan, planChanged) = deduplicate(existingRelations, plan)
if (newPlan.resolved) {
val exprIds = getExprIds(newPlan.asInstanceOf[T])
if (exprIds.nonEmpty) {
- val planWrapper = RelationWrapper(newPlan.getClass, exprIds)
- if (existDuplicatedExprId(existingRelations, planWrapper)) {
+ if (existDuplicatedExprId(existingRelations, newPlan.getClass, exprIds)) {
newPlan = copyNewPlan(newPlan.asInstanceOf[T])
newPlan.copyTagsFrom(plan)
(newPlan, true)
} else {
- existingRelations.add(planWrapper)
+ val attrSet = existingRelations.getOrElseUpdate(newPlan.getClass, mutable.HashSet.empty)
+ exprIds.foreach(attrSet.add)
+ existingRelations.put(newPlan.getClass, attrSet)
(newPlan, planChanged)
}
} else {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala
new file mode 100644
index 0000000000000..08be456f090e2
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/RelationResolution.scala
@@ -0,0 +1,245 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.SQLConfHelper
+import org.apache.spark.sql.catalyst.catalog.{
+ CatalogTableType,
+ TemporaryViewRelation,
+ UnresolvedCatalogRelation
+}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.streaming.StreamingRelationV2
+import org.apache.spark.sql.connector.catalog.{
+ CatalogManager,
+ CatalogPlugin,
+ CatalogV2Util,
+ Identifier,
+ LookupCatalog,
+ Table,
+ V1Table,
+ V2TableWithV1Fallback
+}
+import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
+import org.apache.spark.sql.errors.{DataTypeErrorsBase, QueryCompilationErrors}
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+import org.apache.spark.util.ArrayImplicits._
+
+class RelationResolution(override val catalogManager: CatalogManager)
+ extends DataTypeErrorsBase
+ with Logging
+ with LookupCatalog
+ with SQLConfHelper {
+ val v1SessionCatalog = catalogManager.v1SessionCatalog
+
+ /**
+ * If we are resolving database objects (relations, functions, etc.) inside views, we may need to
+ * expand single or multi-part identifiers with the current catalog and namespace of when the
+ * view was created.
+ */
+ def expandIdentifier(nameParts: Seq[String]): Seq[String] = {
+ if (!isResolvingView || isReferredTempViewName(nameParts)) {
+ return nameParts
+ }
+
+ if (nameParts.length == 1) {
+ AnalysisContext.get.catalogAndNamespace :+ nameParts.head
+ } else if (catalogManager.isCatalogRegistered(nameParts.head)) {
+ nameParts
+ } else {
+ AnalysisContext.get.catalogAndNamespace.head +: nameParts
+ }
+ }
+
+ /**
+ * Lookup temporary view by `identifier`. Returns `None` if the view wasn't found.
+ */
+ def lookupTempView(identifier: Seq[String]): Option[TemporaryViewRelation] = {
+ // We are resolving a view and this name is not a temp view when that view was created. We
+ // return None earlier here.
+ if (isResolvingView && !isReferredTempViewName(identifier)) {
+ return None
+ }
+
+ v1SessionCatalog.getRawLocalOrGlobalTempView(identifier)
+ }
+
+ /**
+ * Resolve relation `u` to v1 relation if it's a v1 table from the session catalog, or to v2
+ * relation. This is for resolving DML commands and SELECT queries.
+ */
+ def resolveRelation(
+ u: UnresolvedRelation,
+ timeTravelSpec: Option[TimeTravelSpec] = None): Option[LogicalPlan] = {
+ val timeTravelSpecFromOptions = TimeTravelSpec.fromOptions(
+ u.options,
+ conf.getConf(SQLConf.TIME_TRAVEL_TIMESTAMP_KEY),
+ conf.getConf(SQLConf.TIME_TRAVEL_VERSION_KEY),
+ conf.sessionLocalTimeZone
+ )
+ if (timeTravelSpec.nonEmpty && timeTravelSpecFromOptions.nonEmpty) {
+ throw new AnalysisException("MULTIPLE_TIME_TRAVEL_SPEC", Map.empty[String, String])
+ }
+ val finalTimeTravelSpec = timeTravelSpec.orElse(timeTravelSpecFromOptions)
+ resolveTempView(
+ u.multipartIdentifier,
+ u.isStreaming,
+ finalTimeTravelSpec.isDefined
+ ).orElse {
+ expandIdentifier(u.multipartIdentifier) match {
+ case CatalogAndIdentifier(catalog, ident) =>
+ val key =
+ (
+ (catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq,
+ finalTimeTravelSpec
+ )
+ AnalysisContext.get.relationCache
+ .get(key)
+ .map { cache =>
+ val cachedRelation = cache.transform {
+ case multi: MultiInstanceRelation =>
+ val newRelation = multi.newInstance()
+ newRelation.copyTagsFrom(multi)
+ newRelation
+ }
+ u.getTagValue(LogicalPlan.PLAN_ID_TAG)
+ .map { planId =>
+ val cachedConnectRelation = cachedRelation.clone()
+ cachedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId)
+ cachedConnectRelation
+ }
+ .getOrElse(cachedRelation)
+ }
+ .orElse {
+ val writePrivilegesString =
+ Option(u.options.get(UnresolvedRelation.REQUIRED_WRITE_PRIVILEGES))
+ val table =
+ CatalogV2Util.loadTable(catalog, ident, finalTimeTravelSpec, writePrivilegesString)
+ val loaded = createRelation(
+ catalog,
+ ident,
+ table,
+ u.clearWritePrivileges.options,
+ u.isStreaming
+ )
+ loaded.foreach(AnalysisContext.get.relationCache.update(key, _))
+ u.getTagValue(LogicalPlan.PLAN_ID_TAG)
+ .map { planId =>
+ loaded.map { loadedRelation =>
+ val loadedConnectRelation = loadedRelation.clone()
+ loadedConnectRelation.setTagValue(LogicalPlan.PLAN_ID_TAG, planId)
+ loadedConnectRelation
+ }
+ }
+ .getOrElse(loaded)
+ }
+ case _ => None
+ }
+ }
+ }
+
+ private def createRelation(
+ catalog: CatalogPlugin,
+ ident: Identifier,
+ table: Option[Table],
+ options: CaseInsensitiveStringMap,
+ isStreaming: Boolean): Option[LogicalPlan] = {
+ table.map {
+ // To utilize this code path to execute V1 commands, e.g. INSERT,
+ // either it must be session catalog, or tracksPartitionsInCatalog
+ // must be false so it does not require use catalog to manage partitions.
+ // Obviously we cannot execute V1Table by V1 code path if the table
+ // is not from session catalog and the table still requires its catalog
+ // to manage partitions.
+ case v1Table: V1Table
+ if CatalogV2Util.isSessionCatalog(catalog)
+ || !v1Table.catalogTable.tracksPartitionsInCatalog =>
+ if (isStreaming) {
+ if (v1Table.v1Table.tableType == CatalogTableType.VIEW) {
+ throw QueryCompilationErrors.permanentViewNotSupportedByStreamingReadingAPIError(
+ ident.quoted
+ )
+ }
+ SubqueryAlias(
+ catalog.name +: ident.asMultipartIdentifier,
+ UnresolvedCatalogRelation(v1Table.v1Table, options, isStreaming = true)
+ )
+ } else {
+ v1SessionCatalog.getRelation(v1Table.v1Table, options)
+ }
+
+ case table =>
+ if (isStreaming) {
+ val v1Fallback = table match {
+ case withFallback: V2TableWithV1Fallback =>
+ Some(UnresolvedCatalogRelation(withFallback.v1Table, isStreaming = true))
+ case _ => None
+ }
+ SubqueryAlias(
+ catalog.name +: ident.asMultipartIdentifier,
+ StreamingRelationV2(
+ None,
+ table.name,
+ table,
+ options,
+ table.columns.toAttributes,
+ Some(catalog),
+ Some(ident),
+ v1Fallback
+ )
+ )
+ } else {
+ SubqueryAlias(
+ catalog.name +: ident.asMultipartIdentifier,
+ DataSourceV2Relation.create(table, Some(catalog), Some(ident), options)
+ )
+ }
+ }
+ }
+
+ private def resolveTempView(
+ identifier: Seq[String],
+ isStreaming: Boolean = false,
+ isTimeTravel: Boolean = false): Option[LogicalPlan] = {
+ lookupTempView(identifier).map { v =>
+ val tempViewPlan = v1SessionCatalog.getTempViewRelation(v)
+ if (isStreaming && !tempViewPlan.isStreaming) {
+ throw QueryCompilationErrors.readNonStreamingTempViewError(identifier.quoted)
+ }
+ if (isTimeTravel) {
+ throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(identifier))
+ }
+ tempViewPlan
+ }
+ }
+
+ private def isResolvingView: Boolean = AnalysisContext.get.catalogAndNamespace.nonEmpty
+
+ private def isReferredTempViewName(nameParts: Seq[String]): Boolean = {
+ val resolver = conf.resolver
+ AnalysisContext.get.referredTempViewNames.exists { n =>
+ (n.length == nameParts.length) && n.zip(nameParts).forall {
+ case (a, b) => resolver(a, b)
+ }
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
index 2642b4a1c5daa..0f9b93cc2986d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveDataFrameDropColumns.scala
@@ -36,7 +36,7 @@ class ResolveDataFrameDropColumns(val catalogManager: CatalogManager)
// df.drop(col("non-existing-column"))
val dropped = d.dropList.map {
case u: UnresolvedAttribute =>
- resolveExpressionByPlanChildren(u, d.child)
+ resolveExpressionByPlanChildren(u, d)
case e => e
}
val remaining = d.child.output.filterNot(attr => dropped.exists(_.semanticEquals(attr)))
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
index d7d53230470d9..f2f86a90d5172 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala
@@ -87,7 +87,7 @@ object ExpressionEncoder {
}
constructProjection(row).get(0, anyObjectType).asInstanceOf[T]
} catch {
- case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" =>
+ case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" =>
throw e
case e: Exception =>
throw QueryExecutionErrors.expressionDecodingError(e, expressions)
@@ -115,7 +115,7 @@ object ExpressionEncoder {
inputRow(0) = t
extractProjection(inputRow)
} catch {
- case e: SparkRuntimeException if e.getErrorClass == "NOT_NULL_ASSERT_VIOLATION" =>
+ case e: SparkRuntimeException if e.getCondition == "NOT_NULL_ASSERT_VIOLATION" =>
throw e
case e: Exception =>
throw QueryExecutionErrors.expressionEncodingError(e, expressions)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
index 08cb03edb78b6..38b927f5bbf38 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala
@@ -32,11 +32,11 @@ import org.apache.spark.sql.internal.types.{AbstractMapType, StringTypeWithCaseA
import org.apache.spark.sql.types.{DataType, MapType, StringType, StructType, VariantType}
import org.apache.spark.unsafe.types.UTF8String
-object ExprUtils extends QueryErrorsBase {
+object ExprUtils extends EvalHelper with QueryErrorsBase {
def evalTypeExpr(exp: Expression): DataType = {
if (exp.foldable) {
- exp.eval() match {
+ prepareForEval(exp).eval() match {
case s: UTF8String if s != null =>
val dataType = DataType.parseTypeWithFallback(
s.toString,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index de15ec43c4f31..6a57ba2aaa569 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -383,10 +383,10 @@ abstract class Expression extends TreeNode[Expression] {
trait FoldableUnevaluable extends Expression {
override def foldable: Boolean = true
- final override def eval(input: InternalRow = null): Any =
+ override def eval(input: InternalRow = null): Any =
throw QueryExecutionErrors.cannotEvaluateExpressionError(this)
- final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
throw QueryExecutionErrors.cannotGenerateCodeForExpressionError(this)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala
index 433f8500fab1f..04d31b5797819 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PartitionTransforms.scala
@@ -17,7 +17,11 @@
package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.SparkException
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.trees.UnaryLike
+import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.types.{DataType, IntegerType}
@@ -37,8 +41,21 @@ import org.apache.spark.sql.types.{DataType, IntegerType}
abstract class PartitionTransformExpression extends Expression with Unevaluable
with UnaryLike[Expression] {
override def nullable: Boolean = true
-}
+ override def eval(input: InternalRow): Any =
+ throw new SparkException(
+ errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY",
+ messageParameters = Map("expression" -> toSQLExpr(this)),
+ cause = null
+ )
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
+ throw new SparkException(
+ errorClass = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY",
+ messageParameters = Map("expression" -> toSQLExpr(this)),
+ cause = null
+ )
+}
/**
* Expression for the v2 partition transform years.
*/
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
index 220920a5a3198..d14c8cb675387 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala
@@ -31,6 +31,7 @@ import org.apache.spark.sql.connector.catalog.{FunctionCatalog, Identifier}
import org.apache.spark.sql.connector.catalog.functions._
import org.apache.spark.sql.connector.catalog.functions.ScalarFunction.MAGIC_METHOD_NAME
import org.apache.spark.sql.connector.expressions.{BucketTransform, Expression => V2Expression, FieldReference, IdentityTransform, Literal => V2Literal, NamedReference, NamedTransform, NullOrdering => V2NullOrdering, SortDirection => V2SortDirection, SortOrder => V2SortOrder, SortValue, Transform}
+import org.apache.spark.sql.errors.DataTypeErrors.toSQLId
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.types._
import org.apache.spark.util.ArrayImplicits._
@@ -182,8 +183,8 @@ object V2ExpressionUtils extends SQLConfHelper with Logging {
ApplyFunctionExpression(scalarFunc, arguments)
case _ =>
throw new AnalysisException(
- errorClass = "_LEGACY_ERROR_TEMP_3055",
- messageParameters = Map("scalarFunc" -> scalarFunc.name()))
+ errorClass = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED",
+ messageParameters = Map("scalarFunc" -> toSQLId(scalarFunc.name())))
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
index 0a4882bfada17..3270c6e87e2cd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.trees.UnaryLike
-import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils}
+import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils, UnsafeRowUtils}
import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryErrorsBase}
import org.apache.spark.sql.types._
import org.apache.spark.util.BoundedPriorityQueue
@@ -145,6 +145,7 @@ case class CollectList(
""",
group = "agg_funcs",
since = "2.0.0")
+// TODO: Make CollectSet collation aware
case class CollectSet(
child: Expression,
mutableAggBufferOffset: Int = 0,
@@ -178,14 +179,15 @@ case class CollectSet(
}
override def checkInputDataTypes(): TypeCheckResult = {
- if (!child.dataType.existsRecursively(_.isInstanceOf[MapType])) {
+ if (!child.dataType.existsRecursively(_.isInstanceOf[MapType]) &&
+ UnsafeRowUtils.isBinaryStable(child.dataType)) {
TypeCheckResult.TypeCheckSuccess
} else {
DataTypeMismatch(
errorSubClass = "UNSUPPORTED_INPUT_TYPE",
messageParameters = Map(
"functionName" -> toSQLId(prettyName),
- "dataType" -> toSQLType(MapType)
+ "dataType" -> (s"${toSQLType(MapType)} " + "or \"COLLATED STRING\"")
)
)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index d8ba1fe840bd0..497fdc0936267 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -298,7 +298,7 @@ abstract class BinaryArithmetic extends BinaryOperator
case "+" => "try_add"
case "-" => "try_subtract"
case "*" => "try_multiply"
- case _ => ""
+ case _ => "unknown_function"
}
val overflowCheck = if (failOnError) {
val javaType = CodeGenerator.boxedType(dataType)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
index c091d51fc177f..bb54749126860 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala
@@ -27,6 +27,7 @@ import org.apache.spark.SparkException.internalError
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedSeed}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
+import org.apache.spark.sql.catalyst.expressions.KnownNotContainsNull
import org.apache.spark.sql.catalyst.expressions.codegen._
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
@@ -5330,15 +5331,12 @@ case class ArrayCompact(child: Expression)
child.dataType.asInstanceOf[ArrayType].elementType, true)
lazy val lambda = LambdaFunction(isNotNull(lv), Seq(lv))
- override lazy val replacement: Expression = ArrayFilter(child, lambda)
+ override lazy val replacement: Expression = KnownNotContainsNull(ArrayFilter(child, lambda))
override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType)
override def prettyName: String = "array_compact"
- override def dataType: ArrayType =
- child.dataType.asInstanceOf[ArrayType].copy(containsNull = false)
-
override protected def withNewChildInternal(newChild: Expression): ArrayCompact =
copy(child = newChild)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala
index 75d912633a0fc..f05db0b090c90 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala
@@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, FalseLiteral}
-import org.apache.spark.sql.types.DataType
+import org.apache.spark.sql.types.{ArrayType, DataType}
trait TaggingExpression extends UnaryExpression {
override def nullable: Boolean = child.nullable
@@ -52,6 +52,17 @@ case class KnownNotNull(child: Expression) extends TaggingExpression {
copy(child = newChild)
}
+case class KnownNotContainsNull(child: Expression) extends TaggingExpression {
+ override def dataType: DataType =
+ child.dataType.asInstanceOf[ArrayType].copy(containsNull = false)
+
+ override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode =
+ child.genCode(ctx)
+
+ override protected def withNewChildInternal(newChild: Expression): KnownNotContainsNull =
+ copy(child = newChild)
+}
+
case class KnownFloatingPointNormalized(child: Expression) extends TaggingExpression {
override protected def withNewChildInternal(newChild: Expression): KnownFloatingPointNormalized =
copy(child = newChild)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
index 3a667f370428e..7128190902550 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala
@@ -415,7 +415,7 @@ abstract class HashExpression[E] extends Expression {
protected def genHashString(
ctx: CodegenContext, stringType: StringType, input: String, result: String): String = {
- if (stringType.supportsBinaryEquality) {
+ if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) {
val baseObject = s"$input.getBaseObject()"
val baseOffset = s"$input.getBaseOffset()"
val numBytes = s"$input.numBytes()"
@@ -566,7 +566,7 @@ abstract class InterpretedHashFunction {
hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed)
case s: UTF8String =>
val st = dataType.asInstanceOf[StringType]
- if (st.supportsBinaryEquality) {
+ if (st.supportsBinaryEquality && !st.usesTrimCollation) {
hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed)
} else {
val stringHash = CollationFactory
@@ -817,7 +817,7 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] {
override protected def genHashString(
ctx: CodegenContext, stringType: StringType, input: String, result: String): String = {
- if (stringType.supportsBinaryEquality) {
+ if (stringType.supportsBinaryEquality && !stringType.usesTrimCollation) {
val baseObject = s"$input.getBaseObject()"
val baseOffset = s"$input.getBaseOffset()"
val numBytes = s"$input.numBytes()"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala
new file mode 100644
index 0000000000000..6291e62304a38
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/json/JsonExpressionEvalUtils.scala
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.spark.sql.catalyst.expressions.json
+
+import com.fasterxml.jackson.core.JsonFactory
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.ExprUtils
+import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils
+import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions}
+import org.apache.spark.sql.catalyst.util.{FailFastMode, FailureSafeParser, PermissiveMode}
+import org.apache.spark.sql.errors.QueryCompilationErrors
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType, VariantType}
+import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.util.Utils
+
+object JsonExpressionEvalUtils {
+
+ def schemaOfJson(
+ jsonFactory: JsonFactory,
+ jsonOptions: JSONOptions,
+ jsonInferSchema: JsonInferSchema,
+ json: UTF8String): UTF8String = {
+ val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser =>
+ parser.nextToken()
+ // To match with schema inference from JSON datasource.
+ jsonInferSchema.inferField(parser) match {
+ case st: StructType =>
+ jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil))
+ case at: ArrayType if at.elementType.isInstanceOf[StructType] =>
+ jsonInferSchema
+ .canonicalizeType(at.elementType, jsonOptions)
+ .map(ArrayType(_, containsNull = at.containsNull))
+ .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull))
+ case other: DataType =>
+ jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse(
+ SQLConf.get.defaultStringType)
+ }
+ }
+
+ UTF8String.fromString(dt.sql)
+ }
+}
+
+class JsonToStructsEvaluator(
+ options: Map[String, String],
+ nullableSchema: DataType,
+ nameOfCorruptRecord: String,
+ timeZoneId: Option[String],
+ variantAllowDuplicateKeys: Boolean) extends Serializable {
+
+ // This converts parsed rows to the desired output by the given schema.
+ @transient
+ private lazy val converter = nullableSchema match {
+ case _: StructType =>
+ (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null
+ case _: ArrayType =>
+ (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null
+ case _: MapType =>
+ (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null
+ }
+
+ @transient
+ private lazy val parser = {
+ val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord)
+ val mode = parsedOptions.parseMode
+ if (mode != PermissiveMode && mode != FailFastMode) {
+ throw QueryCompilationErrors.parseModeUnsupportedError("from_json", mode)
+ }
+ val (parserSchema, actualSchema) = nullableSchema match {
+ case s: StructType =>
+ ExprUtils.verifyColumnNameOfCorruptRecord(s, parsedOptions.columnNameOfCorruptRecord)
+ (s, StructType(s.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)))
+ case other =>
+ (StructType(Array(StructField("value", other))), other)
+ }
+
+ val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = false)
+ val createParser = CreateJacksonParser.utf8String _
+
+ new FailureSafeParser[UTF8String](
+ input => rawParser.parse(input, createParser, identity[UTF8String]),
+ mode,
+ parserSchema,
+ parsedOptions.columnNameOfCorruptRecord)
+ }
+
+ final def evaluate(json: UTF8String): Any = {
+ if (json == null) return null
+ nullableSchema match {
+ case _: VariantType =>
+ VariantExpressionEvalUtils.parseJson(json,
+ allowDuplicateKeys = variantAllowDuplicateKeys)
+ case _ =>
+ converter(parser.parse(json))
+ }
+ }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
index bdcf3f0c1eeab..6eef3d6f9d7df 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions
import java.io._
-import scala.collection.mutable.ArrayBuffer
import scala.util.parsing.combinator.RegexParsers
import com.fasterxml.jackson.core._
@@ -31,7 +30,8 @@ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, CodegenFallback, ExprCode}
import org.apache.spark.sql.catalyst.expressions.codegen.Block.BlockHelper
-import org.apache.spark.sql.catalyst.expressions.variant.VariantExpressionEvalUtils
+import org.apache.spark.sql.catalyst.expressions.json.{JsonExpressionEvalUtils, JsonExpressionUtils, JsonToStructsEvaluator}
+import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke
import org.apache.spark.sql.catalyst.json._
import org.apache.spark.sql.catalyst.trees.TreePattern.{JSON_TO_STRUCT, TreePattern}
import org.apache.spark.sql.catalyst.util._
@@ -638,7 +638,6 @@ case class JsonToStructs(
variantAllowDuplicateKeys: Boolean = SQLConf.get.getConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS))
extends UnaryExpression
with TimeZoneAwareExpression
- with CodegenFallback
with ExpectsInputTypes
with NullIntolerant
with QueryErrorsBase {
@@ -646,7 +645,7 @@ case class JsonToStructs(
// The JSON input data might be missing certain fields. We force the nullability
// of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder
// can generate incorrect files if values are missing in columns declared as non-nullable.
- val nullableSchema = schema.asNullable
+ private val nullableSchema: DataType = schema.asNullable
override def nullable: Boolean = true
@@ -679,53 +678,35 @@ case class JsonToStructs(
messageParameters = Map("schema" -> toSQLType(nullableSchema)))
}
- // This converts parsed rows to the desired output by the given schema.
- @transient
- lazy val converter = nullableSchema match {
- case _: StructType =>
- (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null
- case _: ArrayType =>
- (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null
- case _: MapType =>
- (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null
- }
-
- val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD)
- @transient lazy val parser = {
- val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord)
- val mode = parsedOptions.parseMode
- if (mode != PermissiveMode && mode != FailFastMode) {
- throw QueryCompilationErrors.parseModeUnsupportedError("from_json", mode)
- }
- val (parserSchema, actualSchema) = nullableSchema match {
- case s: StructType =>
- ExprUtils.verifyColumnNameOfCorruptRecord(s, parsedOptions.columnNameOfCorruptRecord)
- (s, StructType(s.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)))
- case other =>
- (StructType(Array(StructField("value", other))), other)
- }
-
- val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = false)
- val createParser = CreateJacksonParser.utf8String _
-
- new FailureSafeParser[UTF8String](
- input => rawParser.parse(input, createParser, identity[UTF8String]),
- mode,
- parserSchema,
- parsedOptions.columnNameOfCorruptRecord)
- }
-
override def dataType: DataType = nullableSchema
override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression =
copy(timeZoneId = Option(timeZoneId))
- override def nullSafeEval(json: Any): Any = nullableSchema match {
- case _: VariantType =>
- VariantExpressionEvalUtils.parseJson(json.asInstanceOf[UTF8String],
- allowDuplicateKeys = variantAllowDuplicateKeys)
- case _ =>
- converter(parser.parse(json.asInstanceOf[UTF8String]))
+ @transient
+ private val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD)
+
+ @transient
+ private lazy val evaluator = new JsonToStructsEvaluator(
+ options, nullableSchema, nameOfCorruptRecord, timeZoneId, variantAllowDuplicateKeys)
+
+ override def nullSafeEval(json: Any): Any = evaluator.evaluate(json.asInstanceOf[UTF8String])
+
+ override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = {
+ val refEvaluator = ctx.addReferenceObj("evaluator", evaluator)
+ val eval = child.genCode(ctx)
+ val resultType = CodeGenerator.boxedType(dataType)
+ val resultTerm = ctx.freshName("result")
+ ev.copy(code =
+ code"""
+ |${eval.code}
+ |$resultType $resultTerm = ($resultType) $refEvaluator.evaluate(${eval.value});
+ |boolean ${ev.isNull} = $resultTerm == null;
+ |${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)};
+ |if (!${ev.isNull}) {
+ | ${ev.value} = $resultTerm;
+ |}
+ |""".stripMargin)
}
override def inputTypes: Seq[AbstractDataType] = StringTypeWithCaseAccentSensitivity :: Nil
@@ -877,7 +858,9 @@ case class StructsToJson(
case class SchemaOfJson(
child: Expression,
options: Map[String, String])
- extends UnaryExpression with CodegenFallback with QueryErrorsBase {
+ extends UnaryExpression
+ with RuntimeReplaceable
+ with QueryErrorsBase {
def this(child: Expression) = this(child, Map.empty[String, String])
@@ -918,26 +901,20 @@ case class SchemaOfJson(
}
}
- override def eval(v: InternalRow): Any = {
- val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser =>
- parser.nextToken()
- // To match with schema inference from JSON datasource.
- jsonInferSchema.inferField(parser) match {
- case st: StructType =>
- jsonInferSchema.canonicalizeType(st, jsonOptions).getOrElse(StructType(Nil))
- case at: ArrayType if at.elementType.isInstanceOf[StructType] =>
- jsonInferSchema
- .canonicalizeType(at.elementType, jsonOptions)
- .map(ArrayType(_, containsNull = at.containsNull))
- .getOrElse(ArrayType(StructType(Nil), containsNull = at.containsNull))
- case other: DataType =>
- jsonInferSchema.canonicalizeType(other, jsonOptions).getOrElse(
- SQLConf.get.defaultStringType)
- }
- }
-
- UTF8String.fromString(dt.sql)
- }
+ @transient private lazy val jsonFactoryObjectType = ObjectType(classOf[JsonFactory])
+ @transient private lazy val jsonOptionsObjectType = ObjectType(classOf[JSONOptions])
+ @transient private lazy val jsonInferSchemaObjectType = ObjectType(classOf[JsonInferSchema])
+
+ override def replacement: Expression = StaticInvoke(
+ JsonExpressionEvalUtils.getClass,
+ dataType,
+ "schemaOfJson",
+ Seq(Literal(jsonFactory, jsonFactoryObjectType),
+ Literal(jsonOptions, jsonOptionsObjectType),
+ Literal(jsonInferSchema, jsonInferSchemaObjectType),
+ child),
+ Seq(jsonFactoryObjectType, jsonOptionsObjectType, jsonInferSchemaObjectType, child.dataType)
+ )
override def prettyName: String = "schema_of_json"
@@ -967,54 +944,26 @@ case class SchemaOfJson(
group = "json_funcs",
since = "3.1.0"
)
-case class LengthOfJsonArray(child: Expression) extends UnaryExpression
- with CodegenFallback with ExpectsInputTypes {
+case class LengthOfJsonArray(child: Expression)
+ extends UnaryExpression
+ with ExpectsInputTypes
+ with RuntimeReplaceable {
override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCaseAccentSensitivity)
override def dataType: DataType = IntegerType
override def nullable: Boolean = true
override def prettyName: String = "json_array_length"
- override def eval(input: InternalRow): Any = {
- val json = child.eval(input).asInstanceOf[UTF8String]
- // return null for null input
- if (json == null) {
- return null
- }
-
- try {
- Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) {
- parser => {
- // return null if null array is encountered.
- if (parser.nextToken() == null) {
- return null
- }
- // Parse the array to compute its length.
- parseCounter(parser, input)
- }
- }
- } catch {
- case _: JsonProcessingException | _: IOException => null
- }
- }
-
- private def parseCounter(parser: JsonParser, input: InternalRow): Any = {
- var length = 0
- // Only JSON array are supported for this function.
- if (parser.currentToken != JsonToken.START_ARRAY) {
- return null
- }
- // Keep traversing until the end of JSON array
- while(parser.nextToken() != JsonToken.END_ARRAY) {
- length += 1
- // skip all the child of inner object or array
- parser.skipChildren()
- }
- length
- }
-
override protected def withNewChildInternal(newChild: Expression): LengthOfJsonArray =
copy(child = newChild)
+
+ override def replacement: Expression = StaticInvoke(
+ classOf[JsonExpressionUtils],
+ dataType,
+ "lengthOfJsonArray",
+ Seq(child),
+ inputTypes
+ )
}
/**
@@ -1040,50 +989,23 @@ case class LengthOfJsonArray(child: Expression) extends UnaryExpression
group = "json_funcs",
since = "3.1.0"
)
-case class JsonObjectKeys(child: Expression) extends UnaryExpression with CodegenFallback
- with ExpectsInputTypes {
+case class JsonObjectKeys(child: Expression)
+ extends UnaryExpression
+ with ExpectsInputTypes
+ with RuntimeReplaceable {
override def inputTypes: Seq[AbstractDataType] = Seq(StringTypeWithCaseAccentSensitivity)
override def dataType: DataType = ArrayType(SQLConf.get.defaultStringType)
override def nullable: Boolean = true
override def prettyName: String = "json_object_keys"
- override def eval(input: InternalRow): Any = {
- val json = child.eval(input).asInstanceOf[UTF8String]
- // return null for `NULL` input
- if(json == null) {
- return null
- }
-
- try {
- Utils.tryWithResource(CreateJacksonParser.utf8String(SharedFactory.jsonFactory, json)) {
- parser => {
- // return null if an empty string or any other valid JSON string is encountered
- if (parser.nextToken() == null || parser.currentToken() != JsonToken.START_OBJECT) {
- return null
- }
- // Parse the JSON string to get all the keys of outermost JSON object
- getJsonKeys(parser, input)
- }
- }
- } catch {
- case _: JsonProcessingException | _: IOException => null
- }
- }
-
- private def getJsonKeys(parser: JsonParser, input: InternalRow): GenericArrayData = {
- val arrayBufferOfKeys = ArrayBuffer.empty[UTF8String]
-
- // traverse until the end of input and ensure it returns valid key
- while(parser.nextValue() != null && parser.currentName() != null) {
- // add current fieldName to the ArrayBuffer
- arrayBufferOfKeys += UTF8String.fromString(parser.currentName)
-
- // skip all the children of inner object or array
- parser.skipChildren()
- }
- new GenericArrayData(arrayBufferOfKeys.toArray[Any])
- }
+ override def replacement: Expression = StaticInvoke(
+ classOf[JsonExpressionUtils],
+ dataType,
+ "jsonObjectKeys",
+ Seq(child),
+ inputTypes
+ )
override protected def withNewChildInternal(newChild: Expression): JsonObjectKeys =
copy(child = newChild)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
index 3cec83facd01d..16bdaa1f7f708 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala
@@ -21,7 +21,7 @@ import org.apache.spark.SparkException
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, TypeCheckResult, UnresolvedSeed}
import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.DataTypeMismatch
-import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLType}
+import org.apache.spark.sql.catalyst.expressions.ExpectsInputTypes.{ordinalNumber, toSQLExpr, toSQLId, toSQLType}
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, FalseLiteral}
import org.apache.spark.sql.catalyst.expressions.codegen.Block._
import org.apache.spark.sql.catalyst.trees.{BinaryLike, TernaryLike, UnaryLike}
@@ -263,7 +263,7 @@ case class Uniform(min: Expression, max: Expression, seedExpression: Expression,
result = DataTypeMismatch(
errorSubClass = "NON_FOLDABLE_INPUT",
messageParameters = Map(
- "inputName" -> name,
+ "inputName" -> toSQLId(name),
"inputType" -> requiredType,
"inputExpr" -> toSQLExpr(expr)))
} else expr.dataType match {
@@ -374,14 +374,14 @@ case class RandStr(
var result: TypeCheckResult = TypeCheckResult.TypeCheckSuccess
def requiredType = "INT or SMALLINT"
Seq((length, "length", 0),
- (seedExpression, "seedExpression", 1)).foreach {
+ (seedExpression, "seed", 1)).foreach {
case (expr: Expression, name: String, index: Int) =>
if (result == TypeCheckResult.TypeCheckSuccess) {
if (!expr.foldable) {
result = DataTypeMismatch(
errorSubClass = "NON_FOLDABLE_INPUT",
messageParameters = Map(
- "inputName" -> name,
+ "inputName" -> toSQLId(name),
"inputType" -> requiredType,
"inputExpr" -> toSQLExpr(expr)))
} else expr.dataType match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
index d982e1f19da0c..9c291634401ee 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala
@@ -81,7 +81,8 @@ class JsonInferSchema(options: JSONOptions) extends Serializable with Logging {
*/
def infer[T](
json: RDD[T],
- createParser: (JsonFactory, T) => JsonParser): StructType = {
+ createParser: (JsonFactory, T) => JsonParser,
+ isReadFile: Boolean = false): StructType = {
val parseMode = options.parseMode
val columnNameOfCorruptRecord = options.columnNameOfCorruptRecord
@@ -96,6 +97,9 @@ class JsonInferSchema(options: JSONOptions) extends Serializable with Logging {
Some(inferField(parser))
}
} catch {
+ // If we are not reading from files but hit `RuntimeException`, it means corrupted record.
+ case e: RuntimeException if !isReadFile =>
+ handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, e)
case e @ (_: JsonProcessingException | _: MalformedInputException) =>
handleJsonErrorsByParseMode(parseMode, columnNameOfCorruptRecord, e)
case e: CharConversionException if options.encoding.isEmpty =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 1601d798283c9..c0cd976b9e9b2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -260,19 +260,32 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] {
q.transformExpressionsDownWithPruning(_.containsPattern(BINARY_ARITHMETIC)) {
case a @ Add(_, _, f) if a.deterministic && a.dataType.isInstanceOf[IntegralType] =>
val (foldables, others) = flattenAdd(a, groupingExpressionSet).partition(_.foldable)
- if (foldables.size > 1) {
+ if (foldables.nonEmpty) {
val foldableExpr = foldables.reduce((x, y) => Add(x, y, f))
- val c = Literal.create(foldableExpr.eval(EmptyRow), a.dataType)
- if (others.isEmpty) c else Add(others.reduce((x, y) => Add(x, y, f)), c, f)
+ val foldableValue = foldableExpr.eval(EmptyRow)
+ if (others.isEmpty) {
+ Literal.create(foldableValue, a.dataType)
+ } else if (foldableValue == 0) {
+ others.reduce((x, y) => Add(x, y, f))
+ } else {
+ Add(others.reduce((x, y) => Add(x, y, f)), Literal.create(foldableValue, a.dataType), f)
+ }
} else {
a
}
case m @ Multiply(_, _, f) if m.deterministic && m.dataType.isInstanceOf[IntegralType] =>
val (foldables, others) = flattenMultiply(m, groupingExpressionSet).partition(_.foldable)
- if (foldables.size > 1) {
+ if (foldables.nonEmpty) {
val foldableExpr = foldables.reduce((x, y) => Multiply(x, y, f))
- val c = Literal.create(foldableExpr.eval(EmptyRow), m.dataType)
- if (others.isEmpty) c else Multiply(others.reduce((x, y) => Multiply(x, y, f)), c, f)
+ val foldableValue = foldableExpr.eval(EmptyRow)
+ if (others.isEmpty || (foldableValue == 0 && !m.nullable)) {
+ Literal.create(foldableValue, m.dataType)
+ } else if (foldableValue == 1) {
+ others.reduce((x, y) => Multiply(x, y, f))
+ } else {
+ Multiply(others.reduce((x, y) => Multiply(x, y, f)),
+ Literal.create(foldableValue, m.dataType), f)
+ }
} else {
m
}
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/catalyst/util/UnsafeRowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
index e296b5be6134b..118dd92c3ed54 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/UnsafeRowUtils.scala
@@ -205,7 +205,9 @@ object UnsafeRowUtils {
* can lead to rows being semantically equal even though their binary representations differ).
*/
def isBinaryStable(dataType: DataType): Boolean = !dataType.existsRecursively {
- case st: StringType => !CollationFactory.fetchCollation(st.collationId).supportsBinaryEquality
+ case st: StringType =>
+ val collation = CollationFactory.fetchCollation(st.collationId)
+ (!collation.supportsBinaryEquality)
case _ => false
}
}
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 0e02e4249addd..9dc15c4a1b78d 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
@@ -1691,12 +1691,6 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
messageParameters = Map("className" -> className))
}
- def cannotSaveIntervalIntoExternalStorageError(): Throwable = {
- new AnalysisException(
- errorClass = "_LEGACY_ERROR_TEMP_1136",
- messageParameters = Map.empty)
- }
-
def cannotResolveAttributeError(name: String, outputStr: String): Throwable = {
new AnalysisException(
errorClass = "_LEGACY_ERROR_TEMP_1137",
@@ -1729,10 +1723,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat
"sourceNames" -> sourceNames.mkString(", ")))
}
- def writeEmptySchemasUnsupportedByDataSourceError(): Throwable = {
+ def writeEmptySchemasUnsupportedByDataSourceError(format: String): Throwable = {
new AnalysisException(
- errorClass = "_LEGACY_ERROR_TEMP_1142",
- messageParameters = Map.empty)
+ errorClass = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE",
+ messageParameters = Map("format" -> format))
}
def insertMismatchedColumnNumberError(
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index 301880f1bfc61..ebcc98a3af27a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -870,7 +870,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
def cannotRemoveReservedPropertyError(property: String): SparkUnsupportedOperationException = {
new SparkUnsupportedOperationException(
- errorClass = "_LEGACY_ERROR_TEMP_2069",
+ errorClass = "CANNOT_REMOVE_RESERVED_PROPERTY",
messageParameters = Map("property" -> property))
}
@@ -1112,7 +1112,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
def cannotAcquireMemoryToBuildUnsafeHashedRelationError(): Throwable = {
new SparkOutOfMemoryError(
- "_LEGACY_ERROR_TEMP_2107")
+ "_LEGACY_ERROR_TEMP_2107",
+ new java.util.HashMap[String, String]())
}
def rowLargerThan256MUnsupportedError(): SparkUnsupportedOperationException = {
@@ -1257,6 +1258,12 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
"dataType" -> toSQLType(dataType)))
}
+ def wrongDatatypeInSomeRows(pos: Int, dataType: DataType): SparkSQLException = {
+ new SparkSQLException(
+ errorClass = "COLUMN_ARRAY_ELEMENT_TYPE_MISMATCH",
+ messageParameters = Map("pos" -> pos.toString(), "type" -> toSQLType(dataType)))
+ }
+
def rootConverterReturnNullError(): SparkRuntimeException = {
new SparkRuntimeException(
errorClass = "INVALID_JSON_ROOT_FIELD",
@@ -2275,7 +2282,7 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
def unsupportedUpdateColumnNullabilityError(): SparkSQLFeatureNotSupportedException = {
new SparkSQLFeatureNotSupportedException(
- errorClass = "_LEGACY_ERROR_TEMP_2271",
+ errorClass = "UNSUPPORTED_FEATURE.UPDATE_COLUMN_NULLABILITY",
messageParameters = Map.empty)
}
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/main/scala/org/apache/spark/sql/types/numerics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala
index 1c860e61973c6..ccd9ed209f92a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/numerics.scala
@@ -55,7 +55,12 @@ private[sql] object ByteExactNumeric extends ByteIsIntegral with Ordering.ByteOr
private[sql] object ShortExactNumeric extends ShortIsIntegral with Ordering.ShortOrdering {
- private def checkOverflow(res: Int, x: Short, y: Short, op: String, hint: String): Unit = {
+ private def checkOverflow(
+ res: Int,
+ x: Short,
+ y: Short,
+ op: String,
+ hint: String = "unknown_function"): Unit = {
if (res > Short.MaxValue || res < Short.MinValue) {
throw QueryExecutionErrors.binaryArithmeticCauseOverflowError(x, op, y, hint)
}
diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java
index 0db155e88aea5..339f16407ae60 100644
--- a/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java
+++ b/sql/catalyst/src/test/java/org/apache/spark/sql/connector/catalog/CatalogLoadingSuite.java
@@ -80,7 +80,7 @@ public void testLoadWithoutConfig() {
SparkException exc = Assertions.assertThrows(CatalogNotFoundException.class,
() -> Catalogs.load("missing", conf));
- Assertions.assertEquals(exc.getErrorClass(), "CATALOG_NOT_FOUND");
+ Assertions.assertEquals(exc.getCondition(), "CATALOG_NOT_FOUND");
Assertions.assertEquals(exc.getMessageParameters().get("catalogName"), "`missing`");
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index e23a753dafe8c..8409f454bfb88 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -1832,4 +1832,14 @@ class AnalysisSuite extends AnalysisTest with Matchers {
preemptedError.clear()
assert(preemptedError.getErrorOpt().isEmpty)
}
+
+ test("SPARK-49782: ResolveDataFrameDropColumns rule resolves complex UnresolvedAttribute") {
+ val function = UnresolvedFunction("trim", Seq(UnresolvedAttribute("i")), isDistinct = false)
+ val addColumnF = Project(Seq(UnresolvedAttribute("i"), Alias(function, "f")()), testRelation5)
+ // Drop column "f" via ResolveDataFrameDropColumns rule.
+ val inputPlan = DataFrameDropColumns(Seq(UnresolvedAttribute("f")), addColumnF)
+ // The expected Project (root node) should only have column "i".
+ val expectedPlan = Project(Seq(UnresolvedAttribute("i")), addColumnF).analyze
+ checkAnalysis(inputPlan, expectedPlan)
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
index 33b9fb488c94f..71744f4d15105 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
@@ -205,7 +205,7 @@ trait AnalysisTest extends PlanTest {
assert(e.message.contains(message))
}
if (condition.isDefined) {
- assert(e.getErrorClass == condition.get)
+ assert(e.getCondition == condition.get)
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
index 3e9a93dc743df..6ee19bab5180a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala
@@ -1133,7 +1133,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite with SQLHelper {
}
}
if (!condition.isEmpty) {
- assert(e.getErrorClass == condition)
+ assert(e.getCondition == condition)
}
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala
index e8239c7523948..f3817e4dd1a8b 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala
@@ -106,7 +106,7 @@ class CSVExprUtilsSuite extends SparkFunSuite {
} catch {
case e: SparkIllegalArgumentException =>
assert(separatorStr.isEmpty)
- assert(e.getErrorClass === expectedErrorClass.get)
+ assert(e.getCondition === expectedErrorClass.get)
}
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
index 35a27f41da80a..6bd5b457ea24e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
@@ -173,7 +173,7 @@ class EncoderResolutionSuite extends PlanTest {
val exception = intercept[SparkRuntimeException] {
fromRow(InternalRow(new GenericArrayData(Array(1, null))))
}
- assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION")
}
test("the real number of fields doesn't match encoder schema: tuple encoder") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
index f73911d344d96..79c6d07d6d218 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala
@@ -279,7 +279,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest {
// Check the error class only since the parameters may change depending on how we are running
// this test case.
val exception = intercept[SparkRuntimeException](toRow(encoder, null))
- assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION")
}
test("RowEncoder should validate external type") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
index 6f3890cafd2ac..92ef24bb8ec63 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala
@@ -636,7 +636,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
checkEvaluation(murmur3Hash1, interpretedHash1)
checkEvaluation(murmur3Hash2, interpretedHash2)
- if (CollationFactory.fetchCollation(collation).supportsBinaryEquality) {
+ if (CollationFactory.fetchCollation(collation).isUtf8BinaryType) {
assert(interpretedHash1 != interpretedHash2)
} else {
assert(interpretedHash1 == interpretedHash2)
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/optimizer/OptimizerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala
index 48cdbbe7be539..70a2ae94109fc 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerSuite.scala
@@ -21,13 +21,13 @@ import org.apache.spark.SparkException
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.{Add, Alias, AttributeReference, IntegerLiteral, Literal, Multiply, NamedExpression, Remainder}
+import org.apache.spark.sql.catalyst.expressions.{Add, Alias, ArrayCompact, AttributeReference, CreateArray, CreateStruct, IntegerLiteral, Literal, MapFromEntries, Multiply, NamedExpression, Remainder}
import org.apache.spark.sql.catalyst.expressions.aggregate.Sum
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LocalRelation, LogicalPlan, OneRowRelation, Project}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.internal.SQLConf
-import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.types.{ArrayType, IntegerType, MapType, StructField, StructType}
/**
* A dummy optimizer rule for testing that decrements integer literals until 0.
@@ -313,4 +313,25 @@ class OptimizerSuite extends PlanTest {
assert(message1.contains("not a valid aggregate expression"))
}
}
+
+ test("SPARK-49924: Keep containsNull after ArrayCompact replacement") {
+ val optimizer = new SimpleTestOptimizer() {
+ override def defaultBatches: Seq[Batch] =
+ Batch("test", fixedPoint,
+ ReplaceExpressions) :: Nil
+ }
+
+ val array1 = ArrayCompact(CreateArray(Literal(1) :: Literal.apply(null) :: Nil, false))
+ val plan1 = Project(Alias(array1, "arr")() :: Nil, OneRowRelation()).analyze
+ val optimized1 = optimizer.execute(plan1)
+ assert(optimized1.schema ===
+ StructType(StructField("arr", ArrayType(IntegerType, false), false) :: Nil))
+
+ val struct = CreateStruct(Literal(1) :: Literal(2) :: Nil)
+ val array2 = ArrayCompact(CreateArray(struct :: Literal.apply(null) :: Nil, false))
+ val plan2 = Project(Alias(MapFromEntries(array2), "map")() :: Nil, OneRowRelation()).analyze
+ val optimized2 = optimizer.execute(plan2)
+ assert(optimized2.schema ===
+ StructType(StructField("map", MapType(IntegerType, IntegerType, false), false) :: Nil))
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala
index f4b2fce74dc49..9090e0c7fc104 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReorderAssociativeOperatorSuite.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.Count
import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
@@ -74,4 +75,35 @@ class ReorderAssociativeOperatorSuite extends PlanTest {
comparePlans(optimized, correctAnswer)
}
+
+ test("SPARK-49915: Handle zero and one in associative operators") {
+ val originalQuery =
+ testRelation.select(
+ $"a" + 0,
+ Literal(-3) + $"a" + 3,
+ $"b" * 0 * 1 * 2 * 3,
+ Count($"b") * 0,
+ $"b" * 1 * 1,
+ ($"b" + 0) * 1 * 2 * 3 * 4,
+ $"a" + 0 + $"b" + 0 + $"c" + 0,
+ $"a" + 0 + $"b" * 1 + $"c" + 0
+ )
+
+ val optimized = Optimize.execute(originalQuery.analyze)
+
+ val correctAnswer =
+ testRelation
+ .select(
+ $"a".as("(a + 0)"),
+ $"a".as("((-3 + a) + 3)"),
+ ($"b" * 0).as("((((b * 0) * 1) * 2) * 3)"),
+ Literal(0L).as("(count(b) * 0)"),
+ $"b".as("((b * 1) * 1)"),
+ ($"b" * 24).as("(((((b + 0) * 1) * 2) * 3) * 4)"),
+ ($"a" + $"b" + $"c").as("""(((((a + 0) + b) + 0) + c) + 0)"""),
+ ($"a" + $"b" + $"c").as("((((a + 0) + (b * 1)) + c) + 0)")
+ ).analyze
+
+ comparePlans(optimized, correctAnswer)
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index b7e2490b552cc..926beacc592a5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -3065,7 +3065,7 @@ class DDLParserSuite extends AnalysisTest {
s"(id BIGINT GENERATED ALWAYS AS IDENTITY $identitySpecStr, val INT) USING foo"
)
}
- assert(exception.getErrorClass === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION")
+ assert(exception.getCondition === "IDENTITY_COLUMNS_DUPLICATED_SEQUENCE_GENERATOR_OPTION")
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
index 2972ba2db21de..2e702e5642a92 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingParserSuite.scala
@@ -50,7 +50,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
val e = intercept[ParseException] {
parseScript(sqlScriptText)
}
- assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getCondition === "PARSE_SYNTAX_ERROR")
assert(e.getMessage.contains("Syntax error"))
assert(e.getMessage.contains("SELECT"))
}
@@ -90,7 +90,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
val e = intercept[ParseException] {
parseScript(sqlScriptText)
}
- assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getCondition === "PARSE_SYNTAX_ERROR")
assert(e.getMessage.contains("Syntax error"))
assert(e.getMessage.contains("at or near ';'"))
}
@@ -105,7 +105,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
val e = intercept[ParseException] {
parseScript(sqlScriptText)
}
- assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getCondition === "PARSE_SYNTAX_ERROR")
assert(e.getMessage.contains("Syntax error"))
assert(e.getMessage.contains("at or near end of input"))
}
@@ -367,7 +367,7 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper {
val e = intercept[ParseException] {
parseScript(sqlScriptText)
}
- assert(e.getErrorClass === "PARSE_SYNTAX_ERROR")
+ assert(e.getCondition === "PARSE_SYNTAX_ERROR")
assert(e.getMessage.contains("Syntax error"))
}
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain
index a78195c4ae295..d42d0fd0a46ee 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_array_compact.explain
@@ -1,2 +1,2 @@
-Project [filter(e#0, lambdafunction(isnotnull(lambda arg#0), lambda arg#0, false)) AS array_compact(e)#0]
+Project [knownnotcontainsnull(filter(e#0, lambdafunction(isnotnull(lambda arg#0), lambda arg#0, false))) AS array_compact(e)#0]
+- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain
index 50ab91560e64a..d70e2eb60aba5 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain
@@ -1,2 +1,2 @@
-Project [json_array_length(g#0) AS json_array_length(g)#0]
+Project [static_invoke(JsonExpressionUtils.lengthOfJsonArray(g#0)) AS json_array_length(g)#0]
+- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain
index 30153bb192e55..8a2ea5336c160 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain
@@ -1,2 +1,2 @@
-Project [json_object_keys(g#0) AS json_object_keys(g)#0]
+Project [static_invoke(JsonExpressionUtils.jsonObjectKeys(g#0)) AS json_object_keys(g)#0]
+- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain
index 8ec799bc58084..b400aeeca5af2 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain
@@ -1,2 +1,2 @@
-Project [schema_of_json([{"col":01}]) AS schema_of_json([{"col":01}])#0]
+Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0]
+- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
diff --git a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain
index 13867949177a4..b400aeeca5af2 100644
--- a/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain
+++ b/sql/connect/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain
@@ -1,2 +1,2 @@
-Project [schema_of_json([{"col":01}], (allowNumericLeadingZeros,true)) AS schema_of_json([{"col":01}])#0]
+Project [static_invoke(JsonExpressionEvalUtils.schemaOfJson(com.fasterxml.jackson.core.JsonFactory, org.apache.spark.sql.catalyst.json.JSONOptions, org.apache.spark.sql.catalyst.json.JsonInferSchema, [{"col":01}])) AS schema_of_json([{"col":01}])#0]
+- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0]
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/service/SparkConnectService.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
index 0468a55e23027..e62c19b66c8e5 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala
@@ -345,7 +345,7 @@ object SparkConnectService extends Logging {
val kvStore = sc.statusStore.store.asInstanceOf[ElementTrackingStore]
listener = new SparkConnectServerListener(kvStore, sc.conf)
sc.listenerBus.addToStatusQueue(listener)
- uiTab = if (sc.getConf.get(UI_ENABLED)) {
+ uiTab = if (sc.conf.get(UI_ENABLED)) {
Some(
new SparkConnectServerTab(
new SparkConnectServerAppStatusStore(kvStore),
diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
index f1636ed1ef092..837d4a4d3ee78 100644
--- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
+++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala
@@ -114,8 +114,8 @@ private[connect] object ErrorUtils extends Logging {
case sparkThrowable: SparkThrowable =>
val sparkThrowableBuilder = FetchErrorDetailsResponse.SparkThrowable
.newBuilder()
- if (sparkThrowable.getErrorClass != null) {
- sparkThrowableBuilder.setErrorClass(sparkThrowable.getErrorClass)
+ if (sparkThrowable.getCondition != null) {
+ sparkThrowableBuilder.setErrorClass(sparkThrowable.getCondition)
}
for (queryCtx <- sparkThrowable.getQueryContext) {
val builder = FetchErrorDetailsResponse.QueryContext
@@ -193,7 +193,7 @@ private[connect] object ErrorUtils extends Logging {
if (state != null && state.nonEmpty) {
errorInfo.putMetadata("sqlState", state)
}
- val errorClass = e.getErrorClass
+ val errorClass = e.getCondition
if (errorClass != null && errorClass.nonEmpty) {
val messageParameters = JsonMethods.compact(
JsonMethods.render(map2jvalue(e.getMessageParameters.asScala.toMap)))
diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala
index 42bb93de05e26..1f522ea28b761 100644
--- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala
+++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala
@@ -37,7 +37,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
val exGetOrCreate = intercept[SparkSQLException] {
SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None)
}
- assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.FORMAT")
+ assert(exGetOrCreate.getCondition == "INVALID_HANDLE.FORMAT")
}
test(
@@ -72,7 +72,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
key,
Some(sessionHolder.session.sessionUUID + "invalid"))
}
- assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CHANGED")
+ assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CHANGED")
}
test(
@@ -85,12 +85,12 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
val exGetOrCreate = intercept[SparkSQLException] {
SparkConnectService.sessionManager.getOrCreateIsolatedSession(key, None)
}
- assert(exGetOrCreate.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED")
+ assert(exGetOrCreate.getCondition == "INVALID_HANDLE.SESSION_CLOSED")
val exGet = intercept[SparkSQLException] {
SparkConnectService.sessionManager.getIsolatedSession(key, None)
}
- assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_CLOSED")
+ assert(exGet.getCondition == "INVALID_HANDLE.SESSION_CLOSED")
val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key)
assert(sessionGetIfPresent.isEmpty)
@@ -102,7 +102,7 @@ class SparkConnectSessionManagerSuite extends SharedSparkSession with BeforeAndA
val exGet = intercept[SparkSQLException] {
SparkConnectService.sessionManager.getIsolatedSession(key, None)
}
- assert(exGet.getErrorClass == "INVALID_HANDLE.SESSION_NOT_FOUND")
+ assert(exGet.getCondition == "INVALID_HANDLE.SESSION_NOT_FOUND")
val sessionGetIfPresent = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key)
assert(sessionGetIfPresent.isEmpty)
diff --git a/sql/connect/shims/pom.xml b/sql/connect/shims/pom.xml
index 6bb12a927738c..d177b4a9971f5 100644
--- a/sql/connect/shims/pom.xml
+++ b/sql/connect/shims/pom.xml
@@ -34,6 +34,13 @@
connect-shims
+
+
+ org.scala-lang
+ scala-library
+
+
+
target/scala-${scala.binary.version}/classes
target/scala-${scala.binary.version}/test-classes
diff --git a/sql/core/pom.xml b/sql/core/pom.xml
index 972cf76d27535..16236940fe072 100644
--- a/sql/core/pom.xml
+++ b/sql/core/pom.xml
@@ -79,6 +79,12 @@
${project.version}
test-jar
test
+
+
+ org.apache.spark
+ spark-connect-shims_${scala.binary.version}
+
+
org.apache.spark
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/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/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
index 3504f6e76f79d..08395ef4c347c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala
@@ -143,9 +143,6 @@ private[sql] object PythonSQLUtils extends Logging {
}
}
- def castTimestampNTZToLong(c: Column): Column =
- Column.internalFn("timestamp_ntz_to_long", c)
-
def unresolvedNamedLambdaVariable(name: String): Column =
Column(internal.UnresolvedNamedLambdaVariable.apply(name))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index 8fc860c503c96..9fbe400a555fc 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -189,10 +189,29 @@ class SparkSqlAstBuilder extends AstBuilder {
val key = SQLConf.SESSION_LOCAL_TIMEZONE.key
if (ctx.interval != null) {
val interval = parseIntervalLiteral(ctx.interval)
- if (interval.months != 0 || interval.days != 0 ||
- math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR ||
- interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) {
- throw QueryParsingErrors.intervalValueOutOfRangeError(ctx.interval())
+ if (interval.months != 0) {
+ throw QueryParsingErrors.intervalValueOutOfRangeError(
+ toSQLValue(interval.months),
+ ctx.interval()
+ )
+ }
+ else if (interval.days != 0) {
+ throw QueryParsingErrors.intervalValueOutOfRangeError(
+ toSQLValue(interval.days),
+ ctx.interval()
+ )
+ }
+ else if (math.abs(interval.microseconds) > 18 * DateTimeConstants.MICROS_PER_HOUR) {
+ throw QueryParsingErrors.intervalValueOutOfRangeError(
+ toSQLValue((math.abs(interval.microseconds) / DateTimeConstants.MICROS_PER_HOUR).toInt),
+ ctx.interval()
+ )
+ }
+ else if (interval.microseconds % DateTimeConstants.MICROS_PER_SECOND != 0) {
+ throw QueryParsingErrors.intervalValueOutOfRangeError(
+ toSQLValue((interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt),
+ ctx.interval()
+ )
} else {
val seconds = (interval.microseconds / DateTimeConstants.MICROS_PER_SECOND).toInt
SetCommand(Some(key -> Some(ZoneOffset.ofTotalSeconds(seconds).toString)))
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
index 8f2b7ca5cba25..750b74aab384f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
@@ -682,7 +682,7 @@ case class HashAggregateExec(
| $unsafeRowKeys, $unsafeRowKeyHash);
| if ($unsafeRowBuffer == null) {
| // failed to allocate the first page
- | throw new $oomeClassName("No enough memory for aggregation");
+ | throw new $oomeClassName("_LEGACY_ERROR_TEMP_3302", new java.util.HashMap());
| }
|}
""".stripMargin
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
index 45a71b4da7287..19a36483abe6d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashMapGenerator.scala
@@ -173,7 +173,8 @@ abstract class HashMapGenerator(
${hashBytes(bytes)}
"""
}
- case st: StringType if st.supportsBinaryEquality => hashBytes(s"$input.getBytes()")
+ case st: StringType if st.supportsBinaryEquality =>
+ hashBytes(s"$input.getBytes()")
case st: StringType if !st.supportsBinaryEquality =>
hashLong(s"CollationFactory.fetchCollation(${st.collationId})" +
s".hashFunction.applyAsLong($input)")
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
index 1ebf0d143bd1f..2f1cda9d0f9be 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.execution.aggregate
+import java.util
+
import org.apache.spark.{SparkException, TaskContext}
import org.apache.spark.internal.Logging
import org.apache.spark.memory.SparkOutOfMemoryError
@@ -210,7 +212,7 @@ class TungstenAggregationIterator(
if (buffer == null) {
// failed to allocate the first page
// scalastyle:off throwerror
- throw new SparkOutOfMemoryError("No enough memory for aggregation")
+ throw new SparkOutOfMemoryError("_LEGACY_ERROR_TEMP_3302", new util.HashMap())
// scalastyle:on throwerror
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index 968c204841e46..3698dc2f0808e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -514,7 +514,8 @@ case class DataSource(
dataSource.createRelation(
sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data))
case format: FileFormat =>
- disallowWritingIntervals(outputColumns.map(_.dataType), forbidAnsiIntervals = false)
+ disallowWritingIntervals(
+ outputColumns.toStructType.asNullable, format.toString, forbidAnsiIntervals = false)
val cmd = planForWritingFileFormat(format, mode, data)
val qe = sparkSession.sessionState.executePlan(cmd)
qe.assertCommandExecuted()
@@ -539,8 +540,8 @@ case class DataSource(
}
SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode)
case format: FileFormat =>
- disallowWritingIntervals(data.schema.map(_.dataType), forbidAnsiIntervals = false)
- DataSource.validateSchema(data.schema, sparkSession.sessionState.conf)
+ disallowWritingIntervals(data.schema, format.toString, forbidAnsiIntervals = false)
+ DataSource.validateSchema(format.toString, data.schema, sparkSession.sessionState.conf)
planForWritingFileFormat(format, mode, data)
case _ => throw SparkException.internalError(
s"${providingClass.getCanonicalName} does not allow create table as select.")
@@ -566,12 +567,15 @@ case class DataSource(
}
private def disallowWritingIntervals(
- dataTypes: Seq[DataType],
+ outputColumns: Seq[StructField],
+ format: String,
forbidAnsiIntervals: Boolean): Unit = {
- dataTypes.foreach(
- TypeUtils.invokeOnceForInterval(_, forbidAnsiIntervals) {
- throw QueryCompilationErrors.cannotSaveIntervalIntoExternalStorageError()
- })
+ outputColumns.foreach { field =>
+ TypeUtils.invokeOnceForInterval(field.dataType, forbidAnsiIntervals) {
+ throw QueryCompilationErrors.dataTypeUnsupportedByDataSourceError(
+ format, field
+ )}
+ }
}
}
@@ -838,7 +842,7 @@ object DataSource extends Logging {
* @param schema
* @param conf
*/
- def validateSchema(schema: StructType, conf: SQLConf): Unit = {
+ def validateSchema(formatName: String, schema: StructType, conf: SQLConf): Unit = {
val shouldAllowEmptySchema = conf.getConf(SQLConf.ALLOW_EMPTY_SCHEMAS_FOR_WRITES)
def hasEmptySchema(schema: StructType): Boolean = {
schema.size == 0 || schema.exists {
@@ -849,7 +853,7 @@ object DataSource extends Logging {
if (!shouldAllowEmptySchema && hasEmptySchema(schema)) {
- throw QueryCompilationErrors.writeEmptySchemasUnsupportedByDataSourceError()
+ throw QueryCompilationErrors.writeEmptySchemasUnsupportedByDataSourceError(formatName)
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
index 7946068b9452e..6e79a2f2a3267 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala
@@ -585,14 +585,26 @@ object JdbcUtils extends Logging with SQLConfHelper {
arr => new GenericArrayData(elementConversion(et0)(arr))
}
+ case IntegerType => arrayConverter[Int]((i: Int) => i)
+ case FloatType => arrayConverter[Float]((f: Float) => f)
+ case DoubleType => arrayConverter[Double]((d: Double) => d)
+ case ShortType => arrayConverter[Short]((s: Short) => s)
+ case BooleanType => arrayConverter[Boolean]((b: Boolean) => b)
+ case LongType => arrayConverter[Long]((l: Long) => l)
+
case _ => (array: Object) => array.asInstanceOf[Array[Any]]
}
(rs: ResultSet, row: InternalRow, pos: Int) =>
- val array = nullSafeConvert[java.sql.Array](
- input = rs.getArray(pos + 1),
- array => new GenericArrayData(elementConversion(et)(array.getArray)))
- row.update(pos, array)
+ try {
+ val array = nullSafeConvert[java.sql.Array](
+ input = rs.getArray(pos + 1),
+ array => new GenericArrayData(elementConversion(et)(array.getArray())))
+ row.update(pos, array)
+ } catch {
+ case e: java.lang.ClassCastException =>
+ throw QueryExecutionErrors.wrongDatatypeInSomeRows(pos, dt)
+ }
case NullType =>
(_: ResultSet, row: InternalRow, pos: Int) => row.update(pos, null)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
index 7c98c31bba220..cb4c4f5290880 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala
@@ -164,7 +164,8 @@ object MultiLineJsonDataSource extends JsonDataSource {
.getOrElse(createParser(_: JsonFactory, _: PortableDataStream))
SQLExecution.withSQLConfPropagated(sparkSession) {
- new JsonInferSchema(parsedOptions).infer[PortableDataStream](sampled, parser)
+ new JsonInferSchema(parsedOptions)
+ .infer[PortableDataStream](sampled, parser, isReadFile = true)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
index 29385904a7525..cbbf9f88f89d4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
@@ -89,9 +89,9 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] {
LogicalRelation(ds.resolveRelation())
} catch {
case _: ClassNotFoundException => u
- case e: SparkIllegalArgumentException if e.getErrorClass != null =>
+ case e: SparkIllegalArgumentException if e.getCondition != null =>
u.failAnalysis(
- errorClass = e.getErrorClass,
+ errorClass = e.getCondition,
messageParameters = e.getMessageParameters.asScala.toMap,
cause = e)
case e: Exception if !e.isInstanceOf[AnalysisException] =>
@@ -469,8 +469,8 @@ object PreprocessTableInsertion extends ResolveInsertionBase {
supportColDefaultValue = true)
} catch {
case e: AnalysisException if staticPartCols.nonEmpty &&
- (e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" ||
- e.getErrorClass == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") =>
+ (e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS" ||
+ e.getCondition == "INSERT_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS") =>
val newException = e.copy(
errorClass = Some("INSERT_PARTITION_COLUMN_ARITY_MISMATCH"),
messageParameters = e.messageParameters ++ Map(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
index 168aea5b041f8..4242fc5d8510a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala
@@ -131,7 +131,7 @@ object FileDataSourceV2 {
// The error is already FAILED_READ_FILE, throw it directly. To be consistent, schema
// inference code path throws `FAILED_READ_FILE`, but the file reading code path can reach
// that code path as well and we should not double-wrap the error.
- case e: SparkException if e.getErrorClass == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" =>
+ case e: SparkException if e.getCondition == "FAILED_READ_FILE.CANNOT_READ_FILE_FOOTER" =>
throw e
case e: SchemaColumnConvertNotSupportedException =>
throw QueryExecutionErrors.parquetColumnDataTypeMismatchError(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
index d890107277d6c..5c0f8c0a4afd9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala
@@ -164,7 +164,7 @@ trait FileScan extends Scan
if (splitFiles.length == 1) {
val path = splitFiles(0).toPath
if (!isSplitable(path) && splitFiles(0).length >
- sparkSession.sparkContext.getConf.get(IO_WARNING_LARGEFILETHRESHOLD)) {
+ sparkSession.sparkContext.conf.get(IO_WARNING_LARGEFILETHRESHOLD)) {
logWarning(log"Loading one large unsplittable file ${MDC(PATH, path.toString)} with only " +
log"one partition, the reason is: ${MDC(REASON, getFileUnSplittableReason(path))}")
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala
index cdcf6f21fd008..f4cabcb69d08c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWrite.scala
@@ -96,7 +96,7 @@ trait FileWrite extends Write {
SchemaUtils.checkColumnNameDuplication(
schema.fields.map(_.name).toImmutableArraySeq, caseSensitiveAnalysis)
}
- DataSource.validateSchema(schema, sqlConf)
+ DataSource.validateSchema(formatName, schema, sqlConf)
// TODO: [SPARK-36340] Unify check schema filed of DataSource V2 Insert.
schema.foreach { field =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
index e669165f4f2f8..8ec903f8e61da 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala
@@ -70,7 +70,16 @@ case class EnsureRequirements(
case (child, distribution) =>
val numPartitions = distribution.requiredNumPartitions
.getOrElse(conf.numShufflePartitions)
- ShuffleExchangeExec(distribution.createPartitioning(numPartitions), child, shuffleOrigin)
+ distribution match {
+ case _: StatefulOpClusteredDistribution =>
+ ShuffleExchangeExec(
+ distribution.createPartitioning(numPartitions), child,
+ REQUIRED_BY_STATEFUL_OPERATOR)
+
+ case _ =>
+ ShuffleExchangeExec(
+ distribution.createPartitioning(numPartitions), child, shuffleOrigin)
+ }
}
// Get the indexes of children which have specified distribution requirements and need to be
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala
index ae11229cd516e..31a3f53eb7191 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala
@@ -177,6 +177,11 @@ case object REBALANCE_PARTITIONS_BY_NONE extends ShuffleOrigin
// the output needs to be partitioned by the given columns.
case object REBALANCE_PARTITIONS_BY_COL extends ShuffleOrigin
+// Indicates that the shuffle operator was added by the internal `EnsureRequirements` rule, but
+// was required by a stateful operator. The physical partitioning is static and Spark shouldn't
+// change it.
+case object REQUIRED_BY_STATEFUL_OPERATOR extends ShuffleOrigin
+
/**
* Performs a shuffle that will result in the desired partitioning.
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
index 8f030884ad33b..14adf951f07e8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala
@@ -374,7 +374,7 @@ abstract class StreamExecution(
"message" -> message))
errorClassOpt = e match {
- case t: SparkThrowable => Option(t.getErrorClass)
+ case t: SparkThrowable => Option(t.getCondition)
case _ => None
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
index 884b8aa3853cb..3df63c41dbf97 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
@@ -282,7 +282,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
newMap
}
catch {
- case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") =>
+ case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") =>
throw e
case e: OutOfMemoryError =>
throw QueryExecutionErrors.notEnoughMemoryToLoadStore(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
index 6ab634668bc2a..870ed79ec1747 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala
@@ -389,7 +389,7 @@ private[sql] class RocksDBStateStoreProvider
new RocksDBStateStore(version)
}
catch {
- case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") =>
+ case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") =>
throw e
case e: OutOfMemoryError =>
throw QueryExecutionErrors.notEnoughMemoryToLoadStore(
@@ -409,7 +409,7 @@ private[sql] class RocksDBStateStoreProvider
new RocksDBStateStore(version)
}
catch {
- case e: SparkException if e.getErrorClass.contains("CANNOT_LOAD_STATE_STORE") =>
+ case e: SparkException if e.getCondition.contains("CANNOT_LOAD_STATE_STORE") =>
throw e
case e: OutOfMemoryError =>
throw QueryExecutionErrors.notEnoughMemoryToLoadStore(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 52b8d35e2fbf8..64689e75e2e5e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -177,7 +177,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
try {
Some(makeTable(catalogName +: ns :+ tableName))
} catch {
- case e: AnalysisException if e.getErrorClass == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" =>
+ case e: AnalysisException if e.getCondition == "UNSUPPORTED_FEATURE.HIVE_TABLE_TYPE" =>
Some(new Table(
name = tableName,
catalog = catalogName,
@@ -189,7 +189,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
}
} catch {
- case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => None
+ case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => None
}
}
@@ -203,7 +203,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
case _ => false
}
} catch {
- case e: AnalysisException if e.getErrorClass == "TABLE_OR_VIEW_NOT_FOUND" => false
+ case e: AnalysisException if e.getCondition == "TABLE_OR_VIEW_NOT_FOUND" => false
}
}
@@ -323,7 +323,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
case _ => false
}
} catch {
- case e: AnalysisException if e.getErrorClass == "UNRESOLVED_ROUTINE" => false
+ case e: AnalysisException if e.getCondition == "UNRESOLVED_ROUTINE" => false
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
index 60258ecbb0d61..8341063e09890 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala
@@ -23,6 +23,7 @@ import java.util
import java.util.Locale
import scala.util.Using
+import scala.util.control.NonFatal
import org.apache.spark.SparkThrowable
import org.apache.spark.internal.LogKeys.COLUMN_NAME
@@ -30,7 +31,7 @@ import org.apache.spark.internal.MDC
import org.apache.spark.sql.catalyst.SQLConfHelper
import org.apache.spark.sql.catalyst.analysis.{IndexAlreadyExistsException, NonEmptyNamespaceException, NoSuchIndexException}
import org.apache.spark.sql.connector.catalog.Identifier
-import org.apache.spark.sql.connector.expressions.NamedReference
+import org.apache.spark.sql.connector.expressions.{Expression, NamedReference}
import org.apache.spark.sql.errors.QueryCompilationErrors
import org.apache.spark.sql.execution.datasources.jdbc.{JDBCOptions, JdbcUtils}
import org.apache.spark.sql.execution.datasources.v2.TableSampleInfo
@@ -300,6 +301,28 @@ private case class PostgresDialect()
}
}
+ class PostgresSQLBuilder extends JDBCSQLBuilder {
+ override def visitExtract(field: String, source: String): String = {
+ field match {
+ case "DAY_OF_YEAR" => s"EXTRACT(DOY FROM $source)"
+ case "YEAR_OF_WEEK" => s"EXTRACT(YEAR FROM $source)"
+ case "DAY_OF_WEEK" => s"EXTRACT(DOW FROM $source)"
+ case _ => super.visitExtract(field, source)
+ }
+ }
+ }
+
+ override def compileExpression(expr: Expression): Option[String] = {
+ val postgresSQLBuilder = new PostgresSQLBuilder()
+ try {
+ Some(postgresSQLBuilder.build(expr))
+ } catch {
+ case NonFatal(e) =>
+ logWarning("Error occurs while compiling V2 expression", e)
+ None
+ }
+ }
+
override def supportsLimit: Boolean = true
override def supportsOffset: Boolean = true
diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
index 9fbd1919a2668..9988d04220f0f 100644
--- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
+++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaColumnExpressionSuite.java
@@ -85,7 +85,7 @@ public void isInCollectionCheckExceptionMessage() {
Dataset df = spark.createDataFrame(rows, schema);
AnalysisException e = Assertions.assertThrows(AnalysisException.class,
() -> df.filter(df.col("a").isInCollection(Arrays.asList(new Column("b")))));
- Assertions.assertTrue(e.getErrorClass().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES"));
+ Assertions.assertTrue(e.getCondition().equals("DATATYPE_MISMATCH.DATA_DIFF_TYPES"));
Map messageParameters = new HashMap<>();
messageParameters.put("functionName", "`in`");
messageParameters.put("dataType", "[\"INT\", \"ARRAY\"]");
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
index 26e9394932a17..37d84f6c5fc00 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/null-handling.sql.out
@@ -69,6 +69,24 @@ Project [a#x, (b#x + c#x) AS (b + c)#x]
+- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet
+-- !query
+select b + 0 from t1 where a = 5
+-- !query analysis
+Project [(b#x + 0) AS (b + 0)#x]
++- Filter (a#x = 5)
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet
+
+
+-- !query
+select -100 + b + 100 from t1 where a = 5
+-- !query analysis
+Project [((-100 + b#x) + 100) AS ((-100 + b) + 100)#x]
++- Filter (a#x = 5)
+ +- SubqueryAlias spark_catalog.default.t1
+ +- Relation spark_catalog.default.t1[a#x,b#x,c#x] parquet
+
+
-- !query
select a+10, b*0 from t1
-- !query analysis
diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out
index 133cd6a60a4fb..31919381c99b6 100644
--- a/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out
+++ b/sql/core/src/test/resources/sql-tests/analyzer-results/random.sql.out
@@ -188,7 +188,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "seed",
+ "inputName" : "`seed`",
"inputType" : "integer or floating-point",
"sqlExpr" : "\"uniform(10, 20, col)\""
},
@@ -211,7 +211,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "min",
+ "inputName" : "`min`",
"inputType" : "integer or floating-point",
"sqlExpr" : "\"uniform(col, 10, 0)\""
},
@@ -436,7 +436,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "length",
+ "inputName" : "`length`",
"inputType" : "INT or SMALLINT",
"sqlExpr" : "\"randstr(col, 0)\""
},
@@ -459,7 +459,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "seedExpression",
+ "inputName" : "`seed`",
"inputType" : "INT or SMALLINT",
"sqlExpr" : "\"randstr(10, col)\""
},
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/inputs/null-handling.sql b/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql
index 040be00503442..dcdf241df73d9 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/null-handling.sql
@@ -10,6 +10,8 @@ insert into t1 values(7,null,null);
-- Adding anything to null gives null
select a, b+c from t1;
+select b + 0 from t1 where a = 5;
+select -100 + b + 100 from t1 where a = 5;
-- Multiplying null by zero gives null
select a+10, b*0 from t1;
diff --git a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out
index ece6dbef1605d..fb96be8317a5b 100644
--- a/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/null-handling.sql.out
@@ -77,6 +77,22 @@ struct
7 NULL
+-- !query
+select b + 0 from t1 where a = 5
+-- !query schema
+struct<(b + 0):int>
+-- !query output
+NULL
+
+
+-- !query
+select -100 + b + 100 from t1 where a = 5
+-- !query schema
+struct<((-100 + b) + 100):int>
+-- !query output
+NULL
+
+
-- !query
select a+10, b*0 from t1
-- !query schema
diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out
index 0b4e5e078ee15..01638abdcec6e 100644
--- a/sql/core/src/test/resources/sql-tests/results/random.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out
@@ -240,7 +240,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "seed",
+ "inputName" : "`seed`",
"inputType" : "integer or floating-point",
"sqlExpr" : "\"uniform(10, 20, col)\""
},
@@ -265,7 +265,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "min",
+ "inputName" : "`min`",
"inputType" : "integer or floating-point",
"sqlExpr" : "\"uniform(col, 10, 0)\""
},
@@ -520,7 +520,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "length",
+ "inputName" : "`length`",
"inputType" : "INT or SMALLINT",
"sqlExpr" : "\"randstr(col, 0)\""
},
@@ -545,7 +545,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException
"sqlState" : "42K09",
"messageParameters" : {
"inputExpr" : "\"col\"",
- "inputName" : "seedExpression",
+ "inputName" : "`seed`",
"inputType" : "INT or SMALLINT",
"sqlExpr" : "\"randstr(10, col)\""
},
diff --git a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out
index d34599a49c5ff..5f0fdef50e3db 100644
--- a/sql/core/src/test/resources/sql-tests/results/timezone.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/timezone.sql.out
@@ -80,7 +80,11 @@ struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
{
- "errorClass" : "_LEGACY_ERROR_TEMP_0044",
+ "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE",
+ "sqlState" : "22006",
+ "messageParameters" : {
+ "input" : "3"
+ },
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
@@ -98,7 +102,11 @@ struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
{
- "errorClass" : "_LEGACY_ERROR_TEMP_0044",
+ "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE",
+ "sqlState" : "22006",
+ "messageParameters" : {
+ "input" : "24"
+ },
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
@@ -116,7 +124,11 @@ struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
{
- "errorClass" : "_LEGACY_ERROR_TEMP_0044",
+ "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE",
+ "sqlState" : "22006",
+ "messageParameters" : {
+ "input" : "19"
+ },
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
@@ -152,7 +164,11 @@ struct<>
-- !query output
org.apache.spark.sql.catalyst.parser.ParseException
{
- "errorClass" : "_LEGACY_ERROR_TEMP_0044",
+ "errorClass" : "INVALID_INTERVAL_FORMAT.TIMEZONE_INTERVAL_OUT_OF_RANGE",
+ "sqlState" : "22006",
+ "messageParameters" : {
+ "input" : "36000"
+ },
"queryContext" : [ {
"objectType" : "",
"objectName" : "",
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
index 96bed479d2e06..4bf7de791b279 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/explain.txt
@@ -175,125 +175,125 @@ Input [6]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, sum#21, cou
Keys [4]: [i_product_name#12, i_brand#9, i_class#10, i_category#11]
Functions [1]: [avg(qoh#18)]
Aggregate Attributes [1]: [avg(qoh#18)#23]
-Results [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, avg(qoh#18)#23 AS qoh#24]
+Results [5]: [i_product_name#12 AS i_product_name#24, i_brand#9 AS i_brand#25, i_class#10 AS i_class#26, i_category#11 AS i_category#27, avg(qoh#18)#23 AS qoh#28]
(27) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30]
+Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34]
(28) HashAggregate [codegen id : 16]
-Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30]
-Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28]
-Functions [1]: [avg(inv_quantity_on_hand#31)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17]
-Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32]
+Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34]
+Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32]
+Functions [1]: [avg(inv_quantity_on_hand#35)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17]
+Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36]
(29) HashAggregate [codegen id : 16]
-Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32]
-Keys [3]: [i_product_name#25, i_brand#26, i_class#27]
-Functions [1]: [partial_avg(qoh#32)]
-Aggregate Attributes [2]: [sum#33, count#34]
-Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
+Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36]
+Keys [3]: [i_product_name#29, i_brand#30, i_class#31]
+Functions [1]: [partial_avg(qoh#36)]
+Aggregate Attributes [2]: [sum#37, count#38]
+Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
(30) Exchange
-Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
-Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
+Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=5]
(31) HashAggregate [codegen id : 17]
-Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
-Keys [3]: [i_product_name#25, i_brand#26, i_class#27]
-Functions [1]: [avg(qoh#32)]
-Aggregate Attributes [1]: [avg(qoh#32)#37]
-Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39]
+Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
+Keys [3]: [i_product_name#29, i_brand#30, i_class#31]
+Functions [1]: [avg(qoh#36)]
+Aggregate Attributes [1]: [avg(qoh#36)#41]
+Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43]
(32) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45]
+Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49]
(33) HashAggregate [codegen id : 25]
-Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45]
-Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43]
-Functions [1]: [avg(inv_quantity_on_hand#46)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17]
-Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47]
+Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49]
+Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47]
+Functions [1]: [avg(inv_quantity_on_hand#50)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17]
+Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51]
(34) HashAggregate [codegen id : 25]
-Input [3]: [i_product_name#40, i_brand#41, qoh#47]
-Keys [2]: [i_product_name#40, i_brand#41]
-Functions [1]: [partial_avg(qoh#47)]
-Aggregate Attributes [2]: [sum#48, count#49]
-Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
+Input [3]: [i_product_name#44, i_brand#45, qoh#51]
+Keys [2]: [i_product_name#44, i_brand#45]
+Functions [1]: [partial_avg(qoh#51)]
+Aggregate Attributes [2]: [sum#52, count#53]
+Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
(35) Exchange
-Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
-Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
+Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=6]
(36) HashAggregate [codegen id : 26]
-Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
-Keys [2]: [i_product_name#40, i_brand#41]
-Functions [1]: [avg(qoh#47)]
-Aggregate Attributes [1]: [avg(qoh#47)#52]
-Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55]
+Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
+Keys [2]: [i_product_name#44, i_brand#45]
+Functions [1]: [avg(qoh#51)]
+Aggregate Attributes [1]: [avg(qoh#51)#56]
+Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59]
(37) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61]
+Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65]
(38) HashAggregate [codegen id : 34]
-Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61]
-Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59]
-Functions [1]: [avg(inv_quantity_on_hand#62)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17]
-Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63]
+Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65]
+Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63]
+Functions [1]: [avg(inv_quantity_on_hand#66)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17]
+Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67]
(39) HashAggregate [codegen id : 34]
-Input [2]: [i_product_name#56, qoh#63]
-Keys [1]: [i_product_name#56]
-Functions [1]: [partial_avg(qoh#63)]
-Aggregate Attributes [2]: [sum#64, count#65]
-Results [3]: [i_product_name#56, sum#66, count#67]
+Input [2]: [i_product_name#60, qoh#67]
+Keys [1]: [i_product_name#60]
+Functions [1]: [partial_avg(qoh#67)]
+Aggregate Attributes [2]: [sum#68, count#69]
+Results [3]: [i_product_name#60, sum#70, count#71]
(40) Exchange
-Input [3]: [i_product_name#56, sum#66, count#67]
-Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Input [3]: [i_product_name#60, sum#70, count#71]
+Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=7]
(41) HashAggregate [codegen id : 35]
-Input [3]: [i_product_name#56, sum#66, count#67]
-Keys [1]: [i_product_name#56]
-Functions [1]: [avg(qoh#63)]
-Aggregate Attributes [1]: [avg(qoh#63)#68]
-Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72]
+Input [3]: [i_product_name#60, sum#70, count#71]
+Keys [1]: [i_product_name#60]
+Functions [1]: [avg(qoh#67)]
+Aggregate Attributes [1]: [avg(qoh#67)#72]
+Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76]
(42) ReusedExchange [Reuses operator id: 23]
-Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78]
+Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82]
(43) HashAggregate [codegen id : 43]
-Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78]
-Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76]
-Functions [1]: [avg(inv_quantity_on_hand#79)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17]
-Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80]
+Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82]
+Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80]
+Functions [1]: [avg(inv_quantity_on_hand#83)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17]
+Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84]
(44) HashAggregate [codegen id : 43]
-Input [1]: [qoh#80]
+Input [1]: [qoh#84]
Keys: []
-Functions [1]: [partial_avg(qoh#80)]
-Aggregate Attributes [2]: [sum#81, count#82]
-Results [2]: [sum#83, count#84]
+Functions [1]: [partial_avg(qoh#84)]
+Aggregate Attributes [2]: [sum#85, count#86]
+Results [2]: [sum#87, count#88]
(45) Exchange
-Input [2]: [sum#83, count#84]
+Input [2]: [sum#87, count#88]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=8]
(46) HashAggregate [codegen id : 44]
-Input [2]: [sum#83, count#84]
+Input [2]: [sum#87, count#88]
Keys: []
-Functions [1]: [avg(qoh#80)]
-Aggregate Attributes [1]: [avg(qoh#80)#85]
-Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90]
+Functions [1]: [avg(qoh#84)]
+Aggregate Attributes [1]: [avg(qoh#84)#89]
+Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94]
(47) Union
(48) TakeOrderedAndProject
-Input [5]: [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24]
-Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#12 ASC NULLS FIRST, i_brand#9 ASC NULLS FIRST, i_class#10 ASC NULLS FIRST, i_category#11 ASC NULLS FIRST], [i_product_name#12, i_brand#9, i_class#10, i_category#11, qoh#24]
+Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28]
+Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28]
===== Subqueries =====
@@ -306,22 +306,22 @@ BroadcastExchange (53)
(49) Scan parquet spark_catalog.default.date_dim
-Output [2]: [d_date_sk#7, d_month_seq#91]
+Output [2]: [d_date_sk#7, d_month_seq#95]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
ReadSchema: struct
(50) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#7, d_month_seq#91]
+Input [2]: [d_date_sk#7, d_month_seq#95]
(51) Filter [codegen id : 1]
-Input [2]: [d_date_sk#7, d_month_seq#91]
-Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#7))
+Input [2]: [d_date_sk#7, d_month_seq#95]
+Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#7))
(52) Project [codegen id : 1]
Output [1]: [d_date_sk#7]
-Input [2]: [d_date_sk#7, d_month_seq#91]
+Input [2]: [d_date_sk#7, d_month_seq#95]
(53) BroadcastExchange
Input [1]: [d_date_sk#7]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt
index 0c4267b3ca513..042f946b8fca4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a.sf100/simplified.txt
@@ -1,7 +1,7 @@
TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
Union
WholeStageCodegen (8)
- HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count]
+ HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count]
HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count]
HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count]
InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
index 4b8993f370f4d..8aab8e91acfc8 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/explain.txt
@@ -160,125 +160,125 @@ Input [6]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, sum#21, coun
Keys [4]: [i_product_name#11, i_brand#8, i_class#9, i_category#10]
Functions [1]: [avg(qoh#18)]
Aggregate Attributes [1]: [avg(qoh#18)#23]
-Results [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, avg(qoh#18)#23 AS qoh#24]
+Results [5]: [i_product_name#11 AS i_product_name#24, i_brand#8 AS i_brand#25, i_class#9 AS i_class#26, i_category#10 AS i_category#27, avg(qoh#18)#23 AS qoh#28]
(24) ReusedExchange [Reuses operator id: 20]
-Output [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30]
+Output [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34]
(25) HashAggregate [codegen id : 10]
-Input [6]: [i_product_name#25, i_brand#26, i_class#27, i_category#28, sum#29, count#30]
-Keys [4]: [i_product_name#25, i_brand#26, i_class#27, i_category#28]
-Functions [1]: [avg(inv_quantity_on_hand#31)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#31)#17]
-Results [4]: [i_product_name#25, i_brand#26, i_class#27, avg(inv_quantity_on_hand#31)#17 AS qoh#32]
+Input [6]: [i_product_name#29, i_brand#30, i_class#31, i_category#32, sum#33, count#34]
+Keys [4]: [i_product_name#29, i_brand#30, i_class#31, i_category#32]
+Functions [1]: [avg(inv_quantity_on_hand#35)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#35)#17]
+Results [4]: [i_product_name#29, i_brand#30, i_class#31, avg(inv_quantity_on_hand#35)#17 AS qoh#36]
(26) HashAggregate [codegen id : 10]
-Input [4]: [i_product_name#25, i_brand#26, i_class#27, qoh#32]
-Keys [3]: [i_product_name#25, i_brand#26, i_class#27]
-Functions [1]: [partial_avg(qoh#32)]
-Aggregate Attributes [2]: [sum#33, count#34]
-Results [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
+Input [4]: [i_product_name#29, i_brand#30, i_class#31, qoh#36]
+Keys [3]: [i_product_name#29, i_brand#30, i_class#31]
+Functions [1]: [partial_avg(qoh#36)]
+Aggregate Attributes [2]: [sum#37, count#38]
+Results [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
(27) Exchange
-Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
-Arguments: hashpartitioning(i_product_name#25, i_brand#26, i_class#27, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
+Arguments: hashpartitioning(i_product_name#29, i_brand#30, i_class#31, 5), ENSURE_REQUIREMENTS, [plan_id=4]
(28) HashAggregate [codegen id : 11]
-Input [5]: [i_product_name#25, i_brand#26, i_class#27, sum#35, count#36]
-Keys [3]: [i_product_name#25, i_brand#26, i_class#27]
-Functions [1]: [avg(qoh#32)]
-Aggregate Attributes [1]: [avg(qoh#32)#37]
-Results [5]: [i_product_name#25, i_brand#26, i_class#27, null AS i_category#38, avg(qoh#32)#37 AS qoh#39]
+Input [5]: [i_product_name#29, i_brand#30, i_class#31, sum#39, count#40]
+Keys [3]: [i_product_name#29, i_brand#30, i_class#31]
+Functions [1]: [avg(qoh#36)]
+Aggregate Attributes [1]: [avg(qoh#36)#41]
+Results [5]: [i_product_name#29, i_brand#30, i_class#31, null AS i_category#42, avg(qoh#36)#41 AS qoh#43]
(29) ReusedExchange [Reuses operator id: 20]
-Output [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45]
+Output [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49]
(30) HashAggregate [codegen id : 16]
-Input [6]: [i_product_name#40, i_brand#41, i_class#42, i_category#43, sum#44, count#45]
-Keys [4]: [i_product_name#40, i_brand#41, i_class#42, i_category#43]
-Functions [1]: [avg(inv_quantity_on_hand#46)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#46)#17]
-Results [3]: [i_product_name#40, i_brand#41, avg(inv_quantity_on_hand#46)#17 AS qoh#47]
+Input [6]: [i_product_name#44, i_brand#45, i_class#46, i_category#47, sum#48, count#49]
+Keys [4]: [i_product_name#44, i_brand#45, i_class#46, i_category#47]
+Functions [1]: [avg(inv_quantity_on_hand#50)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#50)#17]
+Results [3]: [i_product_name#44, i_brand#45, avg(inv_quantity_on_hand#50)#17 AS qoh#51]
(31) HashAggregate [codegen id : 16]
-Input [3]: [i_product_name#40, i_brand#41, qoh#47]
-Keys [2]: [i_product_name#40, i_brand#41]
-Functions [1]: [partial_avg(qoh#47)]
-Aggregate Attributes [2]: [sum#48, count#49]
-Results [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
+Input [3]: [i_product_name#44, i_brand#45, qoh#51]
+Keys [2]: [i_product_name#44, i_brand#45]
+Functions [1]: [partial_avg(qoh#51)]
+Aggregate Attributes [2]: [sum#52, count#53]
+Results [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
(32) Exchange
-Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
-Arguments: hashpartitioning(i_product_name#40, i_brand#41, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
+Arguments: hashpartitioning(i_product_name#44, i_brand#45, 5), ENSURE_REQUIREMENTS, [plan_id=5]
(33) HashAggregate [codegen id : 17]
-Input [4]: [i_product_name#40, i_brand#41, sum#50, count#51]
-Keys [2]: [i_product_name#40, i_brand#41]
-Functions [1]: [avg(qoh#47)]
-Aggregate Attributes [1]: [avg(qoh#47)#52]
-Results [5]: [i_product_name#40, i_brand#41, null AS i_class#53, null AS i_category#54, avg(qoh#47)#52 AS qoh#55]
+Input [4]: [i_product_name#44, i_brand#45, sum#54, count#55]
+Keys [2]: [i_product_name#44, i_brand#45]
+Functions [1]: [avg(qoh#51)]
+Aggregate Attributes [1]: [avg(qoh#51)#56]
+Results [5]: [i_product_name#44, i_brand#45, null AS i_class#57, null AS i_category#58, avg(qoh#51)#56 AS qoh#59]
(34) ReusedExchange [Reuses operator id: 20]
-Output [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61]
+Output [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65]
(35) HashAggregate [codegen id : 22]
-Input [6]: [i_product_name#56, i_brand#57, i_class#58, i_category#59, sum#60, count#61]
-Keys [4]: [i_product_name#56, i_brand#57, i_class#58, i_category#59]
-Functions [1]: [avg(inv_quantity_on_hand#62)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#62)#17]
-Results [2]: [i_product_name#56, avg(inv_quantity_on_hand#62)#17 AS qoh#63]
+Input [6]: [i_product_name#60, i_brand#61, i_class#62, i_category#63, sum#64, count#65]
+Keys [4]: [i_product_name#60, i_brand#61, i_class#62, i_category#63]
+Functions [1]: [avg(inv_quantity_on_hand#66)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#66)#17]
+Results [2]: [i_product_name#60, avg(inv_quantity_on_hand#66)#17 AS qoh#67]
(36) HashAggregate [codegen id : 22]
-Input [2]: [i_product_name#56, qoh#63]
-Keys [1]: [i_product_name#56]
-Functions [1]: [partial_avg(qoh#63)]
-Aggregate Attributes [2]: [sum#64, count#65]
-Results [3]: [i_product_name#56, sum#66, count#67]
+Input [2]: [i_product_name#60, qoh#67]
+Keys [1]: [i_product_name#60]
+Functions [1]: [partial_avg(qoh#67)]
+Aggregate Attributes [2]: [sum#68, count#69]
+Results [3]: [i_product_name#60, sum#70, count#71]
(37) Exchange
-Input [3]: [i_product_name#56, sum#66, count#67]
-Arguments: hashpartitioning(i_product_name#56, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Input [3]: [i_product_name#60, sum#70, count#71]
+Arguments: hashpartitioning(i_product_name#60, 5), ENSURE_REQUIREMENTS, [plan_id=6]
(38) HashAggregate [codegen id : 23]
-Input [3]: [i_product_name#56, sum#66, count#67]
-Keys [1]: [i_product_name#56]
-Functions [1]: [avg(qoh#63)]
-Aggregate Attributes [1]: [avg(qoh#63)#68]
-Results [5]: [i_product_name#56, null AS i_brand#69, null AS i_class#70, null AS i_category#71, avg(qoh#63)#68 AS qoh#72]
+Input [3]: [i_product_name#60, sum#70, count#71]
+Keys [1]: [i_product_name#60]
+Functions [1]: [avg(qoh#67)]
+Aggregate Attributes [1]: [avg(qoh#67)#72]
+Results [5]: [i_product_name#60, null AS i_brand#73, null AS i_class#74, null AS i_category#75, avg(qoh#67)#72 AS qoh#76]
(39) ReusedExchange [Reuses operator id: 20]
-Output [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78]
+Output [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82]
(40) HashAggregate [codegen id : 28]
-Input [6]: [i_product_name#73, i_brand#74, i_class#75, i_category#76, sum#77, count#78]
-Keys [4]: [i_product_name#73, i_brand#74, i_class#75, i_category#76]
-Functions [1]: [avg(inv_quantity_on_hand#79)]
-Aggregate Attributes [1]: [avg(inv_quantity_on_hand#79)#17]
-Results [1]: [avg(inv_quantity_on_hand#79)#17 AS qoh#80]
+Input [6]: [i_product_name#77, i_brand#78, i_class#79, i_category#80, sum#81, count#82]
+Keys [4]: [i_product_name#77, i_brand#78, i_class#79, i_category#80]
+Functions [1]: [avg(inv_quantity_on_hand#83)]
+Aggregate Attributes [1]: [avg(inv_quantity_on_hand#83)#17]
+Results [1]: [avg(inv_quantity_on_hand#83)#17 AS qoh#84]
(41) HashAggregate [codegen id : 28]
-Input [1]: [qoh#80]
+Input [1]: [qoh#84]
Keys: []
-Functions [1]: [partial_avg(qoh#80)]
-Aggregate Attributes [2]: [sum#81, count#82]
-Results [2]: [sum#83, count#84]
+Functions [1]: [partial_avg(qoh#84)]
+Aggregate Attributes [2]: [sum#85, count#86]
+Results [2]: [sum#87, count#88]
(42) Exchange
-Input [2]: [sum#83, count#84]
+Input [2]: [sum#87, count#88]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=7]
(43) HashAggregate [codegen id : 29]
-Input [2]: [sum#83, count#84]
+Input [2]: [sum#87, count#88]
Keys: []
-Functions [1]: [avg(qoh#80)]
-Aggregate Attributes [1]: [avg(qoh#80)#85]
-Results [5]: [null AS i_product_name#86, null AS i_brand#87, null AS i_class#88, null AS i_category#89, avg(qoh#80)#85 AS qoh#90]
+Functions [1]: [avg(qoh#84)]
+Aggregate Attributes [1]: [avg(qoh#84)#89]
+Results [5]: [null AS i_product_name#90, null AS i_brand#91, null AS i_class#92, null AS i_category#93, avg(qoh#84)#89 AS qoh#94]
(44) Union
(45) TakeOrderedAndProject
-Input [5]: [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24]
-Arguments: 100, [qoh#24 ASC NULLS FIRST, i_product_name#11 ASC NULLS FIRST, i_brand#8 ASC NULLS FIRST, i_class#9 ASC NULLS FIRST, i_category#10 ASC NULLS FIRST], [i_product_name#11, i_brand#8, i_class#9, i_category#10, qoh#24]
+Input [5]: [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28]
+Arguments: 100, [qoh#28 ASC NULLS FIRST, i_product_name#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_class#26 ASC NULLS FIRST, i_category#27 ASC NULLS FIRST], [i_product_name#24, i_brand#25, i_class#26, i_category#27, qoh#28]
===== Subqueries =====
@@ -291,22 +291,22 @@ BroadcastExchange (50)
(46) Scan parquet spark_catalog.default.date_dim
-Output [2]: [d_date_sk#6, d_month_seq#91]
+Output [2]: [d_date_sk#6, d_month_seq#95]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
ReadSchema: struct
(47) ColumnarToRow [codegen id : 1]
-Input [2]: [d_date_sk#6, d_month_seq#91]
+Input [2]: [d_date_sk#6, d_month_seq#95]
(48) Filter [codegen id : 1]
-Input [2]: [d_date_sk#6, d_month_seq#91]
-Condition : (((isnotnull(d_month_seq#91) AND (d_month_seq#91 >= 1212)) AND (d_month_seq#91 <= 1223)) AND isnotnull(d_date_sk#6))
+Input [2]: [d_date_sk#6, d_month_seq#95]
+Condition : (((isnotnull(d_month_seq#95) AND (d_month_seq#95 >= 1212)) AND (d_month_seq#95 <= 1223)) AND isnotnull(d_date_sk#6))
(49) Project [codegen id : 1]
Output [1]: [d_date_sk#6]
-Input [2]: [d_date_sk#6, d_month_seq#91]
+Input [2]: [d_date_sk#6, d_month_seq#95]
(50) BroadcastExchange
Input [1]: [d_date_sk#6]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
index 22f73cc9b9db5..d747066f5945b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q22a/simplified.txt
@@ -1,7 +1,7 @@
TakeOrderedAndProject [qoh,i_product_name,i_brand,i_class,i_category]
Union
WholeStageCodegen (5)
- HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),qoh,sum,count]
+ HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(qoh),i_product_name,i_brand,i_class,i_category,qoh,sum,count]
HashAggregate [i_product_name,i_brand,i_class,i_category,qoh] [sum,count,sum,count]
HashAggregate [i_product_name,i_brand,i_class,i_category,sum,count] [avg(inv_quantity_on_hand),qoh,sum,count]
InputAdapter
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
index 9c28ff9f351d8..a4c009f8219b4 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/explain.txt
@@ -186,265 +186,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8,
Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
-Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23]
+Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31]
(25) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33]
+Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41]
(26) HashAggregate [codegen id : 16]
-Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33]
-Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31]
-Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22]
-Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36]
+Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41]
+Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39]
+Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22]
+Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44]
(27) HashAggregate [codegen id : 16]
-Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36]
-Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30]
-Functions [1]: [partial_sum(sumsales#36)]
-Aggregate Attributes [2]: [sum#37, isEmpty#38]
-Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
+Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44]
+Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38]
+Functions [1]: [partial_sum(sumsales#44)]
+Aggregate Attributes [2]: [sum#45, isEmpty#46]
+Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
(28) Exchange
-Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
-Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
+Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=5]
(29) HashAggregate [codegen id : 17]
-Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
-Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30]
-Functions [1]: [sum(sumsales#36)]
-Aggregate Attributes [1]: [sum(sumsales#36)#41]
-Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43]
+Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
+Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38]
+Functions [1]: [sum(sumsales#44)]
+Aggregate Attributes [1]: [sum(sumsales#44)#49]
+Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51]
(30) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53]
+Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61]
(31) HashAggregate [codegen id : 25]
-Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53]
-Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51]
-Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22]
-Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56]
+Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61]
+Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59]
+Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22]
+Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64]
(32) HashAggregate [codegen id : 25]
-Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56]
-Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49]
-Functions [1]: [partial_sum(sumsales#56)]
-Aggregate Attributes [2]: [sum#57, isEmpty#58]
-Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
+Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64]
+Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57]
+Functions [1]: [partial_sum(sumsales#64)]
+Aggregate Attributes [2]: [sum#65, isEmpty#66]
+Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
(33) Exchange
-Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
-Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
+Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=6]
(34) HashAggregate [codegen id : 26]
-Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
-Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49]
-Functions [1]: [sum(sumsales#56)]
-Aggregate Attributes [1]: [sum(sumsales#56)#61]
-Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64]
+Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
+Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57]
+Functions [1]: [sum(sumsales#64)]
+Aggregate Attributes [1]: [sum(sumsales#64)#69]
+Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72]
(35) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74]
+Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82]
(36) HashAggregate [codegen id : 34]
-Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74]
-Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72]
-Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22]
-Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77]
+Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82]
+Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80]
+Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22]
+Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85]
(37) HashAggregate [codegen id : 34]
-Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77]
-Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69]
-Functions [1]: [partial_sum(sumsales#77)]
-Aggregate Attributes [2]: [sum#78, isEmpty#79]
-Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
+Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85]
+Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77]
+Functions [1]: [partial_sum(sumsales#85)]
+Aggregate Attributes [2]: [sum#86, isEmpty#87]
+Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
(38) Exchange
-Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
-Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
+Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=7]
(39) HashAggregate [codegen id : 35]
-Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
-Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69]
-Functions [1]: [sum(sumsales#77)]
-Aggregate Attributes [1]: [sum(sumsales#77)#82]
-Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86]
+Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
+Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77]
+Functions [1]: [sum(sumsales#85)]
+Aggregate Attributes [1]: [sum(sumsales#85)#90]
+Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94]
(40) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96]
+Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104]
(41) HashAggregate [codegen id : 43]
-Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96]
-Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94]
-Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22]
-Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99]
+Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104]
+Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102]
+Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22]
+Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107]
(42) HashAggregate [codegen id : 43]
-Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99]
-Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90]
-Functions [1]: [partial_sum(sumsales#99)]
-Aggregate Attributes [2]: [sum#100, isEmpty#101]
-Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
+Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107]
+Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98]
+Functions [1]: [partial_sum(sumsales#107)]
+Aggregate Attributes [2]: [sum#108, isEmpty#109]
+Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
(43) Exchange
-Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
-Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
+Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=8]
(44) HashAggregate [codegen id : 44]
-Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
-Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90]
-Functions [1]: [sum(sumsales#99)]
-Aggregate Attributes [1]: [sum(sumsales#99)#104]
-Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109]
+Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
+Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98]
+Functions [1]: [sum(sumsales#107)]
+Aggregate Attributes [1]: [sum(sumsales#107)#112]
+Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117]
(45) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119]
+Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127]
(46) HashAggregate [codegen id : 52]
-Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119]
-Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117]
-Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22]
-Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122]
+Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127]
+Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125]
+Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22]
+Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130]
(47) HashAggregate [codegen id : 52]
-Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122]
-Keys [3]: [i_category#110, i_class#111, i_brand#112]
-Functions [1]: [partial_sum(sumsales#122)]
-Aggregate Attributes [2]: [sum#123, isEmpty#124]
-Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
+Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130]
+Keys [3]: [i_category#118, i_class#119, i_brand#120]
+Functions [1]: [partial_sum(sumsales#130)]
+Aggregate Attributes [2]: [sum#131, isEmpty#132]
+Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
(48) Exchange
-Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
-Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
+Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=9]
(49) HashAggregate [codegen id : 53]
-Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
-Keys [3]: [i_category#110, i_class#111, i_brand#112]
-Functions [1]: [sum(sumsales#122)]
-Aggregate Attributes [1]: [sum(sumsales#122)#127]
-Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133]
+Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
+Keys [3]: [i_category#118, i_class#119, i_brand#120]
+Functions [1]: [sum(sumsales#130)]
+Aggregate Attributes [1]: [sum(sumsales#130)#135]
+Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141]
(50) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143]
+Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151]
(51) HashAggregate [codegen id : 61]
-Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143]
-Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141]
-Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22]
-Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146]
+Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151]
+Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149]
+Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22]
+Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154]
(52) HashAggregate [codegen id : 61]
-Input [3]: [i_category#134, i_class#135, sumsales#146]
-Keys [2]: [i_category#134, i_class#135]
-Functions [1]: [partial_sum(sumsales#146)]
-Aggregate Attributes [2]: [sum#147, isEmpty#148]
-Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
+Input [3]: [i_category#142, i_class#143, sumsales#154]
+Keys [2]: [i_category#142, i_class#143]
+Functions [1]: [partial_sum(sumsales#154)]
+Aggregate Attributes [2]: [sum#155, isEmpty#156]
+Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
(53) Exchange
-Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
-Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
+Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=10]
(54) HashAggregate [codegen id : 62]
-Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
-Keys [2]: [i_category#134, i_class#135]
-Functions [1]: [sum(sumsales#146)]
-Aggregate Attributes [1]: [sum(sumsales#146)#151]
-Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158]
+Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
+Keys [2]: [i_category#142, i_class#143]
+Functions [1]: [sum(sumsales#154)]
+Aggregate Attributes [1]: [sum(sumsales#154)#159]
+Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166]
(55) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168]
+Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176]
(56) HashAggregate [codegen id : 70]
-Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168]
-Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166]
-Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22]
-Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171]
+Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176]
+Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174]
+Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22]
+Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179]
(57) HashAggregate [codegen id : 70]
-Input [2]: [i_category#159, sumsales#171]
-Keys [1]: [i_category#159]
-Functions [1]: [partial_sum(sumsales#171)]
-Aggregate Attributes [2]: [sum#172, isEmpty#173]
-Results [3]: [i_category#159, sum#174, isEmpty#175]
+Input [2]: [i_category#167, sumsales#179]
+Keys [1]: [i_category#167]
+Functions [1]: [partial_sum(sumsales#179)]
+Aggregate Attributes [2]: [sum#180, isEmpty#181]
+Results [3]: [i_category#167, sum#182, isEmpty#183]
(58) Exchange
-Input [3]: [i_category#159, sum#174, isEmpty#175]
-Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=11]
+Input [3]: [i_category#167, sum#182, isEmpty#183]
+Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=11]
(59) HashAggregate [codegen id : 71]
-Input [3]: [i_category#159, sum#174, isEmpty#175]
-Keys [1]: [i_category#159]
-Functions [1]: [sum(sumsales#171)]
-Aggregate Attributes [1]: [sum(sumsales#171)#176]
-Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184]
+Input [3]: [i_category#167, sum#182, isEmpty#183]
+Keys [1]: [i_category#167]
+Functions [1]: [sum(sumsales#179)]
+Aggregate Attributes [1]: [sum(sumsales#179)#184]
+Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192]
(60) ReusedExchange [Reuses operator id: 23]
-Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194]
+Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202]
(61) HashAggregate [codegen id : 79]
-Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194]
-Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192]
-Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22]
-Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197]
+Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202]
+Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200]
+Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22]
+Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205]
(62) HashAggregate [codegen id : 79]
-Input [1]: [sumsales#197]
+Input [1]: [sumsales#205]
Keys: []
-Functions [1]: [partial_sum(sumsales#197)]
-Aggregate Attributes [2]: [sum#198, isEmpty#199]
-Results [2]: [sum#200, isEmpty#201]
+Functions [1]: [partial_sum(sumsales#205)]
+Aggregate Attributes [2]: [sum#206, isEmpty#207]
+Results [2]: [sum#208, isEmpty#209]
(63) Exchange
-Input [2]: [sum#200, isEmpty#201]
+Input [2]: [sum#208, isEmpty#209]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=12]
(64) HashAggregate [codegen id : 80]
-Input [2]: [sum#200, isEmpty#201]
+Input [2]: [sum#208, isEmpty#209]
Keys: []
-Functions [1]: [sum(sumsales#197)]
-Aggregate Attributes [1]: [sum(sumsales#197)#202]
-Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211]
+Functions [1]: [sum(sumsales#205)]
+Aggregate Attributes [1]: [sum(sumsales#205)#210]
+Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219]
(65) Union
(66) Sort [codegen id : 81]
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0
(67) WindowGroupLimit
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial
(68) Exchange
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=13]
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=13]
(69) Sort [codegen id : 82]
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0
(70) WindowGroupLimit
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final
(71) Window
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST]
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST]
(72) Filter [codegen id : 83]
-Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
-Condition : (rk#212 <= 100)
+Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
+Condition : (rk#220 <= 100)
(73) TakeOrderedAndProject
-Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
-Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
+Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
+Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
===== Subqueries =====
@@ -457,22 +457,22 @@ BroadcastExchange (78)
(74) Scan parquet spark_catalog.default.date_dim
-Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
ReadSchema: struct
(75) ColumnarToRow [codegen id : 1]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
(76) Filter [codegen id : 1]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
-Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7))
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
+Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7))
(77) Project [codegen id : 1]
Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
(78) BroadcastExchange
Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
index 795fa297b9bad..b6a4358c4d43b 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a.sf100/simplified.txt
@@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
InputAdapter
Union
WholeStageCodegen (8)
- HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+ HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
InputAdapter
Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
WholeStageCodegen (7)
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
index 75d526da4ba71..417af4fe924ee 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/explain.txt
@@ -171,265 +171,265 @@ Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8,
Keys [8]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12]
Functions [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))]
Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22]
-Results [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#23]
+Results [9]: [i_category#16 AS i_category#23, i_class#15 AS i_class#24, i_brand#14 AS i_brand#25, i_product_name#17 AS i_product_name#26, d_year#8 AS d_year#27, d_qoy#10 AS d_qoy#28, d_moy#9 AS d_moy#29, s_store_id#12 AS s_store_id#30, cast(sum(coalesce((ss_sales_price#4 * cast(ss_quantity#3 as decimal(10,0))), 0.00))#22 as decimal(38,2)) AS sumsales#31]
(22) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33]
+Output [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41]
(23) HashAggregate [codegen id : 10]
-Input [10]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31, sum#32, isEmpty#33]
-Keys [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, s_store_id#31]
-Functions [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22]
-Results [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum(coalesce((ss_sales_price#34 * cast(ss_quantity#35 as decimal(10,0))), 0.00))#22 AS sumsales#36]
+Input [10]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39, sum#40, isEmpty#41]
+Keys [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, s_store_id#39]
+Functions [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22]
+Results [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum(coalesce((ss_sales_price#42 * cast(ss_quantity#43 as decimal(10,0))), 0.00))#22 AS sumsales#44]
(24) HashAggregate [codegen id : 10]
-Input [8]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sumsales#36]
-Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30]
-Functions [1]: [partial_sum(sumsales#36)]
-Aggregate Attributes [2]: [sum#37, isEmpty#38]
-Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
+Input [8]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sumsales#44]
+Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38]
+Functions [1]: [partial_sum(sumsales#44)]
+Aggregate Attributes [2]: [sum#45, isEmpty#46]
+Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
(25) Exchange
-Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
-Arguments: hashpartitioning(i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, 5), ENSURE_REQUIREMENTS, [plan_id=4]
+Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
+Arguments: hashpartitioning(i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, 5), ENSURE_REQUIREMENTS, [plan_id=4]
(26) HashAggregate [codegen id : 11]
-Input [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, sum#39, isEmpty#40]
-Keys [7]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30]
-Functions [1]: [sum(sumsales#36)]
-Aggregate Attributes [1]: [sum(sumsales#36)#41]
-Results [9]: [i_category#24, i_class#25, i_brand#26, i_product_name#27, d_year#28, d_qoy#29, d_moy#30, null AS s_store_id#42, sum(sumsales#36)#41 AS sumsales#43]
+Input [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, sum#47, isEmpty#48]
+Keys [7]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38]
+Functions [1]: [sum(sumsales#44)]
+Aggregate Attributes [1]: [sum(sumsales#44)#49]
+Results [9]: [i_category#32, i_class#33, i_brand#34, i_product_name#35, d_year#36, d_qoy#37, d_moy#38, null AS s_store_id#50, sum(sumsales#44)#49 AS sumsales#51]
(27) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53]
+Output [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61]
(28) HashAggregate [codegen id : 16]
-Input [10]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51, sum#52, isEmpty#53]
-Keys [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, d_moy#50, s_store_id#51]
-Functions [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22]
-Results [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum(coalesce((ss_sales_price#54 * cast(ss_quantity#55 as decimal(10,0))), 0.00))#22 AS sumsales#56]
+Input [10]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59, sum#60, isEmpty#61]
+Keys [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, d_moy#58, s_store_id#59]
+Functions [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22]
+Results [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum(coalesce((ss_sales_price#62 * cast(ss_quantity#63 as decimal(10,0))), 0.00))#22 AS sumsales#64]
(29) HashAggregate [codegen id : 16]
-Input [7]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sumsales#56]
-Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49]
-Functions [1]: [partial_sum(sumsales#56)]
-Aggregate Attributes [2]: [sum#57, isEmpty#58]
-Results [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
+Input [7]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sumsales#64]
+Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57]
+Functions [1]: [partial_sum(sumsales#64)]
+Aggregate Attributes [2]: [sum#65, isEmpty#66]
+Results [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
(30) Exchange
-Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
-Arguments: hashpartitioning(i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, 5), ENSURE_REQUIREMENTS, [plan_id=5]
+Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
+Arguments: hashpartitioning(i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, 5), ENSURE_REQUIREMENTS, [plan_id=5]
(31) HashAggregate [codegen id : 17]
-Input [8]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, sum#59, isEmpty#60]
-Keys [6]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49]
-Functions [1]: [sum(sumsales#56)]
-Aggregate Attributes [1]: [sum(sumsales#56)#61]
-Results [9]: [i_category#44, i_class#45, i_brand#46, i_product_name#47, d_year#48, d_qoy#49, null AS d_moy#62, null AS s_store_id#63, sum(sumsales#56)#61 AS sumsales#64]
+Input [8]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, sum#67, isEmpty#68]
+Keys [6]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57]
+Functions [1]: [sum(sumsales#64)]
+Aggregate Attributes [1]: [sum(sumsales#64)#69]
+Results [9]: [i_category#52, i_class#53, i_brand#54, i_product_name#55, d_year#56, d_qoy#57, null AS d_moy#70, null AS s_store_id#71, sum(sumsales#64)#69 AS sumsales#72]
(32) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74]
+Output [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82]
(33) HashAggregate [codegen id : 22]
-Input [10]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72, sum#73, isEmpty#74]
-Keys [8]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, d_qoy#70, d_moy#71, s_store_id#72]
-Functions [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22]
-Results [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum(coalesce((ss_sales_price#75 * cast(ss_quantity#76 as decimal(10,0))), 0.00))#22 AS sumsales#77]
+Input [10]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80, sum#81, isEmpty#82]
+Keys [8]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, d_qoy#78, d_moy#79, s_store_id#80]
+Functions [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22]
+Results [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum(coalesce((ss_sales_price#83 * cast(ss_quantity#84 as decimal(10,0))), 0.00))#22 AS sumsales#85]
(34) HashAggregate [codegen id : 22]
-Input [6]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sumsales#77]
-Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69]
-Functions [1]: [partial_sum(sumsales#77)]
-Aggregate Attributes [2]: [sum#78, isEmpty#79]
-Results [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
+Input [6]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sumsales#85]
+Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77]
+Functions [1]: [partial_sum(sumsales#85)]
+Aggregate Attributes [2]: [sum#86, isEmpty#87]
+Results [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
(35) Exchange
-Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
-Arguments: hashpartitioning(i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, 5), ENSURE_REQUIREMENTS, [plan_id=6]
+Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
+Arguments: hashpartitioning(i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, 5), ENSURE_REQUIREMENTS, [plan_id=6]
(36) HashAggregate [codegen id : 23]
-Input [7]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, sum#80, isEmpty#81]
-Keys [5]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69]
-Functions [1]: [sum(sumsales#77)]
-Aggregate Attributes [1]: [sum(sumsales#77)#82]
-Results [9]: [i_category#65, i_class#66, i_brand#67, i_product_name#68, d_year#69, null AS d_qoy#83, null AS d_moy#84, null AS s_store_id#85, sum(sumsales#77)#82 AS sumsales#86]
+Input [7]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, sum#88, isEmpty#89]
+Keys [5]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77]
+Functions [1]: [sum(sumsales#85)]
+Aggregate Attributes [1]: [sum(sumsales#85)#90]
+Results [9]: [i_category#73, i_class#74, i_brand#75, i_product_name#76, d_year#77, null AS d_qoy#91, null AS d_moy#92, null AS s_store_id#93, sum(sumsales#85)#90 AS sumsales#94]
(37) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96]
+Output [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104]
(38) HashAggregate [codegen id : 28]
-Input [10]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94, sum#95, isEmpty#96]
-Keys [8]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, d_year#91, d_qoy#92, d_moy#93, s_store_id#94]
-Functions [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22]
-Results [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum(coalesce((ss_sales_price#97 * cast(ss_quantity#98 as decimal(10,0))), 0.00))#22 AS sumsales#99]
+Input [10]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102, sum#103, isEmpty#104]
+Keys [8]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, d_year#99, d_qoy#100, d_moy#101, s_store_id#102]
+Functions [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22]
+Results [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum(coalesce((ss_sales_price#105 * cast(ss_quantity#106 as decimal(10,0))), 0.00))#22 AS sumsales#107]
(39) HashAggregate [codegen id : 28]
-Input [5]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sumsales#99]
-Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90]
-Functions [1]: [partial_sum(sumsales#99)]
-Aggregate Attributes [2]: [sum#100, isEmpty#101]
-Results [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
+Input [5]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sumsales#107]
+Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98]
+Functions [1]: [partial_sum(sumsales#107)]
+Aggregate Attributes [2]: [sum#108, isEmpty#109]
+Results [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
(40) Exchange
-Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
-Arguments: hashpartitioning(i_category#87, i_class#88, i_brand#89, i_product_name#90, 5), ENSURE_REQUIREMENTS, [plan_id=7]
+Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
+Arguments: hashpartitioning(i_category#95, i_class#96, i_brand#97, i_product_name#98, 5), ENSURE_REQUIREMENTS, [plan_id=7]
(41) HashAggregate [codegen id : 29]
-Input [6]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, sum#102, isEmpty#103]
-Keys [4]: [i_category#87, i_class#88, i_brand#89, i_product_name#90]
-Functions [1]: [sum(sumsales#99)]
-Aggregate Attributes [1]: [sum(sumsales#99)#104]
-Results [9]: [i_category#87, i_class#88, i_brand#89, i_product_name#90, null AS d_year#105, null AS d_qoy#106, null AS d_moy#107, null AS s_store_id#108, sum(sumsales#99)#104 AS sumsales#109]
+Input [6]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, sum#110, isEmpty#111]
+Keys [4]: [i_category#95, i_class#96, i_brand#97, i_product_name#98]
+Functions [1]: [sum(sumsales#107)]
+Aggregate Attributes [1]: [sum(sumsales#107)#112]
+Results [9]: [i_category#95, i_class#96, i_brand#97, i_product_name#98, null AS d_year#113, null AS d_qoy#114, null AS d_moy#115, null AS s_store_id#116, sum(sumsales#107)#112 AS sumsales#117]
(42) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119]
+Output [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127]
(43) HashAggregate [codegen id : 34]
-Input [10]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117, sum#118, isEmpty#119]
-Keys [8]: [i_category#110, i_class#111, i_brand#112, i_product_name#113, d_year#114, d_qoy#115, d_moy#116, s_store_id#117]
-Functions [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22]
-Results [4]: [i_category#110, i_class#111, i_brand#112, sum(coalesce((ss_sales_price#120 * cast(ss_quantity#121 as decimal(10,0))), 0.00))#22 AS sumsales#122]
+Input [10]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125, sum#126, isEmpty#127]
+Keys [8]: [i_category#118, i_class#119, i_brand#120, i_product_name#121, d_year#122, d_qoy#123, d_moy#124, s_store_id#125]
+Functions [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22]
+Results [4]: [i_category#118, i_class#119, i_brand#120, sum(coalesce((ss_sales_price#128 * cast(ss_quantity#129 as decimal(10,0))), 0.00))#22 AS sumsales#130]
(44) HashAggregate [codegen id : 34]
-Input [4]: [i_category#110, i_class#111, i_brand#112, sumsales#122]
-Keys [3]: [i_category#110, i_class#111, i_brand#112]
-Functions [1]: [partial_sum(sumsales#122)]
-Aggregate Attributes [2]: [sum#123, isEmpty#124]
-Results [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
+Input [4]: [i_category#118, i_class#119, i_brand#120, sumsales#130]
+Keys [3]: [i_category#118, i_class#119, i_brand#120]
+Functions [1]: [partial_sum(sumsales#130)]
+Aggregate Attributes [2]: [sum#131, isEmpty#132]
+Results [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
(45) Exchange
-Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
-Arguments: hashpartitioning(i_category#110, i_class#111, i_brand#112, 5), ENSURE_REQUIREMENTS, [plan_id=8]
+Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
+Arguments: hashpartitioning(i_category#118, i_class#119, i_brand#120, 5), ENSURE_REQUIREMENTS, [plan_id=8]
(46) HashAggregate [codegen id : 35]
-Input [5]: [i_category#110, i_class#111, i_brand#112, sum#125, isEmpty#126]
-Keys [3]: [i_category#110, i_class#111, i_brand#112]
-Functions [1]: [sum(sumsales#122)]
-Aggregate Attributes [1]: [sum(sumsales#122)#127]
-Results [9]: [i_category#110, i_class#111, i_brand#112, null AS i_product_name#128, null AS d_year#129, null AS d_qoy#130, null AS d_moy#131, null AS s_store_id#132, sum(sumsales#122)#127 AS sumsales#133]
+Input [5]: [i_category#118, i_class#119, i_brand#120, sum#133, isEmpty#134]
+Keys [3]: [i_category#118, i_class#119, i_brand#120]
+Functions [1]: [sum(sumsales#130)]
+Aggregate Attributes [1]: [sum(sumsales#130)#135]
+Results [9]: [i_category#118, i_class#119, i_brand#120, null AS i_product_name#136, null AS d_year#137, null AS d_qoy#138, null AS d_moy#139, null AS s_store_id#140, sum(sumsales#130)#135 AS sumsales#141]
(47) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143]
+Output [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151]
(48) HashAggregate [codegen id : 40]
-Input [10]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141, sum#142, isEmpty#143]
-Keys [8]: [i_category#134, i_class#135, i_brand#136, i_product_name#137, d_year#138, d_qoy#139, d_moy#140, s_store_id#141]
-Functions [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22]
-Results [3]: [i_category#134, i_class#135, sum(coalesce((ss_sales_price#144 * cast(ss_quantity#145 as decimal(10,0))), 0.00))#22 AS sumsales#146]
+Input [10]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149, sum#150, isEmpty#151]
+Keys [8]: [i_category#142, i_class#143, i_brand#144, i_product_name#145, d_year#146, d_qoy#147, d_moy#148, s_store_id#149]
+Functions [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22]
+Results [3]: [i_category#142, i_class#143, sum(coalesce((ss_sales_price#152 * cast(ss_quantity#153 as decimal(10,0))), 0.00))#22 AS sumsales#154]
(49) HashAggregate [codegen id : 40]
-Input [3]: [i_category#134, i_class#135, sumsales#146]
-Keys [2]: [i_category#134, i_class#135]
-Functions [1]: [partial_sum(sumsales#146)]
-Aggregate Attributes [2]: [sum#147, isEmpty#148]
-Results [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
+Input [3]: [i_category#142, i_class#143, sumsales#154]
+Keys [2]: [i_category#142, i_class#143]
+Functions [1]: [partial_sum(sumsales#154)]
+Aggregate Attributes [2]: [sum#155, isEmpty#156]
+Results [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
(50) Exchange
-Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
-Arguments: hashpartitioning(i_category#134, i_class#135, 5), ENSURE_REQUIREMENTS, [plan_id=9]
+Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
+Arguments: hashpartitioning(i_category#142, i_class#143, 5), ENSURE_REQUIREMENTS, [plan_id=9]
(51) HashAggregate [codegen id : 41]
-Input [4]: [i_category#134, i_class#135, sum#149, isEmpty#150]
-Keys [2]: [i_category#134, i_class#135]
-Functions [1]: [sum(sumsales#146)]
-Aggregate Attributes [1]: [sum(sumsales#146)#151]
-Results [9]: [i_category#134, i_class#135, null AS i_brand#152, null AS i_product_name#153, null AS d_year#154, null AS d_qoy#155, null AS d_moy#156, null AS s_store_id#157, sum(sumsales#146)#151 AS sumsales#158]
+Input [4]: [i_category#142, i_class#143, sum#157, isEmpty#158]
+Keys [2]: [i_category#142, i_class#143]
+Functions [1]: [sum(sumsales#154)]
+Aggregate Attributes [1]: [sum(sumsales#154)#159]
+Results [9]: [i_category#142, i_class#143, null AS i_brand#160, null AS i_product_name#161, null AS d_year#162, null AS d_qoy#163, null AS d_moy#164, null AS s_store_id#165, sum(sumsales#154)#159 AS sumsales#166]
(52) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168]
+Output [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176]
(53) HashAggregate [codegen id : 46]
-Input [10]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166, sum#167, isEmpty#168]
-Keys [8]: [i_category#159, i_class#160, i_brand#161, i_product_name#162, d_year#163, d_qoy#164, d_moy#165, s_store_id#166]
-Functions [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22]
-Results [2]: [i_category#159, sum(coalesce((ss_sales_price#169 * cast(ss_quantity#170 as decimal(10,0))), 0.00))#22 AS sumsales#171]
+Input [10]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174, sum#175, isEmpty#176]
+Keys [8]: [i_category#167, i_class#168, i_brand#169, i_product_name#170, d_year#171, d_qoy#172, d_moy#173, s_store_id#174]
+Functions [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22]
+Results [2]: [i_category#167, sum(coalesce((ss_sales_price#177 * cast(ss_quantity#178 as decimal(10,0))), 0.00))#22 AS sumsales#179]
(54) HashAggregate [codegen id : 46]
-Input [2]: [i_category#159, sumsales#171]
-Keys [1]: [i_category#159]
-Functions [1]: [partial_sum(sumsales#171)]
-Aggregate Attributes [2]: [sum#172, isEmpty#173]
-Results [3]: [i_category#159, sum#174, isEmpty#175]
+Input [2]: [i_category#167, sumsales#179]
+Keys [1]: [i_category#167]
+Functions [1]: [partial_sum(sumsales#179)]
+Aggregate Attributes [2]: [sum#180, isEmpty#181]
+Results [3]: [i_category#167, sum#182, isEmpty#183]
(55) Exchange
-Input [3]: [i_category#159, sum#174, isEmpty#175]
-Arguments: hashpartitioning(i_category#159, 5), ENSURE_REQUIREMENTS, [plan_id=10]
+Input [3]: [i_category#167, sum#182, isEmpty#183]
+Arguments: hashpartitioning(i_category#167, 5), ENSURE_REQUIREMENTS, [plan_id=10]
(56) HashAggregate [codegen id : 47]
-Input [3]: [i_category#159, sum#174, isEmpty#175]
-Keys [1]: [i_category#159]
-Functions [1]: [sum(sumsales#171)]
-Aggregate Attributes [1]: [sum(sumsales#171)#176]
-Results [9]: [i_category#159, null AS i_class#177, null AS i_brand#178, null AS i_product_name#179, null AS d_year#180, null AS d_qoy#181, null AS d_moy#182, null AS s_store_id#183, sum(sumsales#171)#176 AS sumsales#184]
+Input [3]: [i_category#167, sum#182, isEmpty#183]
+Keys [1]: [i_category#167]
+Functions [1]: [sum(sumsales#179)]
+Aggregate Attributes [1]: [sum(sumsales#179)#184]
+Results [9]: [i_category#167, null AS i_class#185, null AS i_brand#186, null AS i_product_name#187, null AS d_year#188, null AS d_qoy#189, null AS d_moy#190, null AS s_store_id#191, sum(sumsales#179)#184 AS sumsales#192]
(57) ReusedExchange [Reuses operator id: 20]
-Output [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194]
+Output [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202]
(58) HashAggregate [codegen id : 52]
-Input [10]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192, sum#193, isEmpty#194]
-Keys [8]: [i_category#185, i_class#186, i_brand#187, i_product_name#188, d_year#189, d_qoy#190, d_moy#191, s_store_id#192]
-Functions [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))]
-Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22]
-Results [1]: [sum(coalesce((ss_sales_price#195 * cast(ss_quantity#196 as decimal(10,0))), 0.00))#22 AS sumsales#197]
+Input [10]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200, sum#201, isEmpty#202]
+Keys [8]: [i_category#193, i_class#194, i_brand#195, i_product_name#196, d_year#197, d_qoy#198, d_moy#199, s_store_id#200]
+Functions [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))]
+Aggregate Attributes [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22]
+Results [1]: [sum(coalesce((ss_sales_price#203 * cast(ss_quantity#204 as decimal(10,0))), 0.00))#22 AS sumsales#205]
(59) HashAggregate [codegen id : 52]
-Input [1]: [sumsales#197]
+Input [1]: [sumsales#205]
Keys: []
-Functions [1]: [partial_sum(sumsales#197)]
-Aggregate Attributes [2]: [sum#198, isEmpty#199]
-Results [2]: [sum#200, isEmpty#201]
+Functions [1]: [partial_sum(sumsales#205)]
+Aggregate Attributes [2]: [sum#206, isEmpty#207]
+Results [2]: [sum#208, isEmpty#209]
(60) Exchange
-Input [2]: [sum#200, isEmpty#201]
+Input [2]: [sum#208, isEmpty#209]
Arguments: SinglePartition, ENSURE_REQUIREMENTS, [plan_id=11]
(61) HashAggregate [codegen id : 53]
-Input [2]: [sum#200, isEmpty#201]
+Input [2]: [sum#208, isEmpty#209]
Keys: []
-Functions [1]: [sum(sumsales#197)]
-Aggregate Attributes [1]: [sum(sumsales#197)#202]
-Results [9]: [null AS i_category#203, null AS i_class#204, null AS i_brand#205, null AS i_product_name#206, null AS d_year#207, null AS d_qoy#208, null AS d_moy#209, null AS s_store_id#210, sum(sumsales#197)#202 AS sumsales#211]
+Functions [1]: [sum(sumsales#205)]
+Aggregate Attributes [1]: [sum(sumsales#205)#210]
+Results [9]: [null AS i_category#211, null AS i_class#212, null AS i_brand#213, null AS i_product_name#214, null AS d_year#215, null AS d_qoy#216, null AS d_moy#217, null AS s_store_id#218, sum(sumsales#205)#210 AS sumsales#219]
(62) Union
(63) Sort [codegen id : 54]
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0
(64) WindowGroupLimit
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Partial
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Partial
(65) Exchange
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: hashpartitioning(i_category#16, 5), ENSURE_REQUIREMENTS, [plan_id=12]
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: hashpartitioning(i_category#23, 5), ENSURE_REQUIREMENTS, [plan_id=12]
(66) Sort [codegen id : 55]
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16 ASC NULLS FIRST, sumsales#23 DESC NULLS LAST], false, 0
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23 ASC NULLS FIRST, sumsales#31 DESC NULLS LAST], false, 0
(67) WindowGroupLimit
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [i_category#16], [sumsales#23 DESC NULLS LAST], rank(sumsales#23), 100, Final
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [i_category#23], [sumsales#31 DESC NULLS LAST], rank(sumsales#31), 100, Final
(68) Window
-Input [9]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23]
-Arguments: [rank(sumsales#23) windowspecdefinition(i_category#16, sumsales#23 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#212], [i_category#16], [sumsales#23 DESC NULLS LAST]
+Input [9]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31]
+Arguments: [rank(sumsales#31) windowspecdefinition(i_category#23, sumsales#31 DESC NULLS LAST, specifiedwindowframe(RowFrame, unboundedpreceding$(), currentrow$())) AS rk#220], [i_category#23], [sumsales#31 DESC NULLS LAST]
(69) Filter [codegen id : 56]
-Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
-Condition : (rk#212 <= 100)
+Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
+Condition : (rk#220 <= 100)
(70) TakeOrderedAndProject
-Input [10]: [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
-Arguments: 100, [i_category#16 ASC NULLS FIRST, i_class#15 ASC NULLS FIRST, i_brand#14 ASC NULLS FIRST, i_product_name#17 ASC NULLS FIRST, d_year#8 ASC NULLS FIRST, d_qoy#10 ASC NULLS FIRST, d_moy#9 ASC NULLS FIRST, s_store_id#12 ASC NULLS FIRST, sumsales#23 ASC NULLS FIRST, rk#212 ASC NULLS FIRST], [i_category#16, i_class#15, i_brand#14, i_product_name#17, d_year#8, d_qoy#10, d_moy#9, s_store_id#12, sumsales#23, rk#212]
+Input [10]: [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
+Arguments: 100, [i_category#23 ASC NULLS FIRST, i_class#24 ASC NULLS FIRST, i_brand#25 ASC NULLS FIRST, i_product_name#26 ASC NULLS FIRST, d_year#27 ASC NULLS FIRST, d_qoy#28 ASC NULLS FIRST, d_moy#29 ASC NULLS FIRST, s_store_id#30 ASC NULLS FIRST, sumsales#31 ASC NULLS FIRST, rk#220 ASC NULLS FIRST], [i_category#23, i_class#24, i_brand#25, i_product_name#26, d_year#27, d_qoy#28, d_moy#29, s_store_id#30, sumsales#31, rk#220]
===== Subqueries =====
@@ -442,22 +442,22 @@ BroadcastExchange (75)
(71) Scan parquet spark_catalog.default.date_dim
-Output [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Output [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
Batched: true
Location [not included in comparison]/{warehouse_dir}/date_dim]
PushedFilters: [IsNotNull(d_month_seq), GreaterThanOrEqual(d_month_seq,1212), LessThanOrEqual(d_month_seq,1223), IsNotNull(d_date_sk)]
ReadSchema: struct
(72) ColumnarToRow [codegen id : 1]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
(73) Filter [codegen id : 1]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
-Condition : (((isnotnull(d_month_seq#213) AND (d_month_seq#213 >= 1212)) AND (d_month_seq#213 <= 1223)) AND isnotnull(d_date_sk#7))
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
+Condition : (((isnotnull(d_month_seq#221) AND (d_month_seq#221 >= 1212)) AND (d_month_seq#221 <= 1223)) AND isnotnull(d_date_sk#7))
(74) Project [codegen id : 1]
Output [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
-Input [5]: [d_date_sk#7, d_month_seq#213, d_year#8, d_moy#9, d_qoy#10]
+Input [5]: [d_date_sk#7, d_month_seq#221, d_year#8, d_moy#9, d_qoy#10]
(75) BroadcastExchange
Input [4]: [d_date_sk#7, d_year#8, d_moy#9, d_qoy#10]
diff --git a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
index 89393f265a49f..5a43dced056bd 100644
--- a/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
+++ b/sql/core/src/test/resources/tpcds-plan-stability/approved-plans-v2_7/q67a/simplified.txt
@@ -14,7 +14,7 @@ TakeOrderedAndProject [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_
InputAdapter
Union
WholeStageCodegen (5)
- HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),sumsales,sum,isEmpty]
+ HashAggregate [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sum,isEmpty] [sum(coalesce((ss_sales_price * cast(ss_quantity as decimal(10,0))), 0.00)),i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id,sumsales,sum,isEmpty]
InputAdapter
Exchange [i_category,i_class,i_brand,i_product_name,d_year,d_qoy,d_moy,s_store_id] #2
WholeStageCodegen (4)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala
index 879c0c480943d..8600ec4f8787f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationExpressionWalkerSuite.scala
@@ -741,7 +741,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi
assert(resultUTF8.collect() === resultUTF8Lcase.collect())
}
} catch {
- case e: SparkRuntimeException => assert(e.getErrorClass == "USER_RAISED_EXCEPTION")
+ case e: SparkRuntimeException => assert(e.getCondition == "USER_RAISED_EXCEPTION")
case other: Throwable => throw other
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
index 4c3cd93873bd4..d568cd77050fd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLExpressionsSuite.scala
@@ -49,9 +49,13 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
Md5TestCase("Spark", "UTF8_BINARY", "8cde774d6f7333752ed72cacddb05126"),
+ Md5TestCase("Spark", "UTF8_BINARY_RTRIM", "8cde774d6f7333752ed72cacddb05126"),
Md5TestCase("Spark", "UTF8_LCASE", "8cde774d6f7333752ed72cacddb05126"),
+ Md5TestCase("Spark", "UTF8_LCASE_RTRIM", "8cde774d6f7333752ed72cacddb05126"),
Md5TestCase("SQL", "UNICODE", "9778840a0100cb30c982876741b0b5a2"),
- Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2")
+ Md5TestCase("SQL", "UNICODE_RTRIM", "9778840a0100cb30c982876741b0b5a2"),
+ Md5TestCase("SQL", "UNICODE_CI", "9778840a0100cb30c982876741b0b5a2"),
+ Md5TestCase("SQL", "UNICODE_CI_RTRIM", "9778840a0100cb30c982876741b0b5a2")
)
// Supported collations
@@ -81,11 +85,19 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
Sha2TestCase("Spark", "UTF8_BINARY", 256,
"529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
+ Sha2TestCase("Spark", "UTF8_BINARY_RTRIM", 256,
+ "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
Sha2TestCase("Spark", "UTF8_LCASE", 256,
"529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
+ Sha2TestCase("Spark", "UTF8_LCASE_RTRIM", 256,
+ "529bc3b07127ecb7e53a4dcf1991d9152c24537d919178022b2c42657f79a26b"),
Sha2TestCase("SQL", "UNICODE", 256,
"a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"),
+ Sha2TestCase("SQL", "UNICODE_RTRIM", 256,
+ "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"),
Sha2TestCase("SQL", "UNICODE_CI", 256,
+ "a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35"),
+ Sha2TestCase("SQL", "UNICODE_CI_RTRIM", 256,
"a7056a455639d1c7deec82ee787db24a0c1878e2792b4597709f0facf7cc7b35")
)
@@ -114,9 +126,13 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
Sha1TestCase("Spark", "UTF8_BINARY", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
+ Sha1TestCase("Spark", "UTF8_BINARY_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
Sha1TestCase("Spark", "UTF8_LCASE", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
+ Sha1TestCase("Spark", "UTF8_LCASE_RTRIM", "85f5955f4b27a9a4c2aab6ffe5d7189fc298b92c"),
Sha1TestCase("SQL", "UNICODE", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"),
- Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d")
+ Sha1TestCase("SQL", "UNICODE_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"),
+ Sha1TestCase("SQL", "UNICODE_CI", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d"),
+ Sha1TestCase("SQL", "UNICODE_CI_RTRIM", "2064cb643caa8d9e1de12eea7f3e143ca9f8680d")
)
// Supported collations
@@ -144,9 +160,13 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
Crc321TestCase("Spark", "UTF8_BINARY", 1557323817),
+ Crc321TestCase("Spark", "UTF8_BINARY_RTRIM", 1557323817),
Crc321TestCase("Spark", "UTF8_LCASE", 1557323817),
+ Crc321TestCase("Spark", "UTF8_LCASE_RTRIM", 1557323817),
Crc321TestCase("SQL", "UNICODE", 1299261525),
- Crc321TestCase("SQL", "UNICODE_CI", 1299261525)
+ Crc321TestCase("SQL", "UNICODE_RTRIM", 1299261525),
+ Crc321TestCase("SQL", "UNICODE_CI", 1299261525),
+ Crc321TestCase("SQL", "UNICODE_CI_RTRIM", 1299261525)
)
// Supported collations
@@ -172,9 +192,13 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
Murmur3HashTestCase("Spark", "UTF8_BINARY", 228093765),
+ Murmur3HashTestCase("Spark ", "UTF8_BINARY_RTRIM", 1779328737),
Murmur3HashTestCase("Spark", "UTF8_LCASE", -1928694360),
+ Murmur3HashTestCase("Spark ", "UTF8_LCASE_RTRIM", -1928694360),
Murmur3HashTestCase("SQL", "UNICODE", -1923567940),
- Murmur3HashTestCase("SQL", "UNICODE_CI", 1029527950)
+ Murmur3HashTestCase("SQL ", "UNICODE_RTRIM", -1923567940),
+ Murmur3HashTestCase("SQL", "UNICODE_CI", 1029527950),
+ Murmur3HashTestCase("SQL ", "UNICODE_CI_RTRIM", 1029527950)
)
// Supported collations
@@ -200,9 +224,13 @@ class CollationSQLExpressionsSuite
val testCases = Seq(
XxHash64TestCase("Spark", "UTF8_BINARY", -4294468057691064905L),
+ XxHash64TestCase("Spark ", "UTF8_BINARY_RTRIM", 6480371823304753502L),
XxHash64TestCase("Spark", "UTF8_LCASE", -3142112654825786434L),
+ XxHash64TestCase("Spark ", "UTF8_LCASE_RTRIM", -3142112654825786434L),
XxHash64TestCase("SQL", "UNICODE", 5964849564945649886L),
- XxHash64TestCase("SQL", "UNICODE_CI", 3732497619779520590L)
+ XxHash64TestCase("SQL ", "UNICODE_RTRIM", 5964849564945649886L),
+ XxHash64TestCase("SQL", "UNICODE_CI", 3732497619779520590L),
+ XxHash64TestCase("SQL ", "UNICODE_CI_RTRIM", 3732497619779520590L)
)
// Supported collations
@@ -2791,16 +2819,24 @@ class CollationSQLExpressionsSuite
}
}
- test("collect_set supports collation") {
+ test("collect_set does not support collation") {
val collation = "UNICODE"
val query = s"SELECT collect_set(col) FROM VALUES ('a'), ('b'), ('a') AS tab(col);"
withSQLConf(SqlApiConf.DEFAULT_COLLATION -> collation) {
- val result = sql(query).collect().head.getSeq[String](0).toSet
- val expected = Set("a", "b")
- assert(result == expected)
- // check result row data type
- val dataType = ArrayType(StringType(collation), false)
- assert(sql(query).schema.head.dataType == dataType)
+ checkError(
+ exception = intercept[AnalysisException] {
+ sql(query)
+ },
+ condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE",
+ sqlState = Some("42K09"),
+ parameters = Map(
+ "functionName" -> "`collect_set`",
+ "dataType" -> "\"MAP\" or \"COLLATED STRING\"",
+ "sqlExpr" -> "\"collect_set(col)\""),
+ context = ExpectedContext(
+ fragment = "collect_set(col)",
+ start = 7,
+ stop = 22))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala
index ef01f71c68bf9..b6da0b169f050 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSuite.scala
@@ -101,8 +101,12 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
test("collate function syntax") {
assert(sql(s"select collate('aaa', 'utf8_binary')").schema(0).dataType ==
StringType("UTF8_BINARY"))
+ assert(sql(s"select collate('aaa', 'utf8_binary_rtrim')").schema(0).dataType ==
+ StringType("UTF8_BINARY_RTRIM"))
assert(sql(s"select collate('aaa', 'utf8_lcase')").schema(0).dataType ==
StringType("UTF8_LCASE"))
+ assert(sql(s"select collate('aaa', 'utf8_lcase_rtrim')").schema(0).dataType ==
+ StringType("UTF8_LCASE_RTRIM"))
}
test("collate function syntax with default collation set") {
@@ -260,14 +264,23 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
Seq(
("utf8_binary", "aaa", "AAA", false),
("utf8_binary", "aaa", "aaa", true),
+ ("utf8_binary_rtrim", "aaa", "AAA", false),
+ ("utf8_binary_rtrim", "aaa", "aaa ", true),
("utf8_lcase", "aaa", "aaa", true),
("utf8_lcase", "aaa", "AAA", true),
("utf8_lcase", "aaa", "bbb", false),
+ ("utf8_lcase_rtrim", "aaa", "AAA ", true),
+ ("utf8_lcase_rtrim", "aaa", "bbb", false),
("unicode", "aaa", "aaa", true),
("unicode", "aaa", "AAA", false),
+ ("unicode_rtrim", "aaa ", "aaa ", true),
+ ("unicode_rtrim", "aaa", "AAA", false),
("unicode_CI", "aaa", "aaa", true),
("unicode_CI", "aaa", "AAA", true),
- ("unicode_CI", "aaa", "bbb", false)
+ ("unicode_CI", "aaa", "bbb", false),
+ ("unicode_CI_rtrim", "aaa", "aaa", true),
+ ("unicode_CI_rtrim", "aaa ", "AAA ", true),
+ ("unicode_CI_rtrim", "aaa", "bbb", false)
).foreach {
case (collationName, left, right, expected) =>
checkAnswer(
@@ -284,15 +297,19 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
("utf8_binary", "AAA", "aaa", true),
("utf8_binary", "aaa", "aaa", false),
("utf8_binary", "aaa", "BBB", false),
+ ("utf8_binary_rtrim", "aaa ", "aaa ", false),
("utf8_lcase", "aaa", "aaa", false),
("utf8_lcase", "AAA", "aaa", false),
("utf8_lcase", "aaa", "bbb", true),
+ ("utf8_lcase_rtrim", "AAA ", "aaa", false),
("unicode", "aaa", "aaa", false),
("unicode", "aaa", "AAA", true),
("unicode", "aaa", "BBB", true),
+ ("unicode_rtrim", "aaa ", "aaa", false),
("unicode_CI", "aaa", "aaa", false),
("unicode_CI", "aaa", "AAA", false),
- ("unicode_CI", "aaa", "bbb", true)
+ ("unicode_CI", "aaa", "bbb", true),
+ ("unicode_CI_rtrim", "aaa ", "aaa", false)
).foreach {
case (collationName, left, right, expected) =>
checkAnswer(
@@ -355,18 +372,22 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
test("aggregates count respects collation") {
Seq(
+ ("utf8_binary_rtrim", Seq("aaa", "aaa "), Seq(Row(2, "aaa"))),
("utf8_binary", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))),
("utf8_binary", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))),
("utf8_binary", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))),
("utf8_lcase", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))),
("utf8_lcase", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))),
("utf8_lcase", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))),
+ ("utf8_lcase_rtrim", Seq("aaa", "AAA "), Seq(Row(2, "aaa"))),
("unicode", Seq("AAA", "aaa"), Seq(Row(1, "AAA"), Row(1, "aaa"))),
("unicode", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))),
("unicode", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))),
+ ("unicode_rtrim", Seq("aaa", "aaa "), Seq(Row(2, "aaa"))),
("unicode_CI", Seq("aaa", "aaa"), Seq(Row(2, "aaa"))),
("unicode_CI", Seq("AAA", "aaa"), Seq(Row(2, "AAA"))),
- ("unicode_CI", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb")))
+ ("unicode_CI", Seq("aaa", "bbb"), Seq(Row(1, "aaa"), Row(1, "bbb"))),
+ ("unicode_CI_rtrim", Seq("aaa", "AAA "), Seq(Row(2, "aaa")))
).foreach {
case (collationName: String, input: Seq[String], expected: Seq[Row]) =>
checkAnswer(sql(
@@ -1101,12 +1122,218 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
}
}
+ test("Check order by on table with collated string column") {
+ val tableName = "t"
+ Seq(
+ // (collationName, data, expResult)
+ (
+ "", // non-collated
+ Seq((5, "bbb"), (3, "a"), (1, "A"), (4, "aaaa"), (6, "cc"), (2, "BbB")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UTF8_BINARY",
+ Seq((5, "bbb"), (3, "a"), (1, "A"), (4, "aaaa"), (6, "cc"), (2, "BbB")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UTF8_LCASE",
+ Seq((2, "bbb"), (1, "a"), (1, "A"), (1, "aaaa"), (3, "cc"), (2, "BbB")),
+ Seq(1, 1, 1, 2, 2, 3)
+ ),
+ (
+ "UNICODE",
+ Seq((4, "bbb"), (1, "a"), (2, "A"), (3, "aaaa"), (6, "cc"), (5, "BbB")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UNICODE_CI",
+ Seq((2, "bbb"), (1, "a"), (1, "A"), (1, "aaaa"), (3, "cc"), (2, "BbB")),
+ Seq(1, 1, 1, 2, 2, 3)
+ )
+ ).foreach {
+ case (collationName, data, expResult) =>
+ val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName
+ withTable(tableName) {
+ sql(s"create table $tableName (c1 integer, c2 string $collationSetup)")
+ data.foreach {
+ case (c1, c2) =>
+ sql(s"insert into $tableName values ($c1, '$c2')")
+ }
+ checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_)))
+ }
+ }
+ }
+
+ test("Check order by on StructType") {
+ Seq(
+ // (collationName, data, expResult)
+ (
+ "", // non-collated
+ Seq((5, "b", "A"), (3, "aa", "A"), (6, "b", "B"), (2, "A", "c"), (1, "A", "D"),
+ (4, "aa", "B")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UTF8_BINARY",
+ Seq((5, "b", "A"), (3, "aa", "A"), (6, "b", "B"), (2, "A", "c"), (1, "A", "D"),
+ (4, "aa", "B")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UTF8_LCASE",
+ Seq((3, "A", "C"), (2, "A", "b"), (2, "a", "b"), (4, "B", "c"), (1, "a", "a"),
+ (5, "b", "d")),
+ Seq(1, 2, 2, 3, 4, 5)
+ ),
+ (
+ "UNICODE",
+ Seq((4, "A", "C"), (3, "A", "b"), (2, "a", "b"), (5, "b", "c"), (1, "a", "a"),
+ (6, "b", "d")),
+ Seq(1, 2, 3, 4, 5, 6)
+ ),
+ (
+ "UNICODE_CI",
+ Seq((3, "A", "C"), (2, "A", "b"), (2, "a", "b"), (4, "B", "c"), (1, "a", "a"),
+ (5, "b", "d")),
+ Seq(1, 2, 2, 3, 4, 5)
+ )
+ ).foreach {
+ case (collationName, data, expResult) =>
+ val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName
+ val tableName = "t"
+ withTable(tableName) {
+ sql(s"create table $tableName (c1 integer, c2 struct<" +
+ s"s1: string $collationSetup," +
+ s"s2: string $collationSetup>)")
+ data.foreach {
+ case (c1, s1, s2) =>
+ sql(s"insert into $tableName values ($c1, struct('$s1', '$s2'))")
+ }
+ checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_)))
+ }
+ }
+ }
+
+ test("Check order by on StructType with few collated fields") {
+ val data = Seq(
+ (2, "b", "a", "a", "a", "a"),
+ (4, "b", "b", "B", "a", "a"),
+ (1, "a", "a", "a", "a", "a"),
+ (6, "b", "b", "b", "B", "B"),
+ (3, "b", "b", "a", "a", "a"),
+ (5, "b", "b", "b", "B", "a"))
+ val tableName = "t"
+ withTable(tableName) {
+ sql(s"create table $tableName (c1 integer, c2 struct<" +
+ s"s1: string, " +
+ s"s2: string collate UTF8_BINARY, " +
+ s"s3: string collate UTF8_LCASE, " +
+ s"s4: string collate UNICODE, " +
+ s"s5: string collate UNICODE_CI>)")
+ data.foreach {
+ case (order, s1, s2, s3, s4, s5) =>
+ sql(s"insert into $tableName values ($order, struct('$s1', '$s2', '$s3', '$s4', '$s5'))")
+ }
+ val expResult = Seq(1, 2, 3, 4, 5, 6)
+ checkAnswer(sql(s"select c1 from $tableName order by c2"), expResult.map(Row(_)))
+ }
+ }
+
+ test("Check order by on ArrayType with collated strings") {
+ Seq(
+ // (collationName, order, data)
+ (
+ "",
+ Seq((3, Seq("b", "Aa", "c")), (2, Seq("A", "b")), (1, Seq("A")), (2, Seq("A", "b"))),
+ Seq(1, 2, 2, 3)
+ ),
+ (
+ "UTF8_BINARY",
+ Seq((3, Seq("b", "Aa", "c")), (2, Seq("A", "b")), (1, Seq("A")), (2, Seq("A", "b"))),
+ Seq(1, 2, 2, 3)
+ ),
+ (
+ "UTF8_LCASE",
+ Seq((4, Seq("B", "a")), (4, Seq("b", "A")), (2, Seq("aa")), (1, Seq("A")),
+ (5, Seq("b", "e")), (3, Seq("b"))),
+ Seq(1, 2, 3, 4, 4, 5)
+ ),
+ (
+ "UNICODE",
+ Seq((5, Seq("b", "C")), (4, Seq("b", "AA")), (1, Seq("a")), (4, Seq("b", "AA")),
+ (3, Seq("b")), (2, Seq("A", "a"))),
+ Seq(1, 2, 3, 4, 4, 5)
+ ),
+ (
+ "UNICODE_CI",
+ Seq((4, Seq("B", "a")), (4, Seq("b", "A")), (2, Seq("aa")), (1, Seq("A")),
+ (5, Seq("b", "e")), (3, Seq("b"))),
+ Seq(1, 2, 3, 4, 4, 5)
+ )
+ ).foreach {
+ case (collationName, dataWithOrder, expResult) =>
+ val collationSetup = if (collationName.isEmpty) "" else "collate " + collationName
+ val tableName1 = "t1"
+ val tableName2 = "t2"
+ withTable(tableName1, tableName2) {
+ sql(s"create table $tableName1 (c1 integer, c2 array)")
+ sql(s"create table $tableName2 (c1 integer," +
+ s" c2 struct>)")
+ dataWithOrder.foreach {
+ case (order, data) =>
+ val arrayData = data.map(d => s"'$d'").mkString(", ")
+ sql(s"insert into $tableName1 values ($order, array($arrayData))")
+ sql(s"insert into $tableName2 values ($order, struct(array($arrayData)))")
+ }
+ checkAnswer(sql(s"select c1 from $tableName1 order by c2"), expResult.map(Row(_)))
+ checkAnswer(sql(s"select c1 from $tableName2 order by c2"), expResult.map(Row(_)))
+ }
+ }
+ }
+
+ test("Check order by on StructType with different types containing collated strings") {
+ val data = Seq(
+ (5, ("b", Seq(("b", "B", "a"), ("a", "a", "a")), "a")),
+ (2, ("b", Seq(("a", "a", "a")), "a")),
+ (2, ("b", Seq(("a", "a", "a")), "a")),
+ (4, ("b", Seq(("b", "a", "a")), "a")),
+ (3, ("b", Seq(("a", "a", "a"), ("a", "a", "a")), "a")),
+ (5, ("b", Seq(("b", "B", "a")), "a")),
+ (4, ("b", Seq(("b", "a", "a")), "a")),
+ (6, ("b", Seq(("b", "b", "B")), "A")),
+ (5, ("b", Seq(("b", "b", "a")), "a")),
+ (1, ("a", Seq(("a", "a", "a")), "a")),
+ (7, ("b", Seq(("b", "b", "B")), "b")),
+ (6, ("b", Seq(("b", "b", "B")), "a")),
+ (5, ("b", Seq(("b", "b", "a")), "a"))
+ )
+ val tableName = "t"
+ withTable(tableName) {
+ sql(s"create table $tableName " +
+ s"(c1 integer," +
+ s"c2 string," +
+ s"c3 array>," +
+ s"c4 string collate UNICODE_CI)")
+ data.foreach {
+ case (c1, (c2, c3, c4)) =>
+ val c3String = c3.map { case (f1, f2, f3) => s"struct('$f1', '$f2', '$f3')"}
+ .mkString(", ")
+ sql(s"insert into $tableName values ($c1, '$c2', array($c3String), '$c4')")
+ }
+ val expResult = Seq(1, 2, 2, 3, 4, 4, 5, 5, 5, 5, 6, 6, 7)
+ checkAnswer(sql(s"select c1 from $tableName order by c2, c3, c4"), expResult.map(Row(_)))
+ }
+ }
+
for (collation <- Seq("UTF8_BINARY", "UTF8_LCASE", "UNICODE", "UNICODE_CI",
"UNICODE_CI_RTRIM", "")) {
for (codeGen <- Seq("NO_CODEGEN", "CODEGEN_ONLY")) {
val collationSetup = if (collation.isEmpty) "" else " COLLATE " + collation
val supportsBinaryEquality = collation.isEmpty || collation == "UNICODE" ||
- CollationFactory.fetchCollation(collation).supportsBinaryEquality
+ CollationFactory.fetchCollation(collation).isUtf8BinaryType
test(s"Group by on map containing$collationSetup strings ($codeGen)") {
val tableName = "t"
@@ -1331,7 +1558,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
)
// Only if collation doesn't support binary equality, collation key should be injected.
- if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) {
+ if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) {
assert(collectFirst(queryPlan) {
case b: HashJoin => b.leftKeys.head
}.head.isInstanceOf[CollationKey])
@@ -1388,7 +1615,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
)
// Only if collation doesn't support binary equality, collation key should be injected.
- if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) {
+ if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) {
assert(collectFirst(queryPlan) {
case b: BroadcastHashJoinExec => b.leftKeys.head
}.head.asInstanceOf[ArrayTransform].function.asInstanceOf[LambdaFunction].
@@ -1449,7 +1676,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
)
// Only if collation doesn't support binary equality, collation key should be injected.
- if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) {
+ if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) {
assert(collectFirst(queryPlan) {
case b: BroadcastHashJoinExec => b.leftKeys.head
}.head.asInstanceOf[ArrayTransform].function.
@@ -1508,7 +1735,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
)
// Only if collation doesn't support binary equality, collation key should be injected.
- if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) {
+ if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) {
assert(queryPlan.toString().contains("collationkey"))
} else {
assert(!queryPlan.toString().contains("collationkey"))
@@ -1567,7 +1794,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper {
)
// Only if collation doesn't support binary equality, collation key should be injected.
- if (!CollationFactory.fetchCollation(t.collation).supportsBinaryEquality) {
+ if (!CollationFactory.fetchCollation(t.collation).isUtf8BinaryType) {
assert(queryPlan.toString().contains("collationkey"))
} else {
assert(!queryPlan.toString().contains("collationkey"))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
index e80c3b23a7db3..25f4d9f62354a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala
@@ -648,7 +648,7 @@ class DataFrameAggregateSuite extends QueryTest
condition = "DATATYPE_MISMATCH.UNSUPPORTED_INPUT_TYPE",
parameters = Map(
"functionName" -> "`collect_set`",
- "dataType" -> "\"MAP\"",
+ "dataType" -> "\"MAP\" or \"COLLATED STRING\"",
"sqlExpr" -> "\"collect_set(b)\""
),
context = ExpectedContext(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
index 8c1cc6c3bea1d..48ea0e01a4372 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameComplexTypeSuite.scala
@@ -39,6 +39,15 @@ import org.apache.spark.unsafe.types.CalendarInterval
class DataFrameComplexTypeSuite extends QueryTest with SharedSparkSession {
import testImplicits._
+ test("ArrayTransform with scan input") {
+ withTempPath { f =>
+ spark.sql("select array(array(1, null, 3), array(4, 5, null), array(null, 8, 9)) as a")
+ .write.parquet(f.getAbsolutePath)
+ val df = spark.read.parquet(f.getAbsolutePath).selectExpr("transform(a, (x, i) -> x)")
+ checkAnswer(df, Row(Seq(Seq(1, null, 3), Seq(4, 5, null), Seq(null, 8, 9))))
+ }
+ }
+
test("UDF on struct") {
val f = udf((a: String) => a)
val df = sparkContext.parallelize(Seq((1, 1))).toDF("a", "b")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
index 47691e1ccd40f..39c839ae5a518 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala
@@ -478,7 +478,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
intercept[AnalysisException](df.select(expr)),
condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
parameters = Map(
- "inputName" -> "length",
+ "inputName" -> "`length`",
"inputType" -> "INT or SMALLINT",
"inputExpr" -> "\"a\"",
"sqlExpr" -> "\"randstr(a, 10)\""),
@@ -530,7 +530,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSparkSession {
intercept[AnalysisException](df.select(expr)),
condition = "DATATYPE_MISMATCH.NON_FOLDABLE_INPUT",
parameters = Map(
- "inputName" -> "min",
+ "inputName" -> "`min`",
"inputType" -> "integer or floating-point",
"inputExpr" -> "\"a\"",
"sqlExpr" -> "\"uniform(a, 10)\""),
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/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
index e44bd5de4f4c4..9c529d1422119 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala
@@ -128,13 +128,20 @@ class FileBasedDataSourceSuite extends QueryTest
allFileBasedDataSources.foreach { format =>
test(s"SPARK-23372 error while writing empty schema files using $format") {
+ val formatMapping = Map(
+ "csv" -> "CSV",
+ "json" -> "JSON",
+ "parquet" -> "Parquet",
+ "orc" -> "ORC",
+ "text" -> "Text"
+ )
withTempPath { outputPath =>
checkError(
exception = intercept[AnalysisException] {
spark.emptyDataFrame.write.format(format).save(outputPath.toString)
},
- condition = "_LEGACY_ERROR_TEMP_1142",
- parameters = Map.empty
+ condition = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE",
+ parameters = Map("format" -> formatMapping(format))
)
}
@@ -150,8 +157,8 @@ class FileBasedDataSourceSuite extends QueryTest
exception = intercept[AnalysisException] {
df.write.format(format).save(outputPath.toString)
},
- condition = "_LEGACY_ERROR_TEMP_1142",
- parameters = Map.empty
+ condition = "EMPTY_SCHEMA_NOT_SUPPORTED_FOR_DATASOURCE",
+ parameters = Map("format" -> formatMapping(format))
)
}
}
@@ -506,14 +513,23 @@ class FileBasedDataSourceSuite extends QueryTest
withSQLConf(
SQLConf.USE_V1_SOURCE_LIST.key -> useV1List,
SQLConf.LEGACY_INTERVAL_ENABLED.key -> "true") {
+ val formatMapping = Map(
+ "csv" -> "CSV",
+ "json" -> "JSON",
+ "parquet" -> "Parquet",
+ "orc" -> "ORC"
+ )
// write path
Seq("csv", "json", "parquet", "orc").foreach { format =>
checkError(
exception = intercept[AnalysisException] {
sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir)
},
- condition = "_LEGACY_ERROR_TEMP_1136",
- parameters = Map.empty
+ condition = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE",
+ parameters = Map(
+ "format" -> formatMapping(format),
+ "columnName" -> "`INTERVAL '1 days'`",
+ "columnType" -> "\"INTERVAL\"")
)
}
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/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
index 16118526f2fe4..76919d6583106 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -163,9 +163,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession
exception = intercept[SparkUnsupportedOperationException] {
Seq(InvalidInJava(1)).toDS()
},
- condition = "_LEGACY_ERROR_TEMP_2140",
+ condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME",
parameters = Map(
- "fieldName" -> "abstract",
+ "fieldName" -> "`abstract`",
"walkedTypePath" -> "- root class: \"org.apache.spark.sql.InvalidInJava\""))
}
@@ -174,9 +174,9 @@ class ScalaReflectionRelationSuite extends SparkFunSuite with SharedSparkSession
exception = intercept[SparkUnsupportedOperationException] {
Seq(InvalidInJava2(1)).toDS()
},
- condition = "_LEGACY_ERROR_TEMP_2140",
+ condition = "INVALID_JAVA_IDENTIFIER_AS_FIELD_NAME",
parameters = Map(
- "fieldName" -> "0",
+ "fieldName" -> "`0`",
"walkedTypePath" ->
"- root class: \"org.apache.spark.sql.ScalaReflectionRelationSuite.InvalidInJava2\""))
}
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/VariantEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala
index 19d4ac23709b6..fe5c6ef004920 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/VariantEndToEndSuite.scala
@@ -16,7 +16,7 @@
*/
package org.apache.spark.sql
-import org.apache.spark.SparkThrowable
+import org.apache.spark.{SparkException, SparkRuntimeException}
import org.apache.spark.sql.QueryTest.sameRows
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Cast, Literal}
@@ -359,16 +359,24 @@ class VariantEndToEndSuite extends QueryTest with SharedSparkSession {
val expectedMetadata: Array[Byte] = Array(VERSION, 3, 0, 1, 2, 3, 'a', 'b', 'c')
assert(actual === new VariantVal(expectedValue, expectedMetadata))
}
- withSQLConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS.key -> "false") {
- val df = Seq(json).toDF("j")
- .selectExpr("from_json(j,'variant')")
- checkError(
- exception = intercept[SparkThrowable] {
+ // Check whether the parse_json and from_json expressions throw the correct exception.
+ Seq("from_json(j, 'variant')", "parse_json(j)").foreach { expr =>
+ withSQLConf(SQLConf.VARIANT_ALLOW_DUPLICATE_KEYS.key -> "false") {
+ val df = Seq(json).toDF("j").selectExpr(expr)
+ val exception = intercept[SparkException] {
df.collect()
- },
- condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION",
- parameters = Map("badRecord" -> json, "failFastMode" -> "FAILFAST")
- )
+ }
+ checkError(
+ exception = exception,
+ condition = "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION",
+ parameters = Map("badRecord" -> json, "failFastMode" -> "FAILFAST")
+ )
+ checkError(
+ exception = exception.getCause.asInstanceOf[SparkRuntimeException],
+ condition = "VARIANT_DUPLICATE_KEY",
+ parameters = Map("key" -> "a")
+ )
+ }
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
index d6599debd3b11..6b0fd6084099c 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2FunctionSuite.scala
@@ -414,8 +414,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase {
new JavaStrLen(new JavaStrLenNoImpl))
checkError(
exception = intercept[AnalysisException](sql("SELECT testcat.ns.strlen('abc')").collect()),
- condition = "_LEGACY_ERROR_TEMP_3055",
- parameters = Map("scalarFunc" -> "strlen"),
+ condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED",
+ parameters = Map("scalarFunc" -> "`strlen`"),
context = ExpectedContext(
fragment = "testcat.ns.strlen('abc')",
start = 7,
@@ -448,8 +448,8 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase {
addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddMismatchMagic))
checkError(
exception = intercept[AnalysisException](sql("SELECT testcat.ns.add(1L, 2L)").collect()),
- condition = "_LEGACY_ERROR_TEMP_3055",
- parameters = Map("scalarFunc" -> "long_add_mismatch_magic"),
+ condition = "SCALAR_FUNCTION_NOT_FULLY_IMPLEMENTED",
+ parameters = Map("scalarFunc" -> "`long_add_mismatch_magic`"),
context = ExpectedContext(
fragment = "testcat.ns.add(1L, 2L)",
start = 7,
@@ -458,6 +458,23 @@ class DataSourceV2FunctionSuite extends DatasourceV2SQLBase {
)
}
+ test("SPARK-49549: scalar function w/ mismatch a compatible ScalarFunction#produceResult") {
+ case object CharLength extends ScalarFunction[Int] {
+ override def inputTypes(): Array[DataType] = Array(StringType)
+ override def resultType(): DataType = IntegerType
+ override def name(): String = "CHAR_LENGTH"
+ }
+
+ catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps)
+ addFunction(Identifier.of(Array("ns"), "my_strlen"), StrLen(CharLength))
+ checkError(
+ exception = intercept[SparkUnsupportedOperationException]
+ (sql("SELECT testcat.ns.my_strlen('abc')").collect()),
+ condition = "SCALAR_FUNCTION_NOT_COMPATIBLE",
+ parameters = Map("scalarFunc" -> "`CHAR_LENGTH`")
+ )
+ }
+
test("SPARK-35390: scalar function w/ type coercion") {
catalog("testcat").asInstanceOf[SupportsNamespaces].createNamespace(Array("ns"), emptyProps)
addFunction(Identifier.of(Array("ns"), "add"), new JavaLongAdd(new JavaLongAddDefault(false)))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
index 6b58d23e92603..52ae1bf5d9d3b 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DataSourceV2SQLSuite.scala
@@ -840,7 +840,7 @@ class DataSourceV2SQLSuiteV1Filter
val exception = intercept[SparkRuntimeException] {
insertNullValueAndCheck()
}
- assert(exception.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(exception.getCondition == "NOT_NULL_ASSERT_VIOLATION")
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala
index 9d4e4fc016722..053616c88d638 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoTableSuiteBase.scala
@@ -1326,7 +1326,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase {
| UPDATE SET s = named_struct('n_i', null, 'n_l', -1L)
|""".stripMargin)
}
- assert(e1.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(e1.getCondition == "NOT_NULL_ASSERT_VIOLATION")
val e2 = intercept[SparkRuntimeException] {
sql(
@@ -1337,7 +1337,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase {
| UPDATE SET s = named_struct('n_i', null, 'n_l', -1L)
|""".stripMargin)
}
- assert(e2.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(e2.getCondition == "NOT_NULL_ASSERT_VIOLATION")
val e3 = intercept[SparkRuntimeException] {
sql(
@@ -1348,7 +1348,7 @@ abstract class MergeIntoTableSuiteBase extends RowLevelOperationSuiteBase {
| INSERT (pk, s, dep) VALUES (s.pk, named_struct('n_i', null, 'n_l', -1L), 'invalid')
|""".stripMargin)
}
- assert(e3.getErrorClass == "NOT_NULL_ASSERT_VIOLATION")
+ assert(e3.getCondition == "NOT_NULL_ASSERT_VIOLATION")
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
index b4fdf50447458..92c175fe2f94a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryCompilationErrorsSuite.scala
@@ -1003,7 +1003,7 @@ class QueryCompilationErrorsSuite
val exception = intercept[AnalysisException] {
sql(queryWithTrailingComma)
}
- assert(exception.getErrorClass === "TRAILING_COMMA_IN_SELECT")
+ assert(exception.getCondition === "TRAILING_COMMA_IN_SELECT")
}
val unresolvedColumnErrors = Seq(
@@ -1017,7 +1017,7 @@ class QueryCompilationErrorsSuite
val exception = intercept[AnalysisException] {
sql(query)
}
- assert(exception.getErrorClass === "UNRESOLVED_COLUMN.WITH_SUGGESTION")
+ assert(exception.getCondition === "UNRESOLVED_COLUMN.WITH_SUGGESTION")
}
// sanity checks
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
index 9d1448d0ac09d..1adb1fdf05032 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
@@ -35,11 +35,12 @@ import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, Encoder, Kry
import org.apache.spark.sql.catalyst.FunctionIdentifier
import org.apache.spark.sql.catalyst.analysis.{NamedParameter, UnresolvedGenerator}
import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder}
-import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression}
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Concat, CreateArray, EmptyRow, Expression, Flatten, Grouping, Literal, RowNumber, UnaryExpression, Years}
import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode._
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode}
import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean
import org.apache.spark.sql.catalyst.rules.RuleIdCollection
+import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLExpr
import org.apache.spark.sql.execution.datasources.jdbc.{DriverRegistry, JDBCOptions}
import org.apache.spark.sql.execution.datasources.jdbc.connection.ConnectionProvider
import org.apache.spark.sql.execution.datasources.orc.OrcTest
@@ -292,7 +293,7 @@ class QueryExecutionErrorsSuite
val e = intercept[SparkException] {
df.write.parquet(dir.getCanonicalPath)
}
- assert(e.getErrorClass == "TASK_WRITE_FAILED")
+ assert(e.getCondition == "TASK_WRITE_FAILED")
val format = "Parquet"
val config = "\"" + SQLConf.PARQUET_REBASE_MODE_IN_WRITE.key + "\""
@@ -311,7 +312,7 @@ class QueryExecutionErrorsSuite
val ex = intercept[SparkException] {
spark.read.schema("time timestamp_ntz").orc(file.getCanonicalPath).collect()
}
- assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
+ assert(ex.getCondition.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException],
condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST",
@@ -333,7 +334,7 @@ class QueryExecutionErrorsSuite
val ex = intercept[SparkException] {
spark.read.schema("time timestamp_ltz").orc(file.getCanonicalPath).collect()
}
- assert(ex.getErrorClass.startsWith("FAILED_READ_FILE"))
+ assert(ex.getCondition.startsWith("FAILED_READ_FILE"))
checkError(
exception = ex.getCause.asInstanceOf[SparkUnsupportedOperationException],
condition = "UNSUPPORTED_FEATURE.ORC_TYPE_CAST",
@@ -381,7 +382,7 @@ class QueryExecutionErrorsSuite
}
val e2 = e1.getCause.asInstanceOf[SparkException]
- assert(e2.getErrorClass == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION")
+ assert(e2.getCondition == "MALFORMED_RECORD_IN_PARSING.WITHOUT_SUGGESTION")
checkError(
exception = e2.getCause.asInstanceOf[SparkRuntimeException],
@@ -920,7 +921,7 @@ class QueryExecutionErrorsSuite
val e = intercept[StreamingQueryException] {
query.awaitTermination()
}
- assert(e.getErrorClass === "STREAM_FAILED")
+ assert(e.getCondition === "STREAM_FAILED")
assert(e.getCause.isInstanceOf[NullPointerException])
}
@@ -1006,6 +1007,17 @@ class QueryExecutionErrorsSuite
sqlState = "XX000")
}
+ test("PartitionTransformExpression error on eval") {
+ val expr = Years(Literal("foo"))
+ val e = intercept[SparkException] {
+ expr.eval()
+ }
+ checkError(
+ exception = e,
+ condition = "PARTITION_TRANSFORM_EXPRESSION_NOT_IN_PARTITIONED_BY",
+ parameters = Map("expression" -> toSQLExpr(expr)))
+ }
+
test("INTERNAL_ERROR: Calling doGenCode on unresolved") {
val e = intercept[SparkException] {
val ctx = new CodegenContext
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
index 75f016d050de9..c5e64c96b2c8a 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala
@@ -904,7 +904,7 @@ class AdaptiveQueryExecSuite
val error = intercept[SparkException] {
aggregated.count()
}
- assert(error.getErrorClass === "INVALID_BUCKET_FILE")
+ assert(error.getCondition === "INVALID_BUCKET_FILE")
assert(error.getMessage contains "Invalid bucket file")
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
index deb62eb3ac234..387a2baa256bf 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala
@@ -368,7 +368,7 @@ class BinaryFileFormatSuite extends QueryTest with SharedSparkSession {
checkAnswer(readContent(), expected)
}
}
- assert(caught.getErrorClass.startsWith("FAILED_READ_FILE"))
+ assert(caught.getCondition.startsWith("FAILED_READ_FILE"))
assert(caught.getCause.getMessage.contains("exceeds the max length allowed"))
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala
new file mode 100644
index 0000000000000..8c8304503cef8
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVParsingOptionsSuite.scala
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.csv
+
+import org.apache.spark.sql.{QueryTest, Row}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class CSVParsingOptionsSuite extends QueryTest with SharedSparkSession {
+ import testImplicits._
+
+ test("SPARK-49955: null string value does not mean corrupted file") {
+ val str = "abc"
+ val stringDataset = Seq(str, null).toDS()
+ val df = spark.read.csv(stringDataset)
+ // `spark.read.csv(rdd)` removes all null values at the beginning.
+ checkAnswer(df, Seq(Row("abc")))
+ val df2 = spark.read.option("mode", "failfast").csv(stringDataset)
+ checkAnswer(df2, Seq(Row("abc")))
+ }
+}
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/json/JsonParsingOptionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
index 703085dca66f1..11cc0b99bbde7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonParsingOptionsSuite.scala
@@ -17,6 +17,7 @@
package org.apache.spark.sql.execution.datasources.json
+import org.apache.spark.SparkException
import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{StringType, StructType}
@@ -185,4 +186,14 @@ class JsonParsingOptionsSuite extends QueryTest with SharedSparkSession {
assert(df.first().getString(0) == "Cazen Lee")
assert(df.first().getString(1) == "$10")
}
+
+ test("SPARK-49955: null string value does not mean corrupted file") {
+ val str = "{\"name\": \"someone\"}"
+ val stringDataset = Seq(str, null).toDS()
+ val df = spark.read.json(stringDataset)
+ checkAnswer(df, Seq(Row(null, "someone"), Row(null, null)))
+
+ val e = intercept[SparkException](spark.read.option("mode", "failfast").json(stringDataset))
+ assert(e.getCause.isInstanceOf[NullPointerException])
+ }
}
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/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
index c88f51a6b7d06..8091d6e64fdc1 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala
@@ -1173,7 +1173,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
exception = intercept[SparkUnsupportedOperationException] {
catalog.alterNamespace(testNs, NamespaceChange.removeProperty(p))
},
- condition = "_LEGACY_ERROR_TEMP_2069",
+ condition = "CANNOT_REMOVE_RESERVED_PROPERTY",
parameters = Map("property" -> p))
}
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/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
index aca968745d198..0cc4f7bf2548e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala
@@ -504,6 +504,12 @@ class ColumnVectorSuite extends SparkFunSuite with SQLHelper {
val arr = new ColumnarArray(testVector, 0, testVector.capacity)
assert(arr.toSeq(testVector.dataType) == expected)
assert(arr.copy().toSeq(testVector.dataType) == expected)
+
+ if (expected.nonEmpty) {
+ val withOffset = new ColumnarArray(testVector, 1, testVector.capacity - 1)
+ assert(withOffset.toSeq(testVector.dataType) == expected.tail)
+ assert(withOffset.copy().toSeq(testVector.dataType) == expected.tail)
+ }
}
testVectors("getInts with dictionary and nulls", 3, IntegerType) { testVector =>
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))
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
index 054c7e644ff55..0550fae3805d4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCV2Suite.scala
@@ -2688,7 +2688,7 @@ class JDBCV2Suite extends QueryTest with SharedSparkSession with ExplainSuiteHel
val df = sql("SELECT SUM(2147483647 + DEPT) FROM h2.test.employee")
checkAggregateRemoved(df, ansiMode)
val expectedPlanFragment = if (ansiMode) {
- "PushedAggregates: [SUM(2147483647 + DEPT)], " +
+ "PushedAggregates: [SUM(DEPT + 2147483647)], " +
"PushedFilters: [], " +
"PushedGroupByExpressions: []"
} 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/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
index 8471995cb1e50..c12846d7512d9 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala
@@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.Complete
import org.apache.spark.sql.catalyst.types.DataTypeUtils.toAttributes
import org.apache.spark.sql.connector.read.InputPartition
import org.apache.spark.sql.connector.read.streaming.{Offset => OffsetV2, ReadLimit}
-import org.apache.spark.sql.execution.exchange.ReusedExchangeExec
+import org.apache.spark.sql.execution.exchange.{REQUIRED_BY_STATEFUL_OPERATOR, ReusedExchangeExec, ShuffleExchangeExec}
import org.apache.spark.sql.execution.streaming._
import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter}
import org.apache.spark.sql.functions._
@@ -1448,6 +1448,28 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi
}
}
+ test("SPARK-49905 shuffle added by stateful operator should use the shuffle origin " +
+ "`REQUIRED_BY_STATEFUL_OPERATOR`") {
+ val inputData = MemoryStream[Int]
+
+ // Use the streaming aggregation as an example - all stateful operators are using the same
+ // distribution, named `StatefulOpClusteredDistribution`.
+ val df = inputData.toDF().groupBy("value").count()
+
+ testStream(df, OutputMode.Update())(
+ AddData(inputData, 1, 2, 3, 1, 2, 3),
+ CheckAnswer((1, 2), (2, 2), (3, 2)),
+ Execute { qe =>
+ val shuffleOpt = qe.lastExecution.executedPlan.collect {
+ case s: ShuffleExchangeExec => s
+ }
+
+ assert(shuffleOpt.nonEmpty, "No shuffle exchange found in the query plan")
+ assert(shuffleOpt.head.shuffleOrigin === REQUIRED_BY_STATEFUL_OPERATOR)
+ }
+ )
+ }
+
private def checkAppendOutputModeException(df: DataFrame): Unit = {
withTempDir { outputDir =>
withTempDir { checkpointDir =>
diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala
index 8a8bdd4d38ee3..59d1b61f2f8e7 100644
--- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala
+++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServerErrors.scala
@@ -38,7 +38,7 @@ object HiveThriftServerErrors {
def runningQueryError(e: Throwable, format: ErrorMessageFormat.Value): Throwable = e match {
case st: SparkThrowable if format == ErrorMessageFormat.PRETTY =>
- val errorClassPrefix = Option(st.getErrorClass).map(e => s"[$e] ").getOrElse("")
+ val errorClassPrefix = Option(st.getCondition).map(e => s"[$e] ").getOrElse("")
new HiveSQLException(
s"Error running query: $errorClassPrefix${st.toString}", st.getSqlState, st)
case st: SparkThrowable with Throwable =>