From d824e9ee659c82b61e54c6a08bc0dcbdf0eb9888 Mon Sep 17 00:00:00 2001 From: Nebojsa Savic Date: Tue, 9 Jul 2024 22:31:05 +0800 Subject: [PATCH 01/64] [SPARK-46625] CTE with Identifier clause as reference ### What changes were proposed in this pull request? DECLARE agg = 'max'; DECLARE col = 'c1'; DECLARE tab = 'T'; WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab); -- OR WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T'); Currently we don't support Identifier clause as part of CTE reference. ### Why are the changes needed? Adding support for Identifier clause as part of CTE reference for both constant string expressions and session variables. ### Does this PR introduce _any_ user-facing change? It contains user facing changes in sense that identifier clause as cte reference will now be supported. ### How was this patch tested? Added tests as part of this PR. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47180 from nebojsa-db/SPARK-46625. Authored-by: Nebojsa Savic Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 3 + .../catalyst/analysis/CTESubstitution.scala | 13 +++- .../analysis/ResolveIdentifierClause.scala | 15 +++- .../sql/catalyst/analysis/unresolved.scala | 13 +++- .../sql/catalyst/trees/TreePatterns.scala | 1 + .../identifier-clause.sql.out | 73 +++++++++++++++++++ .../sql-tests/inputs/identifier-clause.sql | 16 ++++ .../results/identifier-clause.sql.out | 53 ++++++++++++++ 8 files changed, 181 insertions(+), 6 deletions(-) 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 ba6764444bdf3..95e2ddd40af1f 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 @@ -1796,6 +1796,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor case s: Sort if !s.resolved || s.missingInput.nonEmpty => resolveReferencesInSort(s) + case u: UnresolvedWithCTERelations => + UnresolvedWithCTERelations(this.apply(u.unresolvedPlan), u.cteRelations) + case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString(conf.maxToStringFields)}") q.mapExpressions(resolveExpressionByPlanChildren(_, q, includeLastResort = true)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala index 2982d8477fcc1..ff0dbcd7ef153 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.{Command, CTEInChildren, CTERelationDef, CTERelationRef, InsertIntoDir, LogicalPlan, ParsedStatement, SubqueryAlias, UnresolvedWith, WithCTE} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ -import org.apache.spark.sql.catalyst.util.TypeUtils._ +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_POLICY @@ -272,7 +272,8 @@ object CTESubstitution extends Rule[LogicalPlan] { alwaysInline: Boolean, cteRelations: Seq[(String, CTERelationDef)]): LogicalPlan = { plan.resolveOperatorsUpWithPruning( - _.containsAnyPattern(RELATION_TIME_TRAVEL, UNRESOLVED_RELATION, PLAN_EXPRESSION)) { + _.containsAnyPattern(RELATION_TIME_TRAVEL, UNRESOLVED_RELATION, PLAN_EXPRESSION, + UNRESOLVED_IDENTIFIER)) { case RelationTimeTravel(UnresolvedRelation(Seq(table), _, _), _, _) if cteRelations.exists(r => plan.conf.resolver(r._1, table)) => throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(table)) @@ -287,6 +288,14 @@ object CTESubstitution extends Rule[LogicalPlan] { } }.getOrElse(u) + case p: PlanWithUnresolvedIdentifier => + // We must look up CTE relations first when resolving `UnresolvedRelation`s, + // but we can't do it here as `PlanWithUnresolvedIdentifier` is a leaf node + // and may produce `UnresolvedRelation` later. + // Here we wrap it with `UnresolvedWithCTERelations` so that we can + // delay the CTE relations lookup after `PlanWithUnresolvedIdentifier` is resolved. + UnresolvedWithCTERelations(p, cteRelations) + case other => // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala index f04b7799e35ea..e0142c445ae82 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveIdentifierClause.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.{AliasHelper, EvalHelper, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationRef, LogicalPlan, SubqueryAlias} import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} -import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_IDENTIFIER +import org.apache.spark.sql.catalyst.trees.TreePattern.{UNRESOLVED_IDENTIFIER, UNRESOLVED_IDENTIFIER_WITH_CTE} import org.apache.spark.sql.types.StringType /** @@ -35,9 +35,18 @@ class ResolveIdentifierClause(earlyBatches: Seq[RuleExecutor[LogicalPlan]#Batch] } override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUpWithPruning( - _.containsAnyPattern(UNRESOLVED_IDENTIFIER)) { + _.containsAnyPattern(UNRESOLVED_IDENTIFIER, UNRESOLVED_IDENTIFIER_WITH_CTE)) { case p: PlanWithUnresolvedIdentifier if p.identifierExpr.resolved => executor.execute(p.planBuilder.apply(evalIdentifierExpr(p.identifierExpr))) + case u @ UnresolvedWithCTERelations(p, cteRelations) => + this.apply(p) match { + case u @ UnresolvedRelation(Seq(table), _, _) => + cteRelations.find(r => plan.conf.resolver(r._1, table)).map { case (_, d) => + // Add a `SubqueryAlias` for hint-resolving rules to match relation names. + SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) + }.getOrElse(u) + case other => other + } case other => other.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_IDENTIFIER)) { case e: ExpressionWithUnresolvedIdentifier if e.identifierExpr.resolved => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index a2cab60b392b6..abb7e7956f184 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIden import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, UnaryNode} +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LeafNode, LogicalPlan, UnaryNode} import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId @@ -65,6 +65,17 @@ case class PlanWithUnresolvedIdentifier( final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER) } +/** + * A logical plan placeholder which delays CTE resolution + * to moment when PlanWithUnresolvedIdentifier gets resolved + */ +case class UnresolvedWithCTERelations( + unresolvedPlan: LogicalPlan, + cteRelations: Seq[(String, CTERelationDef)]) + extends UnresolvedLeafNode { + final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_IDENTIFIER_WITH_CTE) +} + /** * An expression placeholder that holds the identifier clause string expression. It will be * replaced by the actual expression with the evaluated identifier string. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala index c5cc1eaf8f05d..6258bd615b440 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreePatterns.scala @@ -151,6 +151,7 @@ object TreePattern extends Enumeration { val UNRESOLVED_FUNCTION: Value = Value val UNRESOLVED_HINT: Value = Value val UNRESOLVED_WINDOW_EXPRESSION: Value = Value + val UNRESOLVED_IDENTIFIER_WITH_CTE: Value = Value // Unresolved Plan patterns (Alphabetically ordered) val UNRESOLVED_FUNC: Value = Value diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out index b3e2cd5ada950..f0bf8b883dd8b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/identifier-clause.sql.out @@ -985,6 +985,79 @@ DropTable false, false +- ResolvedIdentifier V2SessionCatalog(spark_catalog), default.t2 +-- !query +DECLARE agg = 'max' +-- !query analysis +CreateVariable defaultvalueexpression(max, 'max'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.agg + + +-- !query +DECLARE col = 'c1' +-- !query analysis +CreateVariable defaultvalueexpression(c1, 'c1'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.col + + +-- !query +DECLARE tab = 'T' +-- !query analysis +CreateVariable defaultvalueexpression(T, 'T'), false ++- ResolvedIdentifier org.apache.spark.sql.catalyst.analysis.FakeSystemCatalog$@xxxxxxxx, session.tab + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias S +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] +:- CTERelationDef xxxx, false +: +- SubqueryAlias T +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias T + +- CTERelationRef xxxx, true, [c1#x, c2#x], false + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias ABC +: +- Project [col1#x AS c1#x, col2#x AS c2#x] +: +- LocalRelation [col1#x, col2#x] ++- Aggregate [max(c1#x) AS max(c1)#x] + +- SubqueryAlias ABC + +- CTERelationRef xxxx, true, [c1#x, c2#x], false + + -- !query SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql index 46461dcd048e3..4aa8019097fdf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/identifier-clause.sql @@ -141,6 +141,22 @@ drop view v1; drop table t1; drop table t2; +-- SPARK-46625: CTE reference with identifier clause and session variables +DECLARE agg = 'max'; +DECLARE col = 'c1'; +DECLARE tab = 'T'; + +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab); + +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T'); + +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC'); + -- Not supported SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1); SELECT T1.c1 FROM VALUES(1) AS T1(c1) JOIN VALUES(1) AS T2(c1) USING (IDENTIFIER('c1')); diff --git a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out index 2aa809324a763..952fb8fdc2bd2 100644 --- a/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/identifier-clause.sql.out @@ -1115,6 +1115,59 @@ struct<> +-- !query +DECLARE agg = 'max' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE col = 'c1' +-- !query schema +struct<> +-- !query output + + + +-- !query +DECLARE tab = 'T' +-- !query schema +struct<> +-- !query output + + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) +-- !query schema +struct +-- !query output +c + + +-- !query +WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), + T(c1, c2) AS (VALUES ('a', 'b'), ('c', 'd')) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') +-- !query schema +struct +-- !query output +c + + +-- !query +WITH ABC(c1, c2) AS (VALUES(1, 2), (2, 3)) +SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('A' || 'BC') +-- !query schema +struct +-- !query output +2 + + -- !query SELECT row_number() OVER IDENTIFIER('x.win') FROM VALUES(1) AS T(c1) WINDOW win AS (ORDER BY c1) -- !query schema From 4c99c4df7f9cda9b35bc7401c92da22ea683861b Mon Sep 17 00:00:00 2001 From: Lingkai Kong Date: Tue, 9 Jul 2024 11:29:52 -0700 Subject: [PATCH 02/64] [SPARK-48716] Add jobGroupId to SparkListenerSQLExecutionStart ### What changes were proposed in this pull request? Add jobGroupId to SparkListenerSQLExecutionStart ### Why are the changes needed? JobGroupId can be used to combine jobs within the same group. This is going to be useful in the listener so it makes the job grouping easy to do ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit Test ### Was this patch authored or co-authored using generative AI tooling? No Closes #47092 from gjxdxh/gjxdxh/SPARK-48716. Authored-by: Lingkai Kong Signed-off-by: Josh Rosen --- .../spark/sql/execution/SQLExecution.scala | 5 +-- .../execution/ui/SQLAppStatusListener.scala | 2 +- .../spark/sql/execution/ui/SQLListener.scala | 3 +- .../sql/execution/SQLExecutionSuite.scala | 33 +++++++++++++++++++ .../ui/SQLAppStatusListenerSuite.scala | 2 +- 5 files changed, 40 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala index f4be03c90be75..7c03bad90ebbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SQLExecution.scala @@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicLong import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal -import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper} +import org.apache.spark.{ErrorMessageFormat, JobArtifactSet, SparkContext, SparkEnv, SparkException, SparkThrowable, SparkThrowableHelper} import org.apache.spark.SparkContext.{SPARK_JOB_DESCRIPTION, SPARK_JOB_INTERRUPT_ON_CANCEL} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX} @@ -128,7 +128,8 @@ object SQLExecution extends Logging { sparkPlanInfo = SparkPlanInfo.EMPTY, time = System.currentTimeMillis(), modifiedConfigs = redactedConfigs, - jobTags = sc.getJobTags() + jobTags = sc.getJobTags(), + jobGroupId = Option(sc.getLocalProperty(SparkContext.SPARK_JOB_GROUP_ID)) ) try { body match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index bf33ba2c96f19..dcbf328c71e33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -343,7 +343,7 @@ class SQLAppStatusListener( private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { val SparkListenerSQLExecutionStart(executionId, rootExecutionId, description, details, - physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _) = event + physicalPlanDescription, sparkPlanInfo, time, modifiedConfigs, _, _) = event val planGraph = SparkPlanGraph(sparkPlanInfo) val sqlPlanMetrics = planGraph.allNodes.flatMap { node => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 3a22dd23548fc..416b9547b0462 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -54,7 +54,8 @@ case class SparkListenerSQLExecutionStart( sparkPlanInfo: SparkPlanInfo, time: Long, modifiedConfigs: Map[String, String] = Map.empty, - jobTags: Set[String] = Set.empty) + jobTags: Set[String] = Set.empty, + jobGroupId: Option[String] = None) extends SparkListenerEvent @DeveloperApi diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala index b8a109919f8f6..94d33731b6de5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLExecutionSuite.scala @@ -227,6 +227,7 @@ class SQLExecutionSuite extends SparkFunSuite with SQLConfHelper { spark.range(1).collect() + spark.sparkContext.listenerBus.waitUntilEmpty() assert(jobTags.contains(jobTag)) assert(sqlJobTags.contains(jobTag)) } finally { @@ -234,6 +235,38 @@ class SQLExecutionSuite extends SparkFunSuite with SQLConfHelper { spark.stop() } } + + test("jobGroupId property") { + val spark = SparkSession.builder().master("local[*]").appName("test").getOrCreate() + val JobGroupId = "test-JobGroupId" + try { + spark.sparkContext.setJobGroup(JobGroupId, "job Group id") + + var jobGroupIdOpt: Option[String] = None + var sqlJobGroupIdOpt: Option[String] = None + spark.sparkContext.addSparkListener(new SparkListener { + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobGroupIdOpt = Some(jobStart.properties.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = { + event match { + case e: SparkListenerSQLExecutionStart => + sqlJobGroupIdOpt = e.jobGroupId + } + } + }) + + spark.range(1).collect() + + spark.sparkContext.listenerBus.waitUntilEmpty() + assert(jobGroupIdOpt.contains(JobGroupId)) + assert(sqlJobGroupIdOpt.contains(JobGroupId)) + } finally { + spark.sparkContext.clearJobGroup() + spark.stop() + } + } } object SQLExecutionSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala index 17e77cf8d8fb3..e63ff019a2b6c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala @@ -344,7 +344,7 @@ abstract class SQLAppStatusListenerSuite extends SharedSparkSession with JsonTes val listener = new SparkListener { override def onOtherEvent(event: SparkListenerEvent): Unit = { event match { - case SparkListenerSQLExecutionStart(_, _, _, _, planDescription, _, _, _, _) => + case SparkListenerSQLExecutionStart(_, _, _, _, planDescription, _, _, _, _, _) => assert(expected.forall(planDescription.contains)) checkDone = true case _ => // ignore other events From f73884328cb8d2bdf81447049117f5a903b93a2e Mon Sep 17 00:00:00 2001 From: Wei Guo Date: Wed, 10 Jul 2024 09:55:50 +0900 Subject: [PATCH 03/64] [SPARK-44728][PYTHON][DOCS] Fix the incorrect naming and missing params in func docs in `builtin.py` ### What changes were proposed in this pull request? Fix the incorrect naming and missing params in func docs in `builtin.py`. ### Why are the changes needed? Some params' name in `pySpark` docs are wrong, for example: ![image](https://github.com/apache/spark/assets/16032294/af0ca3c9-b085-4364-8cfc-814371f21b4b) ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Passed GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47269 from wayneguow/py_docs. Authored-by: Wei Guo Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 30 +++++++++++++------------ 1 file changed, 16 insertions(+), 14 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1508b042b61a6..0ff830e8d48d6 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -5289,7 +5289,7 @@ def corr(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: ---------- col1 : :class:`~pyspark.sql.Column` or str first column to calculate correlation. - col1 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or str second column to calculate correlation. Returns @@ -5322,7 +5322,7 @@ def covar_pop(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: ---------- col1 : :class:`~pyspark.sql.Column` or str first column to calculate covariance. - col1 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or str second column to calculate covariance. Returns @@ -5355,7 +5355,7 @@ def covar_samp(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: ---------- col1 : :class:`~pyspark.sql.Column` or str first column to calculate covariance. - col1 : :class:`~pyspark.sql.Column` or str + col2 : :class:`~pyspark.sql.Column` or str second column to calculate covariance. Returns @@ -5494,7 +5494,7 @@ def first(col: "ColumnOrName", ignorenulls: bool = False) -> Column: | Bob| 5| +-----+----------+ - Now, to ignore any nulls we needs to set ``ignorenulls`` to `True` + To ignore any null values, set ``ignorenulls`` to `True` >>> df.groupby("name").agg(first("age", ignorenulls=True)).orderBy("name").show() +-----+----------+ @@ -5767,7 +5767,7 @@ def last(col: "ColumnOrName", ignorenulls: bool = False) -> Column: | Bob| 5| +-----+---------+ - Now, to ignore any nulls we needs to set ``ignorenulls`` to `True` + To ignore any null values, set ``ignorenulls`` to `True` >>> df.groupby("name").agg(last("age", ignorenulls=True)).orderBy("name").show() +-----+---------+ @@ -6529,7 +6529,7 @@ def greatest(*cols: "ColumnOrName") -> Column: Parameters ---------- - col : :class:`~pyspark.sql.Column` or str + cols: :class:`~pyspark.sql.Column` or str columns to check for greatest value. Returns @@ -7086,7 +7086,7 @@ def any_value(col: "ColumnOrName", ignoreNulls: Optional[Union[bool, Column]] = ---------- col : :class:`~pyspark.sql.Column` or str target column to work on. - ignorenulls : :class:`~pyspark.sql.Column` or bool, optional + ignoreNulls : :class:`~pyspark.sql.Column` or bool, optional if first value is null then look for first non-null value. Returns @@ -12148,7 +12148,7 @@ def substr( Parameters ---------- - src : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or str A column of string. pos : :class:`~pyspark.sql.Column` or str A column of string, the substring of `str` that starts at `pos`. @@ -12922,7 +12922,7 @@ def like( When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, it falls back to Spark 1.6 behavior regarding string literal parsing. For example, if the config is enabled, the pattern to match "\abc" should be "\abc". - escape : :class:`~pyspark.sql.Column`, optional + escapeChar : :class:`~pyspark.sql.Column`, optional An character added since Spark 3.0. The default escape character is the '\'. If an escape character precedes a special symbol or another escape character, the following character is matched literally. It is invalid to escape any other character. @@ -12972,7 +12972,7 @@ def ilike( When SQL config 'spark.sql.parser.escapedStringLiterals' is enabled, it falls back to Spark 1.6 behavior regarding string literal parsing. For example, if the config is enabled, the pattern to match "\abc" should be "\abc". - escape : :class:`~pyspark.sql.Column`, optional + escapeChar : :class:`~pyspark.sql.Column`, optional An character added since Spark 3.0. The default escape character is the '\'. If an escape character precedes a special symbol or another escape character, the following character is matched literally. It is invalid to escape any other character. @@ -19723,6 +19723,8 @@ def bucket(numBuckets: Union[Column, int], col: "ColumnOrName") -> Column: Parameters ---------- + numBuckets : :class:`~pyspark.sql.Column` or int + the number of buckets col : :class:`~pyspark.sql.Column` or str target date or timestamp column to work on. @@ -19748,7 +19750,7 @@ def bucket(numBuckets: Union[Column, int], col: "ColumnOrName") -> Column: @_try_remote_functions def call_udf(udfName: str, *cols: "ColumnOrName") -> Column: """ - Call an user-defined function. + Call a user-defined function. .. versionadded:: 3.4.0 @@ -20133,7 +20135,7 @@ def isnotnull(col: "ColumnOrName") -> Column: def equal_null(col1: "ColumnOrName", col2: "ColumnOrName") -> Column: """ Returns same result as the EQUAL(=) operator for non-null operands, - but returns true if both are null, false if one of the them is null. + but returns true if both are null, false if one of them is null. .. versionadded:: 3.5.0 @@ -20394,7 +20396,7 @@ def aes_decrypt( Returns ------- :class:`~pyspark.sql.Column` - A new column that contains an decrypted value. + A new column that contains a decrypted value. Examples -------- @@ -20515,7 +20517,7 @@ def try_aes_decrypt( Returns ------- :class:`~pyspark.sql.Column` - A new column that contains an decrypted value or a NULL value. + A new column that contains a decrypted value or a NULL value. Examples -------- From b5f3e1e8ddc3f3deba4f4f21f916dc439f610066 Mon Sep 17 00:00:00 2001 From: wforget <643348094@qq.com> Date: Wed, 10 Jul 2024 09:09:45 +0800 Subject: [PATCH 04/64] [SPARK-48817][SQL] Eagerly execute union multi commands together ### What changes were proposed in this pull request? Eagerly execute union multi commands together. ### Why are the changes needed? MultiInsert is split to multiple sql executions, resulting in no exchange reuse. Reproduce sql: ``` create table wangzhen_t1(c1 int); create table wangzhen_t2(c1 int); create table wangzhen_t3(c1 int); insert into wangzhen_t1 values (1), (2), (3); from (select /*+ REPARTITION(3) */ c1 from wangzhen_t1) insert overwrite table wangzhen_t2 select c1 insert overwrite table wangzhen_t3 select c1; ``` In Spark 3.1, there is only one SQL execution and there is a reuse exchange. ![image](https://github.com/apache/spark/assets/17894939/5ff68392-aaa8-4e6b-8cac-1687880796b9) However, in Spark 3.5, it was split to multiple executions and there was no ReuseExchange. ![image](https://github.com/apache/spark/assets/17894939/afdb14b6-5007-4923-802d-535149974ecf) ![image](https://github.com/apache/spark/assets/17894939/0d60e8db-9da7-4906-8d07-2b622b55e6ab) ### Does this PR introduce _any_ user-facing change? yes, multi inserts will executed in one execution. ### How was this patch tested? added unit test ### Was this patch authored or co-authored using generative AI tooling? No Closes #47224 from wForget/SPARK-48817. Authored-by: wforget <643348094@qq.com> Signed-off-by: youxiduo --- .../spark/sql/execution/QueryExecution.scala | 21 ++++++++---- .../apache/spark/sql/SQLInsertTestSuite.scala | 33 ++++++++++++++++++- 2 files changed, 47 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 357484ca19df2..5c894eb7555b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.catalyst.expressions.codegen.ByteCodeStats import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, Command, CommandResult, CreateTableAsSelect, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, ReplaceTableAsSelect, ReturnAnswer, Union} import org.apache.spark.sql.catalyst.rules.{PlanChangeLogger, Rule} import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat import org.apache.spark.sql.catalyst.util.truncatedString @@ -110,15 +110,17 @@ class QueryExecution( case _ => "command" } - private def eagerlyExecuteCommands(p: LogicalPlan) = p transformDown { - case c: Command => + private def eagerlyExecuteCommands(p: LogicalPlan) = { + def eagerlyExecute( + p: LogicalPlan, + name: String, + mode: CommandExecutionMode.Value): LogicalPlan = { // Since Command execution will eagerly take place here, // and in most cases be the bulk of time and effort, // with the rest of processing of the root plan being just outputting command results, // for eagerly executed commands we mark this place as beginning of execution. tracker.setReadyForExecution() - val qe = sparkSession.sessionState.executePlan(c, CommandExecutionMode.NON_ROOT) - val name = commandExecutionName(c) + val qe = sparkSession.sessionState.executePlan(p, mode) val result = QueryExecution.withInternalError(s"Eagerly executed $name failed.") { SQLExecution.withNewExecutionId(qe, Some(name)) { qe.executedPlan.executeCollect() @@ -129,7 +131,14 @@ class QueryExecution( qe.commandExecuted, qe.executedPlan, result.toImmutableArraySeq) - case other => other + } + p transformDown { + case u @ Union(children, _, _) if children.forall(_.isInstanceOf[Command]) => + eagerlyExecute(u, "multi-commands", CommandExecutionMode.SKIP) + case c: Command => + val name = commandExecutionName(c) + eagerlyExecute(c, name, CommandExecutionMode.NON_ROOT) + } } // The plan that has been normalized by custom rules, so that it's more likely to hit cache. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala index 05b29badceeee..63ed26bdeddf1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLInsertTestSuite.scala @@ -21,6 +21,9 @@ import org.apache.spark.{SparkConf, SparkNumberFormatException, SparkThrowable} import org.apache.spark.sql.catalyst.expressions.Hex import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.connector.catalog.InMemoryPartitionTableCatalog +import org.apache.spark.sql.execution.CommandResultExec +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.unsafe.types.UTF8String @@ -28,7 +31,7 @@ import org.apache.spark.unsafe.types.UTF8String /** * The base trait for SQL INSERT. */ -trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { +trait SQLInsertTestSuite extends QueryTest with SQLTestUtils with AdaptiveSparkPlanHelper { import testImplicits._ @@ -519,6 +522,34 @@ trait SQLInsertTestSuite extends QueryTest with SQLTestUtils { } } } + + test("SPARK-48817: test multi inserts") { + withTable("t1", "t2", "t3") { + createTable("t1", Seq("i"), Seq("int")) + createTable("t2", Seq("i"), Seq("int")) + createTable("t3", Seq("i"), Seq("int")) + sql(s"INSERT INTO t1 VALUES (1), (2), (3)") + val df = sql( + """ + |FROM (select /*+ REPARTITION(3) */ i from t1) + |INSERT INTO t2 SELECT i + |INSERT INTO t3 SELECT i + |""".stripMargin + ) + checkAnswer(spark.table("t2"), Seq(Row(1), Row(2), Row(3))) + checkAnswer(spark.table("t3"), Seq(Row(1), Row(2), Row(3))) + + val commandResults = df.queryExecution.executedPlan.collect { + case c: CommandResultExec => c + } + assert(commandResults.size == 1) + + val reusedExchanges = collect(commandResults.head.commandPhysicalPlan) { + case r: ReusedExchangeExec => r + } + assert(reusedExchanges.size == 1) + } + } } class FileSourceSQLInsertTestSuite extends SQLInsertTestSuite with SharedSparkSession { From f76bc3a843f4216588a29ff14cafdd870fd0254c Mon Sep 17 00:00:00 2001 From: "thomas.hart" Date: Wed, 10 Jul 2024 12:14:05 +0800 Subject: [PATCH 05/64] [SPARK-48822][DOCS] Add examples section header to `format_number` docstring ### What changes were proposed in this pull request? This PR adds and "Examples" section header to `format_number` docstring. ### Why are the changes needed? To improve the documentation. ### Does this PR introduce any user-facing change? No changes in behavior are introduced. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47237 from thomhart31/docs-format_number. Lead-authored-by: thomas.hart Co-authored-by: Thomas Hart Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 0ff830e8d48d6..6fd8fdfec8ea9 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -10654,6 +10654,8 @@ def format_number(col: "ColumnOrName", d: int) -> Column: :class:`~pyspark.sql.Column` the column of formatted results. + Examples + -------- >>> spark.createDataFrame([(5,)], ['a']).select(format_number('a', 4).alias('v')).collect() [Row(v='5.0000')] """ From 7600c97ecb33f2d2681534c2cc6145ab39414cd6 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Wed, 10 Jul 2024 12:49:29 +0800 Subject: [PATCH 06/64] [SPARK-36680][SQL] Supports Dynamic Table Options for Spark SQL ### What changes were proposed in this pull request? in Spark SQL, add 'WITH OPTIONS' syntax to support dynamic relation options. This is a continuation of https://github.com/apache/spark/pull/41683 based on cloud-fan's nice suggestion. That was itself a continuation of https://github.com/apache/spark/pull/34072. ### Why are the changes needed? This will allow Spark SQL to have equivalence to DataFrameReader API. For example, it is possible to specify options today to DataSources as follows via the API: ``` spark.read.format("jdbc").option("fetchSize", 0).load() ``` This pr allows an equivalent Spark SQL syntax to specify options: ``` SELECT * FROM jdbcTable WITH OPTIONS(`fetchSize` = 0) ``` ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Unit test in DataSourceV2SQLSuite ### Was this patch authored or co-authored using generative AI tooling? No Closes #46707 from szehon-ho/spark-36680. Authored-by: Szehon Ho Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 6 +++- .../sql/catalyst/parser/AstBuilder.scala | 14 ++++++--- .../sql/connector/DataSourceV2SQLSuite.scala | 29 +++++++++++++++++++ 3 files changed, 44 insertions(+), 5 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 2f5bf8bbfec14..3f6f2772f4961 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -855,13 +855,17 @@ identifierComment relationPrimary : identifierReference temporalClause? - sample? tableAlias #tableName + optionsClause? sample? tableAlias #tableName | LEFT_PAREN query RIGHT_PAREN sample? tableAlias #aliasedQuery | LEFT_PAREN relation RIGHT_PAREN sample? tableAlias #aliasedRelation | inlineTable #inlineTableDefault2 | functionTable #tableValuedFunction ; +optionsClause + : WITH options=propertyList + ; + inlineTable : VALUES expression (COMMA expression)* tableAlias ; 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 7f93e993c6fa8..8e80b9a497c8d 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 @@ -53,6 +53,7 @@ import org.apache.spark.sql.errors.DataTypeErrors.toSQLStmt import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.LEGACY_BANG_EQUALS_NOT import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.random.RandomSampler @@ -1611,7 +1612,7 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { * Create an aliased table reference. This is typically used in FROM clauses. */ override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { - val relation = createUnresolvedRelation(ctx.identifierReference) + val relation = createUnresolvedRelation(ctx.identifierReference, Option(ctx.optionsClause)) val table = mayApplyAliasPlan( ctx.tableAlias, relation.optionalMap(ctx.temporalClause)(withTimeTravel)) table.optionalMap(ctx.sample)(withSample) @@ -3004,11 +3005,16 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { } /** - * Create an [[UnresolvedRelation]] from an identifier reference. + * Create an [[UnresolvedRelation]] from an identifier reference and an options clause. */ private def createUnresolvedRelation( - ctx: IdentifierReferenceContext): LogicalPlan = withOrigin(ctx) { - withIdentClause(ctx, UnresolvedRelation(_)) + ctx: IdentifierReferenceContext, + optionsClause: Option[OptionsClauseContext] = None): LogicalPlan = withOrigin(ctx) { + val options = optionsClause.map{ clause => + new CaseInsensitiveStringMap(visitPropertyKeyValues(clause.options).asJava) + }.getOrElse(CaseInsensitiveStringMap.empty) + withIdentClause(ctx, parts => + new UnresolvedRelation(parts, options, isStreaming = 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 0382efaf9d7e5..e776de7e8222c 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 @@ -3518,6 +3518,35 @@ class DataSourceV2SQLSuiteV1Filter } } + test("SPARK-36680: Supports Dynamic Table Options for Spark SQL") { + val t1 = s"${catalogAndNamespace}table" + withTable(t1) { + sql(s"CREATE TABLE $t1 (id bigint, data string) USING $v2Format") + sql(s"INSERT INTO $t1 VALUES (1, 'a'), (2, 'b')") + + var df = sql(s"SELECT * FROM $t1") + var collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.isEmpty) + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + df = sql(s"SELECT * FROM $t1 WITH (`split-size` = 5)") + collected = df.queryExecution.optimizedPlan.collect { + case scan: DataSourceV2ScanRelation => + assert(scan.relation.options.get("split-size") == "5") + } + assert (collected.size == 1) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + + val noValues = intercept[AnalysisException]( + sql(s"SELECT * FROM $t1 WITH (`split-size`)")) + assert(noValues.message.contains( + "Operation not allowed: Values must be specified for key(s): [split-size]")) + } + } + private def testNotSupportedV2Command( sqlCommand: String, sqlParams: String, From a3c1e306a4b3568761cbe01b0e12648547367798 Mon Sep 17 00:00:00 2001 From: Yuchen Liu Date: Wed, 10 Jul 2024 16:17:31 +0900 Subject: [PATCH 07/64] [SPARK-48772][SS][SQL] State Data Source Change Feed Reader Mode ### What changes were proposed in this pull request? This PR adds ability of showing the evolution of state as Change Data Capture (CDC) format to state data source. An example usage: ``` .format("statestore") .option("readChangeFeed", true) .option("changeStartBatchId", 5) #required .option("changeEndBatchId", 10) #not required, default: latest batch Id available ``` _Note that this mode does not support the option "joinSide"._ ### Why are the changes needed? The current state reader can only return the entire state at a specific version. If an error occurs related to state, knowing the change of state across versions to find out at which version state starts to go wrong is important for debugging purposes. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Adds a new test suite `StateDataSourceChangeDataReadSuite` that includes 1) testing input error 2) testing new API added 3) integration test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47188 from eason-yuchen-liu/readStateChange. Lead-authored-by: Yuchen Liu Co-authored-by: Yuchen Liu <170372783+eason-yuchen-liu@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../resources/error/error-conditions.json | 2 +- .../v2/state/StateDataSource.scala | 165 ++++++--- .../v2/state/StatePartitionReader.scala | 123 +++++-- .../v2/state/StateScanBuilder.scala | 30 +- .../datasources/v2/state/StateTable.scala | 42 ++- ...StreamStreamJoinStatePartitionReader.scala | 3 +- .../state/HDFSBackedStateStoreProvider.scala | 43 +++ .../execution/streaming/state/RocksDB.scala | 2 +- .../state/RocksDBStateStoreProvider.scala | 50 +++ .../streaming/state/StateStore.scala | 24 +- .../streaming/state/StateStoreChangelog.scala | 109 +++++- .../StateDataSourceChangeDataReadSuite.scala | 321 ++++++++++++++++++ .../v2/state/StateDataSourceTestBase.scala | 2 +- 13 files changed, 812 insertions(+), 104 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 06f8d3a782525..efb054ef95d7a 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3835,7 +3835,7 @@ "STATE_STORE_PROVIDER_DOES_NOT_SUPPORT_FINE_GRAINED_STATE_REPLAY" : { "message" : [ "The given State Store Provider does not extend org.apache.spark.sql.execution.streaming.state.SupportsFineGrainedReplay.", - "Therefore, it does not support option snapshotStartBatchId in state data source." + "Therefore, it does not support option snapshotStartBatchId or readChangeFeed in state data source." ], "sqlState" : "42K06" }, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala index 213573a756bc9..e2c5499fe439d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSource.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.execution.streaming.StreamingCheckpointConstants.{DI import org.apache.spark.sql.execution.streaming.StreamingSymmetricHashJoinHelper.{LeftSide, RightSide} import org.apache.spark.sql.execution.streaming.state.{StateSchemaCompatibilityChecker, StateStore, StateStoreConf, StateStoreId, StateStoreProviderId} import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -94,10 +94,20 @@ class StateDataSource extends TableProvider with DataSourceRegister { manager.readSchemaFile() } - new StructType() - .add("key", keySchema) - .add("value", valueSchema) - .add("partition_id", IntegerType) + if (sourceOptions.readChangeFeed) { + new StructType() + .add("batch_id", LongType) + .add("change_type", StringType) + .add("key", keySchema) + .add("value", valueSchema) + .add("partition_id", IntegerType) + } else { + new StructType() + .add("key", keySchema) + .add("value", valueSchema) + .add("partition_id", IntegerType) + } + } catch { case NonFatal(e) => throw StateDataSourceErrors.failedToReadStateSchema(sourceOptions, e) @@ -125,21 +135,38 @@ class StateDataSource extends TableProvider with DataSourceRegister { override def supportsExternalMetadata(): Boolean = false } +case class FromSnapshotOptions( + snapshotStartBatchId: Long, + snapshotPartitionId: Int) + +case class ReadChangeFeedOptions( + changeStartBatchId: Long, + changeEndBatchId: Long +) + case class StateSourceOptions( resolvedCpLocation: String, batchId: Long, operatorId: Int, storeName: String, joinSide: JoinSideValues, - snapshotStartBatchId: Option[Long], - snapshotPartitionId: Option[Int]) { + readChangeFeed: Boolean, + fromSnapshotOptions: Option[FromSnapshotOptions], + readChangeFeedOptions: Option[ReadChangeFeedOptions]) { def stateCheckpointLocation: Path = new Path(resolvedCpLocation, DIR_NAME_STATE) override def toString: String = { - s"StateSourceOptions(checkpointLocation=$resolvedCpLocation, batchId=$batchId, " + - s"operatorId=$operatorId, storeName=$storeName, joinSide=$joinSide, " + - s"snapshotStartBatchId=${snapshotStartBatchId.getOrElse("None")}, " + - s"snapshotPartitionId=${snapshotPartitionId.getOrElse("None")})" + var desc = s"StateSourceOptions(checkpointLocation=$resolvedCpLocation, batchId=$batchId, " + + s"operatorId=$operatorId, storeName=$storeName, joinSide=$joinSide" + if (fromSnapshotOptions.isDefined) { + desc += s", snapshotStartBatchId=${fromSnapshotOptions.get.snapshotStartBatchId}" + desc += s", snapshotPartitionId=${fromSnapshotOptions.get.snapshotPartitionId}" + } + if (readChangeFeedOptions.isDefined) { + desc += s", changeStartBatchId=${readChangeFeedOptions.get.changeStartBatchId}" + desc += s", changeEndBatchId=${readChangeFeedOptions.get.changeEndBatchId}" + } + desc + ")" } } @@ -151,6 +178,9 @@ object StateSourceOptions extends DataSourceOptions { val JOIN_SIDE = newOption("joinSide") val SNAPSHOT_START_BATCH_ID = newOption("snapshotStartBatchId") val SNAPSHOT_PARTITION_ID = newOption("snapshotPartitionId") + val READ_CHANGE_FEED = newOption("readChangeFeed") + val CHANGE_START_BATCH_ID = newOption("changeStartBatchId") + val CHANGE_END_BATCH_ID = newOption("changeEndBatchId") object JoinSideValues extends Enumeration { type JoinSideValues = Value @@ -172,16 +202,6 @@ object StateSourceOptions extends DataSourceOptions { throw StateDataSourceErrors.requiredOptionUnspecified(PATH) }.get - val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation) - - val batchId = Option(options.get(BATCH_ID)).map(_.toLong).orElse { - Some(getLastCommittedBatch(sparkSession, resolvedCpLocation)) - }.get - - if (batchId < 0) { - throw StateDataSourceErrors.invalidOptionValueIsNegative(BATCH_ID) - } - val operatorId = Option(options.get(OPERATOR_ID)).map(_.toInt) .orElse(Some(0)).get @@ -210,30 +230,97 @@ object StateSourceOptions extends DataSourceOptions { throw StateDataSourceErrors.conflictOptions(Seq(JOIN_SIDE, STORE_NAME)) } - val snapshotStartBatchId = Option(options.get(SNAPSHOT_START_BATCH_ID)).map(_.toLong) - if (snapshotStartBatchId.exists(_ < 0)) { - throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_START_BATCH_ID) - } else if (snapshotStartBatchId.exists(_ > batchId)) { - throw StateDataSourceErrors.invalidOptionValue( - SNAPSHOT_START_BATCH_ID, s"value should be less than or equal to $batchId") - } + val resolvedCpLocation = resolvedCheckpointLocation(hadoopConf, checkpointLocation) + + var batchId = Option(options.get(BATCH_ID)).map(_.toLong) + val snapshotStartBatchId = Option(options.get(SNAPSHOT_START_BATCH_ID)).map(_.toLong) val snapshotPartitionId = Option(options.get(SNAPSHOT_PARTITION_ID)).map(_.toInt) - if (snapshotPartitionId.exists(_ < 0)) { - throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_PARTITION_ID) - } - // both snapshotPartitionId and snapshotStartBatchId are required at the same time, because - // each partition may have different checkpoint status - if (snapshotPartitionId.isDefined && snapshotStartBatchId.isEmpty) { - throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_START_BATCH_ID) - } else if (snapshotPartitionId.isEmpty && snapshotStartBatchId.isDefined) { - throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_PARTITION_ID) + val readChangeFeed = Option(options.get(READ_CHANGE_FEED)).exists(_.toBoolean) + + val changeStartBatchId = Option(options.get(CHANGE_START_BATCH_ID)).map(_.toLong) + var changeEndBatchId = Option(options.get(CHANGE_END_BATCH_ID)).map(_.toLong) + + var fromSnapshotOptions: Option[FromSnapshotOptions] = None + var readChangeFeedOptions: Option[ReadChangeFeedOptions] = None + + if (readChangeFeed) { + if (joinSide != JoinSideValues.none) { + throw StateDataSourceErrors.conflictOptions(Seq(JOIN_SIDE, READ_CHANGE_FEED)) + } + if (batchId.isDefined) { + throw StateDataSourceErrors.conflictOptions(Seq(BATCH_ID, READ_CHANGE_FEED)) + } + if (snapshotStartBatchId.isDefined) { + throw StateDataSourceErrors.conflictOptions(Seq(SNAPSHOT_START_BATCH_ID, READ_CHANGE_FEED)) + } + if (snapshotPartitionId.isDefined) { + throw StateDataSourceErrors.conflictOptions(Seq(SNAPSHOT_PARTITION_ID, READ_CHANGE_FEED)) + } + + if (changeStartBatchId.isEmpty) { + throw StateDataSourceErrors.requiredOptionUnspecified(CHANGE_START_BATCH_ID) + } + changeEndBatchId = Some( + changeEndBatchId.getOrElse(getLastCommittedBatch(sparkSession, resolvedCpLocation))) + + // changeStartBatchId and changeEndBatchId must all be defined at this point + if (changeStartBatchId.get < 0) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(CHANGE_START_BATCH_ID) + } + if (changeEndBatchId.get < changeStartBatchId.get) { + throw StateDataSourceErrors.invalidOptionValue(CHANGE_END_BATCH_ID, + s"$CHANGE_END_BATCH_ID cannot be smaller than $CHANGE_START_BATCH_ID. " + + s"Please check the input to $CHANGE_END_BATCH_ID, or if you are using its default " + + s"value, make sure that $CHANGE_START_BATCH_ID is less than ${changeEndBatchId.get}.") + } + + batchId = Some(changeEndBatchId.get) + + readChangeFeedOptions = Option( + ReadChangeFeedOptions(changeStartBatchId.get, changeEndBatchId.get)) + } else { + if (changeStartBatchId.isDefined) { + throw StateDataSourceErrors.invalidOptionValue(CHANGE_START_BATCH_ID, + s"Only specify this option when $READ_CHANGE_FEED is set to true.") + } + if (changeEndBatchId.isDefined) { + throw StateDataSourceErrors.invalidOptionValue(CHANGE_END_BATCH_ID, + s"Only specify this option when $READ_CHANGE_FEED is set to true.") + } + + batchId = Some(batchId.getOrElse(getLastCommittedBatch(sparkSession, resolvedCpLocation))) + + if (batchId.get < 0) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(BATCH_ID) + } + if (snapshotStartBatchId.exists(_ < 0)) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_START_BATCH_ID) + } else if (snapshotStartBatchId.exists(_ > batchId.get)) { + throw StateDataSourceErrors.invalidOptionValue( + SNAPSHOT_START_BATCH_ID, s"value should be less than or equal to ${batchId.get}") + } + if (snapshotPartitionId.exists(_ < 0)) { + throw StateDataSourceErrors.invalidOptionValueIsNegative(SNAPSHOT_PARTITION_ID) + } + // both snapshotPartitionId and snapshotStartBatchId are required at the same time, because + // each partition may have different checkpoint status + if (snapshotPartitionId.isDefined && snapshotStartBatchId.isEmpty) { + throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_START_BATCH_ID) + } else if (snapshotPartitionId.isEmpty && snapshotStartBatchId.isDefined) { + throw StateDataSourceErrors.requiredOptionUnspecified(SNAPSHOT_PARTITION_ID) + } + + if (snapshotStartBatchId.isDefined && snapshotPartitionId.isDefined) { + fromSnapshotOptions = Some( + FromSnapshotOptions(snapshotStartBatchId.get, snapshotPartitionId.get)) + } } StateSourceOptions( - resolvedCpLocation, batchId, operatorId, storeName, - joinSide, snapshotStartBatchId, snapshotPartitionId) + resolvedCpLocation, batchId.get, operatorId, storeName, joinSide, + readChangeFeed, fromSnapshotOptions, readChangeFeedOptions) } private def resolvedCheckpointLocation( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala index 8461603e96525..6201cf1157ab3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala @@ -23,8 +23,10 @@ import org.apache.spark.sql.connector.read.{InputPartition, PartitionReader, Par import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataTableEntry import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil import org.apache.spark.sql.execution.streaming.state._ +import org.apache.spark.sql.execution.streaming.state.RecordType.{getRecordTypeAsString, RecordType} import org.apache.spark.sql.types.StructType -import org.apache.spark.util.SerializableConfiguration +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.{NextIterator, SerializableConfiguration} /** * An implementation of [[PartitionReaderFactory]] for State data source. This is used to support @@ -37,8 +39,14 @@ class StatePartitionReaderFactory( stateStoreMetadata: Array[StateMetadataTableEntry]) extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - new StatePartitionReader(storeConf, hadoopConf, - partition.asInstanceOf[StateStoreInputPartition], schema, stateStoreMetadata) + val stateStoreInputPartition = partition.asInstanceOf[StateStoreInputPartition] + if (stateStoreInputPartition.sourceOptions.readChangeFeed) { + new StateStoreChangeDataPartitionReader(storeConf, hadoopConf, + stateStoreInputPartition, schema, stateStoreMetadata) + } else { + new StatePartitionReader(storeConf, hadoopConf, + stateStoreInputPartition, schema, stateStoreMetadata) + } } } @@ -46,18 +54,17 @@ class StatePartitionReaderFactory( * An implementation of [[PartitionReader]] for State data source. This is used to support * general read from a state store instance, rather than specific to the operator. */ -class StatePartitionReader( +abstract class StatePartitionReaderBase( storeConf: StateStoreConf, hadoopConf: SerializableConfiguration, partition: StateStoreInputPartition, schema: StructType, stateStoreMetadata: Array[StateMetadataTableEntry]) extends PartitionReader[InternalRow] with Logging { - private val keySchema = SchemaUtil.getSchemaAsDataType(schema, "key").asInstanceOf[StructType] private val valueSchema = SchemaUtil.getSchemaAsDataType(schema, "value").asInstanceOf[StructType] - private lazy val provider: StateStoreProvider = { + protected lazy val provider: StateStoreProvider = { val stateStoreId = StateStoreId(partition.sourceOptions.stateCheckpointLocation.toString, partition.sourceOptions.operatorId, partition.partition, partition.sourceOptions.storeName) val stateStoreProviderId = StateStoreProviderId(stateStoreId, partition.queryId) @@ -88,25 +95,7 @@ class StatePartitionReader( useMultipleValuesPerKey = false) } - private lazy val store: ReadStateStore = { - partition.sourceOptions.snapshotStartBatchId match { - case None => provider.getReadStore(partition.sourceOptions.batchId + 1) - - case Some(snapshotStartBatchId) => - if (!provider.isInstanceOf[SupportsFineGrainedReplay]) { - throw StateStoreErrors.stateStoreProviderDoesNotSupportFineGrainedReplay( - provider.getClass.toString) - } - provider.asInstanceOf[SupportsFineGrainedReplay] - .replayReadStateFromSnapshot( - snapshotStartBatchId + 1, - partition.sourceOptions.batchId + 1) - } - } - - private lazy val iter: Iterator[InternalRow] = { - store.iterator().map(pair => unifyStateRowPair((pair.key, pair.value))) - } + protected val iter: Iterator[InternalRow] private var current: InternalRow = _ @@ -124,9 +113,46 @@ class StatePartitionReader( override def close(): Unit = { current = null - store.abort() provider.close() } +} + +/** + * An implementation of [[StatePartitionReaderBase]] for the normal mode of State Data + * Source. It reads the the state at a particular batchId. + */ +class StatePartitionReader( + storeConf: StateStoreConf, + hadoopConf: SerializableConfiguration, + partition: StateStoreInputPartition, + schema: StructType, + stateStoreMetadata: Array[StateMetadataTableEntry]) + extends StatePartitionReaderBase(storeConf, hadoopConf, partition, schema, stateStoreMetadata) { + + private lazy val store: ReadStateStore = { + partition.sourceOptions.fromSnapshotOptions match { + case None => provider.getReadStore(partition.sourceOptions.batchId + 1) + + case Some(fromSnapshotOptions) => + if (!provider.isInstanceOf[SupportsFineGrainedReplay]) { + throw StateStoreErrors.stateStoreProviderDoesNotSupportFineGrainedReplay( + provider.getClass.toString) + } + provider.asInstanceOf[SupportsFineGrainedReplay] + .replayReadStateFromSnapshot( + fromSnapshotOptions.snapshotStartBatchId + 1, + partition.sourceOptions.batchId + 1) + } + } + + override lazy val iter: Iterator[InternalRow] = { + store.iterator().map(pair => unifyStateRowPair((pair.key, pair.value))) + } + + override def close(): Unit = { + store.abort() + super.close() + } private def unifyStateRowPair(pair: (UnsafeRow, UnsafeRow)): InternalRow = { val row = new GenericInternalRow(3) @@ -136,3 +162,48 @@ class StatePartitionReader( row } } + +/** + * An implementation of [[StatePartitionReaderBase]] for the readChangeFeed mode of State Data + * Source. It reads the change of state over batches of a particular partition. + */ +class StateStoreChangeDataPartitionReader( + storeConf: StateStoreConf, + hadoopConf: SerializableConfiguration, + partition: StateStoreInputPartition, + schema: StructType, + stateStoreMetadata: Array[StateMetadataTableEntry]) + extends StatePartitionReaderBase(storeConf, hadoopConf, partition, schema, stateStoreMetadata) { + + private lazy val changeDataReader: + NextIterator[(RecordType.Value, UnsafeRow, UnsafeRow, Long)] = { + if (!provider.isInstanceOf[SupportsFineGrainedReplay]) { + throw StateStoreErrors.stateStoreProviderDoesNotSupportFineGrainedReplay( + provider.getClass.toString) + } + provider.asInstanceOf[SupportsFineGrainedReplay] + .getStateStoreChangeDataReader( + partition.sourceOptions.readChangeFeedOptions.get.changeStartBatchId + 1, + partition.sourceOptions.readChangeFeedOptions.get.changeEndBatchId + 1) + } + + override lazy val iter: Iterator[InternalRow] = { + changeDataReader.iterator.map(unifyStateChangeDataRow) + } + + override def close(): Unit = { + changeDataReader.closeIfNeeded() + super.close() + } + + private def unifyStateChangeDataRow(row: (RecordType, UnsafeRow, UnsafeRow, Long)): + InternalRow = { + val result = new GenericInternalRow(5) + result.update(0, row._4) + result.update(1, UTF8String.fromString(getRecordTypeAsString(row._1))) + result.update(2, row._2) + result.update(3, row._3) + result.update(4, partition.partition) + result + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala index 821a36977fed1..01f966ae948ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateScanBuilder.scala @@ -86,17 +86,18 @@ class StateScan( assert((tail - head + 1) == partitionNums.length, s"No continuous partitions in state: ${partitionNums.mkString("Array(", ", ", ")")}") - sourceOptions.snapshotPartitionId match { + sourceOptions.fromSnapshotOptions match { case None => partitionNums.map { pn => new StateStoreInputPartition(pn, queryId, sourceOptions) }.toArray - case Some(snapshotPartitionId) => - if (partitionNums.contains(snapshotPartitionId)) { - Array(new StateStoreInputPartition(snapshotPartitionId, queryId, sourceOptions)) + case Some(fromSnapshotOptions) => + if (partitionNums.contains(fromSnapshotOptions.snapshotPartitionId)) { + Array(new StateStoreInputPartition( + fromSnapshotOptions.snapshotPartitionId, queryId, sourceOptions)) } else { throw StateStoreErrors.stateStoreSnapshotPartitionNotFound( - snapshotPartitionId, sourceOptions.operatorId, + fromSnapshotOptions.snapshotPartitionId, sourceOptions.operatorId, sourceOptions.stateCheckpointLocation.toString) } } @@ -128,16 +129,27 @@ class StateScan( override def toBatch: Batch = this override def description(): String = { - val desc = s"StateScan " + + var desc = s"StateScan " + s"[stateCkptLocation=${sourceOptions.stateCheckpointLocation}]" + s"[batchId=${sourceOptions.batchId}][operatorId=${sourceOptions.operatorId}]" + s"[storeName=${sourceOptions.storeName}]" if (sourceOptions.joinSide != JoinSideValues.none) { - desc + s"[joinSide=${sourceOptions.joinSide}]" - } else { - desc + desc += s"[joinSide=${sourceOptions.joinSide}]" + } + sourceOptions.fromSnapshotOptions match { + case Some(fromSnapshotOptions) => + desc += s"[snapshotStartBatchId=${fromSnapshotOptions.snapshotStartBatchId}]" + desc += s"[snapshotPartitionId=${fromSnapshotOptions.snapshotPartitionId}]" + case _ => + } + sourceOptions.readChangeFeedOptions match { + case Some(fromSnapshotOptions) => + desc += s"[changeStartBatchId=${fromSnapshotOptions.changeStartBatchId}" + desc += s"[changeEndBatchId=${fromSnapshotOptions.changeEndBatchId}" + case _ => } + desc } private def stateCheckpointPartitionsLocation: Path = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala index 2d2c9631e5370..2fc85cd8aa968 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.execution.datasources.v2.state.StateSourceOptions.Jo import org.apache.spark.sql.execution.datasources.v2.state.metadata.StateMetadataTableEntry import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil import org.apache.spark.sql.execution.streaming.state.StateStoreConf -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ArrayImplicits._ @@ -59,11 +59,17 @@ class StateTable( if (sourceOptions.joinSide != JoinSideValues.none) { desc += s"[joinSide=${sourceOptions.joinSide}]" } - if (sourceOptions.snapshotStartBatchId.isDefined) { - desc += s"[snapshotStartBatchId=${sourceOptions.snapshotStartBatchId}]" + sourceOptions.fromSnapshotOptions match { + case Some(fromSnapshotOptions) => + desc += s"[snapshotStartBatchId=${fromSnapshotOptions.snapshotStartBatchId}]" + desc += s"[snapshotPartitionId=${fromSnapshotOptions.snapshotPartitionId}]" + case _ => } - if (sourceOptions.snapshotPartitionId.isDefined) { - desc += s"[snapshotPartitionId=${sourceOptions.snapshotPartitionId}]" + sourceOptions.readChangeFeedOptions match { + case Some(fromSnapshotOptions) => + desc += s"[changeStartBatchId=${fromSnapshotOptions.changeStartBatchId}" + desc += s"[changeEndBatchId=${fromSnapshotOptions.changeEndBatchId}" + case _ => } desc } @@ -76,16 +82,26 @@ class StateTable( override def properties(): util.Map[String, String] = Map.empty[String, String].asJava private def isValidSchema(schema: StructType): Boolean = { - if (schema.fieldNames.toImmutableArraySeq != Seq("key", "value", "partition_id")) { - false - } else if (!SchemaUtil.getSchemaAsDataType(schema, "key").isInstanceOf[StructType]) { - false - } else if (!SchemaUtil.getSchemaAsDataType(schema, "value").isInstanceOf[StructType]) { - false - } else if (!SchemaUtil.getSchemaAsDataType(schema, "partition_id").isInstanceOf[IntegerType]) { + val expectedFieldNames = + if (sourceOptions.readChangeFeed) { + Seq("batch_id", "change_type", "key", "value", "partition_id") + } else { + Seq("key", "value", "partition_id") + } + val expectedTypes = Map( + "batch_id" -> classOf[LongType], + "change_type" -> classOf[StringType], + "key" -> classOf[StructType], + "value" -> classOf[StructType], + "partition_id" -> classOf[IntegerType]) + + if (schema.fieldNames.toImmutableArraySeq != expectedFieldNames) { false } else { - true + schema.fieldNames.forall { fieldName => + expectedTypes(fieldName).isAssignableFrom( + SchemaUtil.getSchemaAsDataType(schema, fieldName).getClass) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala index 91f42db46dfb0..673ec3414c237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StreamStreamJoinStatePartitionReader.scala @@ -117,7 +117,8 @@ class StreamStreamJoinStatePartitionReader( formatVersion, skippedNullValueCount = None, useStateStoreCoordinator = false, - snapshotStartVersion = partition.sourceOptions.snapshotStartBatchId.map(_ + 1) + snapshotStartVersion = + partition.sourceOptions.fromSnapshotOptions.map(_.snapshotStartBatchId + 1) ) } 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 c4a41ceb4caf4..2ec36166f9f22 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 @@ -978,4 +978,47 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with result } + + override def getStateStoreChangeDataReader(startVersion: Long, endVersion: Long): + StateStoreChangeDataReader = { + new HDFSBackedStateStoreChangeDataReader(fm, baseDir, startVersion, endVersion, + CompressionCodec.createCodec(sparkConf, storeConf.compressionCodec), + keySchema, valueSchema) + } +} + +/** [[StateStoreChangeDataReader]] implementation for [[HDFSBackedStateStoreProvider]] */ +class HDFSBackedStateStoreChangeDataReader( + fm: CheckpointFileManager, + stateLocation: Path, + startVersion: Long, + endVersion: Long, + compressionCodec: CompressionCodec, + keySchema: StructType, + valueSchema: StructType) + extends StateStoreChangeDataReader( + fm, stateLocation, startVersion, endVersion, compressionCodec) { + + override protected var changelogSuffix: String = "delta" + + override def getNext(): (RecordType.Value, UnsafeRow, UnsafeRow, Long) = { + val reader = currentChangelogReader() + if (reader == null) { + return null + } + val (recordType, keyArray, valueArray) = reader.next() + val keyRow = new UnsafeRow(keySchema.fields.length) + keyRow.pointTo(keyArray, keyArray.length) + if (valueArray == null) { + (recordType, keyRow, null, currentChangelogVersion - 1) + } else { + val valueRow = new UnsafeRow(valueSchema.fields.length) + // If valueSize in existing file is not multiple of 8, floor it to multiple of 8. + // This is a workaround for the following: + // Prior to Spark 2.3 mistakenly append 4 bytes to the value row in + // `RowBasedKeyValueBatch`, which gets persisted into the checkpoint data + valueRow.pointTo(valueArray, (valueArray.length / 8) * 8) + (recordType, keyRow, valueRow, currentChangelogVersion - 1) + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 15ef8832ef358..6215d1aaf4b55 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -324,7 +324,7 @@ class RocksDB( } } } finally { - if (changelogReader != null) changelogReader.close() + if (changelogReader != null) changelogReader.closeIfNeeded() } } loadedVersion = endVersion 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 497d489464488..a5a8d27116ce0 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 @@ -18,18 +18,22 @@ package org.apache.spark.sql.execution.streaming.state import java.io._ +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicInteger import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ +import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -461,6 +465,19 @@ private[sql] class RocksDBStateStoreProvider } } + override def getStateStoreChangeDataReader(startVersion: Long, endVersion: Long): + StateStoreChangeDataReader = { + val statePath = stateStoreId.storeCheckpointLocation() + val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf) + new RocksDBStateStoreChangeDataReader( + CheckpointFileManager.create(statePath, hadoopConf), + statePath, + startVersion, + endVersion, + CompressionCodec.createCodec(sparkConf, storeConf.compressionCodec), + keyValueEncoderMap) + } + /** * Class for column family related utility functions. * Verification functions for column family names, column family operation validations etc. @@ -670,3 +687,36 @@ object RocksDBStateStoreProvider { CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE, CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES) } + +/** [[StateStoreChangeDataReader]] implementation for [[RocksDBStateStoreProvider]] */ +class RocksDBStateStoreChangeDataReader( + fm: CheckpointFileManager, + stateLocation: Path, + startVersion: Long, + endVersion: Long, + compressionCodec: CompressionCodec, + keyValueEncoderMap: + ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)]) + extends StateStoreChangeDataReader( + fm, stateLocation, startVersion, endVersion, compressionCodec) { + + override protected var changelogSuffix: String = "changelog" + + override def getNext(): (RecordType.Value, UnsafeRow, UnsafeRow, Long) = { + val reader = currentChangelogReader() + if (reader == null) { + return null + } + val (recordType, keyArray, valueArray) = reader.next() + // Todo: does not support multiple virtual column families + val (rocksDBKeyStateEncoder, rocksDBValueStateEncoder) = + keyValueEncoderMap.get(StateStore.DEFAULT_COL_FAMILY_NAME) + val keyRow = rocksDBKeyStateEncoder.decodeKey(keyArray) + if (valueArray == null) { + (recordType, keyRow, null, currentChangelogVersion - 1) + } else { + val valueRow = rocksDBValueStateEncoder.decodeValue(valueArray) + (recordType, keyRow, valueRow, currentChangelogVersion - 1) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 76fd36bd726a6..0dc5414b7398a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{NextIterator, ThreadUtils, Utils} /** * Base trait for a versioned key-value store which provides read operations. Each instance of a @@ -439,9 +439,9 @@ object StateStoreProvider { } /** - * This is an optional trait to be implemented by [[StateStoreProvider]]s that can read fine - * grained state data which is replayed from a specific snapshot version. It is used by the - * snapshotStartBatchId option in state data source. + * This is an optional trait to be implemented by [[StateStoreProvider]]s that can read the change + * of state store over batches. This is used by State Data Source with additional options like + * snapshotStartBatchId or readChangeFeed. */ trait SupportsFineGrainedReplay { @@ -469,6 +469,22 @@ trait SupportsFineGrainedReplay { def replayReadStateFromSnapshot(snapshotVersion: Long, endVersion: Long): ReadStateStore = { new WrappedReadStateStore(replayStateFromSnapshot(snapshotVersion, endVersion)) } + + /** + * Return an iterator that reads all the entries of changelogs from startVersion to + * endVersion. + * Each record is represented by a tuple of (recordType: [[RecordType.Value]], key: [[UnsafeRow]], + * value: [[UnsafeRow]], batchId: [[Long]]) + * A put record is returned as a tuple(recordType, key, value, batchId) + * A delete record is return as a tuple(recordType, key, null, batchId) + * + * @param startVersion starting changelog version + * @param endVersion ending changelog version + * @return iterator that gives tuple(recordType: [[RecordType.Value]], nested key: [[UnsafeRow]], + * nested value: [[UnsafeRow]], batchId: [[Long]]) + */ + def getStateStoreChangeDataReader(startVersion: Long, endVersion: Long): + NextIterator[(RecordType.Value, UnsafeRow, UnsafeRow, Long)] } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index 04388589bb0b3..d189daa6e841b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -28,6 +28,7 @@ import org.apache.hadoop.fs.{FSError, Path} import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.io.CompressionCodec +import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.CheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream @@ -55,6 +56,15 @@ object RecordType extends Enumeration { } } + def getRecordTypeAsString(recordType: RecordType): String = { + recordType match { + case PUT_RECORD => "update" + case DELETE_RECORD => "delete" + case _ => throw StateStoreErrors.unsupportedOperationException( + "getRecordTypeAsString", recordType.toString) + } + } + // Generate record type from byte representation def getRecordTypeFromByte(byte: Byte): RecordType = { byte match { @@ -260,17 +270,17 @@ abstract class StateStoreChangelogReader( } protected val input: DataInputStream = decompressStream(sourceStream) - def close(): Unit = { if (input != null) input.close() } + override protected def close(): Unit = { if (input != null) input.close() } override def getNext(): (RecordType.Value, Array[Byte], Array[Byte]) } /** * Read an iterator of change record from the changelog file. - * A record is represented by ByteArrayPair(recordType: RecordType.Value, - * key: Array[Byte], value: Array[Byte], colFamilyName: String) - * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName) - * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName) + * A record is represented by tuple(recordType: RecordType.Value, + * key: Array[Byte], value: Array[Byte]) + * A put record is returned as a tuple(recordType, key, value) + * A delete record is return as a tuple(recordType, key, null) */ class StateStoreChangelogReaderV1( fm: CheckpointFileManager, @@ -307,10 +317,10 @@ class StateStoreChangelogReaderV1( /** * Read an iterator of change record from the changelog file. - * A record is represented by ByteArrayPair(recordType: RecordType.Value, - * key: Array[Byte], value: Array[Byte], colFamilyName: String) - * A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName) - * A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName) + * A record is represented by tuple(recordType: RecordType.Value, + * key: Array[Byte], value: Array[Byte]) + * A put record is returned as a tuple(recordType, key, value) + * A delete record is return as a tuple(recordType, key, null) */ class StateStoreChangelogReaderV2( fm: CheckpointFileManager, @@ -355,3 +365,84 @@ class StateStoreChangelogReaderV2( } } } + +/** + * Base class representing a iterator that iterates over a range of changelog files in a state + * store. In each iteration, it will return a tuple of (changeType: [[RecordType]], + * nested key: [[UnsafeRow]], nested value: [[UnsafeRow]], batchId: [[Long]]) + * + * @param fm checkpoint file manager used to manage streaming query checkpoint + * @param stateLocation location of the state store + * @param startVersion start version of the changelog file to read + * @param endVersion end version of the changelog file to read + * @param compressionCodec de-compression method using for reading changelog file + */ +abstract class StateStoreChangeDataReader( + fm: CheckpointFileManager, + stateLocation: Path, + startVersion: Long, + endVersion: Long, + compressionCodec: CompressionCodec) + extends NextIterator[(RecordType.Value, UnsafeRow, UnsafeRow, Long)] with Logging { + + assert(startVersion >= 1) + assert(endVersion >= startVersion) + + /** + * Iterator that iterates over the changelog files in the state store. + */ + private class ChangeLogFileIterator extends Iterator[Path] { + + private var currentVersion = StateStoreChangeDataReader.this.startVersion - 1 + + /** returns the version of the changelog returned by the latest [[next]] function call */ + def getVersion: Long = currentVersion + + override def hasNext: Boolean = currentVersion < StateStoreChangeDataReader.this.endVersion + + override def next(): Path = { + currentVersion += 1 + getChangelogPath(currentVersion) + } + + private def getChangelogPath(version: Long): Path = + new Path( + StateStoreChangeDataReader.this.stateLocation, + s"$version.${StateStoreChangeDataReader.this.changelogSuffix}") + } + + /** file format of the changelog files */ + protected var changelogSuffix: String + private lazy val fileIterator = new ChangeLogFileIterator + private var changelogReader: StateStoreChangelogReader = null + + /** + * Get a changelog reader that has at least one record left to read. If there is no readers left, + * return null. + */ + protected def currentChangelogReader(): StateStoreChangelogReader = { + while (changelogReader == null || !changelogReader.hasNext) { + if (changelogReader != null) { + changelogReader.closeIfNeeded() + changelogReader = null + } + if (!fileIterator.hasNext) { + finished = true + return null + } + // Todo: Does not support StateStoreChangelogReaderV2 + changelogReader = + new StateStoreChangelogReaderV1(fm, fileIterator.next(), compressionCodec) + } + changelogReader + } + + /** get the version of the current changelog reader */ + protected def currentChangelogVersion: Long = fileIterator.getVersion + + override def close(): Unit = { + if (changelogReader != null) { + changelogReader.closeIfNeeded() + } + } +} 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 new file mode 100644 index 0000000000000..2858d356d4c9a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceChangeDataReadSuite.scala @@ -0,0 +1,321 @@ +/* + * 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.v2.state + +import org.apache.hadoop.conf.Configuration +import org.scalatest.Assertions + +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.execution.streaming.state._ +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.types.StructType + +class HDFSBackedStateDataSourceChangeDataReaderSuite extends StateDataSourceChangeDataReaderSuite { + override protected def newStateStoreProvider(): HDFSBackedStateStoreProvider = + new HDFSBackedStateStoreProvider +} + +class RocksDBWithChangelogCheckpointStateDataSourceChangeDataReaderSuite extends + StateDataSourceChangeDataReaderSuite { + override protected def newStateStoreProvider(): RocksDBStateStoreProvider = + new RocksDBStateStoreProvider + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set("spark.sql.streaming.stateStore.rocksdb.changelogCheckpointing.enabled", + "true") + } +} + +abstract class StateDataSourceChangeDataReaderSuite extends StateDataSourceTestBase + with Assertions { + + import testImplicits._ + import StateStoreTestsHelper._ + + protected val keySchema: StructType = StateStoreTestsHelper.keySchema + protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema + + protected def newStateStoreProvider(): StateStoreProvider + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set(SQLConf.STREAMING_NO_DATA_MICRO_BATCHES_ENABLED, false) + spark.conf.set(SQLConf.STATE_STORE_PROVIDER_CLASS.key, newStateStoreProvider().getClass.getName) + } + + /** + * Calls the overridable [[newStateStoreProvider]] to create the state store provider instance. + * Initialize it with the configuration set by child classes. + * + * @param checkpointDir path to store state information + * @return instance of class extending [[StateStoreProvider]] + */ + private def getNewStateStoreProvider(checkpointDir: String): StateStoreProvider = { + val provider = newStateStoreProvider() + provider.init( + StateStoreId(checkpointDir, 0, 0), + keySchema, + valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema), + useColumnFamilies = false, + StateStoreConf(spark.sessionState.conf), + new Configuration) + provider + } + + test("ERROR: specify changeStartBatchId in normal mode") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceInvalidOptionValue] { + spark.read.format("statestore") + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) + .load(tempDir.getAbsolutePath) + } + assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + } + } + + test("ERROR: changeStartBatchId is set to negative") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceInvalidOptionValueIsNegative] { + spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, -1) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + } + assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.IS_NEGATIVE") + } + } + + test("ERROR: changeEndBatchId is set to less than changeStartBatchId") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceInvalidOptionValue] { + spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 1) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + } + assert(exc.getErrorClass === "STDS_INVALID_OPTION_VALUE.WITH_MESSAGE") + } + } + + test("ERROR: joinSide option is used together with readChangeFeed") { + withTempDir { tempDir => + val exc = intercept[StateDataSourceConflictOptions] { + spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.JOIN_SIDE, "left") + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 0) + .load(tempDir.getAbsolutePath) + } + assert(exc.getErrorClass === "STDS_CONFLICT_OPTIONS") + } + } + + test("getChangeDataReader of state store provider") { + def withNewStateStore(provider: StateStoreProvider, version: Int)(f: StateStore => Unit): + Unit = { + val stateStore = provider.getStore(version) + f(stateStore) + stateStore.commit() + } + + withTempDir { tempDir => + val provider = getNewStateStoreProvider(tempDir.getAbsolutePath) + withNewStateStore(provider, 0) { stateStore => + put(stateStore, "a", 1, 1) } + withNewStateStore(provider, 1) { stateStore => + put(stateStore, "b", 2, 2) } + withNewStateStore(provider, 2) { stateStore => + stateStore.remove(dataToKeyRow("a", 1)) } + withNewStateStore(provider, 3) { stateStore => + stateStore.remove(dataToKeyRow("b", 2)) } + + val reader = + provider.asInstanceOf[SupportsFineGrainedReplay].getStateStoreChangeDataReader(1, 4) + + assert(reader.next() === (RecordType.PUT_RECORD, dataToKeyRow("a", 1), dataToValueRow(1), 0L)) + assert(reader.next() === (RecordType.PUT_RECORD, dataToKeyRow("b", 2), dataToValueRow(2), 1L)) + assert(reader.next() === + (RecordType.DELETE_RECORD, dataToKeyRow("a", 1), null, 2L)) + assert(reader.next() === + (RecordType.DELETE_RECORD, dataToKeyRow("b", 2), null, 3L)) + } + } + + test("read global streaming limit state change feed") { + withTempDir { tempDir => + val inputData = MemoryStream[Int] + val df = inputData.toDF().limit(10) + testStream(df)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddData(inputData, 1, 2, 3, 4), + ProcessAllAvailable(), + AddData(inputData, 5, 6, 7, 8), + ProcessAllAvailable(), + AddData(inputData, 9, 10, 11, 12), + ProcessAllAvailable() + ) + + val stateDf = spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) + .load(tempDir.getAbsolutePath) + + val expectedDf = Seq( + Row(0L, "update", Row(null), Row(4), 0), + Row(1L, "update", Row(null), Row(8), 0), + Row(2L, "update", Row(null), Row(10), 0) + ) + + checkAnswer(stateDf, expectedDf) + } + } + + test("read streaming aggregate state change feed") { + withTempDir { tempDir => + val inputData = MemoryStream[Int] + val df = inputData.toDF().groupBy("value").count() + testStream(df, OutputMode.Update)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddData(inputData, 1, 2, 3, 4), + ProcessAllAvailable(), + AddData(inputData, 2, 3, 4, 5), + ProcessAllAvailable(), + AddData(inputData, 3, 4, 5, 6), + ProcessAllAvailable() + ) + + val stateDf = spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) + .load(tempDir.getAbsolutePath) + + val expectedDf = Seq( + Row(0L, "update", Row(3), Row(1), 1), + Row(1L, "update", Row(3), Row(2), 1), + Row(1L, "update", Row(5), Row(1), 1), + Row(2L, "update", Row(3), Row(3), 1), + Row(2L, "update", Row(5), Row(2), 1), + Row(0L, "update", Row(4), Row(1), 2), + Row(1L, "update", Row(4), Row(2), 2), + Row(2L, "update", Row(4), Row(3), 2), + Row(0L, "update", Row(1), Row(1), 3), + Row(0L, "update", Row(2), Row(1), 4), + Row(1L, "update", Row(2), Row(2), 4), + Row(2L, "update", Row(6), Row(1), 4) + ) + + checkAnswer(stateDf, expectedDf) + } + } + + test("read streaming deduplication state change feed") { + withTempDir { tempDir => + val inputData = MemoryStream[Int] + val df = inputData.toDF().dropDuplicates("value") + testStream(df, OutputMode.Update)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddData(inputData, 1, 2, 3, 4), + ProcessAllAvailable(), + AddData(inputData, 2, 3, 4, 5), + ProcessAllAvailable(), + AddData(inputData, 3, 4, 5, 6), + ProcessAllAvailable() + ) + + val stateDf = spark.read.format("statestore") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 2) + .load(tempDir.getAbsolutePath) + + val expectedDf = Seq( + Row(0L, "update", Row(1), Row(null), 3), + Row(0L, "update", Row(2), Row(null), 4), + Row(0L, "update", Row(3), Row(null), 1), + Row(0L, "update", Row(4), Row(null), 2), + Row(1L, "update", Row(5), Row(null), 1), + Row(2L, "update", Row(6), Row(null), 4) + ) + + checkAnswer(stateDf, expectedDf) + } + } + + test("read stream-stream join state change feed") { + withTempDir { tempDir => + val inputData = MemoryStream[(Int, Long)] + val leftDf = + inputData.toDF().select(col("_1").as("leftKey"), col("_2").as("leftValue")) + val rightDf = + inputData.toDF().select((col("_1") * 2).as("rightKey"), col("_2").as("rightValue")) + val df = leftDf.join(rightDf).where("leftKey == rightKey") + + testStream(df)( + StartStream(checkpointLocation = tempDir.getAbsolutePath), + AddData(inputData, (1, 1L), (2, 2L)), + ProcessAllAvailable(), + AddData(inputData, (3, 3L), (4, 4L)), + ProcessAllAvailable() + ) + + val keyWithIndexToValueDf = spark.read.format("statestore") + .option(StateSourceOptions.STORE_NAME, "left-keyWithIndexToValue") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 1) + .load(tempDir.getAbsolutePath) + + val keyWithIndexToValueExpectedDf = Seq( + Row(1L, "update", Row(3, 0L), Row(3, 3L, false), 1), + Row(1L, "update", Row(4, 0L), Row(4, 4L, true), 2), + Row(0L, "update", Row(1, 0L), Row(1, 1L, false), 3), + Row(0L, "update", Row(2, 0L), Row(2, 2L, false), 4), + Row(0L, "update", Row(2, 0L), Row(2, 2L, true), 4) + ) + + checkAnswer(keyWithIndexToValueDf, keyWithIndexToValueExpectedDf) + + val keyToNumValuesDf = spark.read.format("statestore") + .option(StateSourceOptions.STORE_NAME, "left-keyToNumValues") + .option(StateSourceOptions.READ_CHANGE_FEED, value = true) + .option(StateSourceOptions.CHANGE_START_BATCH_ID, 0) + .option(StateSourceOptions.CHANGE_END_BATCH_ID, 1) + .load(tempDir.getAbsolutePath) + + val keyToNumValuesDfExpectedDf = Seq( + Row(1L, "update", Row(3), Row(1L), 1), + Row(1L, "update", Row(4), Row(1L), 2), + Row(0L, "update", Row(1), Row(1L), 3), + Row(0L, "update", Row(2), Row(1L), 4) + ) + + checkAnswer(keyToNumValuesDf, keyToNumValuesDfExpectedDf) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala index f5392cc823f78..705d9f125964f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTestBase.scala @@ -383,7 +383,7 @@ trait StateDataSourceTestBase extends StreamTest with StateStoreMetricsTest { } } - private def getStreamStreamJoinQuery(inputStream: MemoryStream[(Int, Long)]): DataFrame = { + protected def getStreamStreamJoinQuery(inputStream: MemoryStream[(Int, Long)]): DataFrame = { val df = inputStream.toDS() .select(col("_1").as("value"), timestamp_seconds($"_2").as("timestamp")) From b13fc16ebfe00dff782ecccfc65d02bf1fc64688 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 10 Jul 2024 15:27:07 +0800 Subject: [PATCH 08/64] [SPARK-48807][SQL] Binary Support for CSV datasource ### What changes were proposed in this pull request? SPARK-42237 disabled binary output for CSV because the binary values use `java.lang.Object.toString` for outputting. Now we have meaningful binary string representations support in UnivocityGenerator, we can support it now. ### Why are the changes needed? improve csv with spark sql types ### Does this PR introduce _any_ user-facing change? Yes, but it's from failures to success with binary csv tables ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47212 from yaooqinn/SPARK-48807. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../sql/catalyst/csv/UnivocityParser.scala | 3 ++ .../datasources/csv/CSVFileFormat.scala | 2 -- .../datasources/v2/csv/CSVTable.scala | 4 +-- .../execution/datasources/csv/CSVSuite.scala | 36 ++++++++++--------- 4 files changed, 24 insertions(+), 21 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 1c9bdfdb493f7..ccc8f30a9a9c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -253,6 +253,9 @@ class UnivocityParser( case _: StringType => (d: String) => nullSafeDatum(d, name, nullable, options)(UTF8String.fromString) + case _: BinaryType => (d: String) => + nullSafeDatum(d, name, nullable, options)(_.getBytes) + case CalendarIntervalType => (d: String) => nullSafeDatum(d, name, nullable, options) { datum => IntervalUtils.safeStringToInterval(UTF8String.fromString(datum)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 3338006b7bf57..931a3610507f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -150,8 +150,6 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { override def supportDataType(dataType: DataType): Boolean = dataType match { case _: VariantType => false - case _: BinaryType => false - case _: AtomicType => true case udt: UserDefinedType[_] => supportDataType(udt.sqlType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala index 0000caaa8928b..8b4fd3af6ded7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.connector.write.{LogicalWriteInfo, Write, WriteBuild import org.apache.spark.sql.execution.datasources.FileFormat import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.FileTable -import org.apache.spark.sql.types.{AtomicType, BinaryType, DataType, StructType, UserDefinedType} +import org.apache.spark.sql.types.{AtomicType, DataType, StructType, UserDefinedType} import org.apache.spark.sql.util.CaseInsensitiveStringMap case class CSVTable( @@ -55,8 +55,6 @@ case class CSVTable( } override def supportsDataType(dataType: DataType): Boolean = dataType match { - case _: BinaryType => false - case _: AtomicType => true case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) 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 cb442c44832bd..497e9c15f2292 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 @@ -38,9 +38,11 @@ import org.apache.logging.log4j.Level import org.apache.spark.{SparkConf, SparkException, SparkIllegalArgumentException, SparkRuntimeException, SparkUpgradeException, TestUtils} import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoders, QueryTest, Row} import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.catalyst.expressions.ToStringBase import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, HadoopCompressionCodec} import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} +import org.apache.spark.sql.internal.SQLConf.BinaryOutputStyle import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -3173,22 +3175,24 @@ abstract class CSVSuite } } - test("SPARK-42237: change binary to unsupported dataType") { - withTempPath { path => - val colName: String = "value" - checkError( - exception = intercept[AnalysisException] { - Seq(Array[Byte](1, 2)) - .toDF(colName) - .write - .csv(path.getCanonicalPath) - }, - errorClass = "UNSUPPORTED_DATA_TYPE_FOR_DATASOURCE", - parameters = Map( - "columnName" -> s"`$colName`", - "columnType" -> "\"BINARY\"", - "format" -> "CSV") - ) + test("SPARK-48807: Binary support for csv") { + BinaryOutputStyle.values.foreach { style => + withTempPath { path => + withSQLConf(SQLConf.BINARY_OUTPUT_STYLE.key -> style.toString) { + val df = Seq((1, "Spark SQL".getBytes())).toDF("id", "value") + df.write + .option("ds_option", "value") + .format(dataSourceFormat) + .save(path.getCanonicalPath) + val expectedStr = ToStringBase.getBinaryFormatter("Spark SQL".getBytes()) + checkAnswer( + spark.read.csv(path.getCanonicalPath), + Row("1", expectedStr.toString)) + checkAnswer( + spark.read.schema(df.schema).csv(path.getCanonicalPath), + Row(1, expectedStr.getBytes)) + } + } } } From f2dd0b3338a6937bbfbea6cd5fffb2bf9992a1f3 Mon Sep 17 00:00:00 2001 From: Wei Guo Date: Wed, 10 Jul 2024 16:37:45 +0900 Subject: [PATCH 09/64] [SPARK-48848][PYTHON][DOCS] Set the upper bound version of `sphinxcontrib-*` in `dev/requirements.txt` with `sphinx==4.5.0` ### What changes were proposed in this pull request? This PR aims to set the upper bound version of 'sphinxcontrib-*' in `dev/requirements.txt` with `sphinx==4.5.0`. ### Why are the changes needed? Currently, if Spark developers use the command `pip install --upgrade -r dev/requirements.txt` directly to install python-related dependencies, the automatically installed `sphinxcontrib-*` versions don't match `sphinx==4.5.0`. Refered to the issue: https://github.com/sphinx-doc/sphinx/issues/11890. Then they execute the `make html` command for building pySpark docs and the following error will appear: image This problem has been avoided through pinning `sphinxcontrib-*` in workflows of Spark GA: ![image](https://github.com/apache/spark/assets/16032294/bf4906f1-a76d-47bd-af42-f263537f371c) So we can do the similar way by setting the upper bound version of in `requirements.txt` and it will be helpful for Spark developers when making pySpark docs. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47270 from wayneguow/py_require. Authored-by: Wei Guo Signed-off-by: Hyukjin Kwon --- dev/requirements.txt | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/dev/requirements.txt b/dev/requirements.txt index c9d07fae000be..e0216a63ba790 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -37,6 +37,13 @@ nbsphinx numpydoc jinja2 sphinx==4.5.0 +# With sphinx 4.5.0, we need to set the upperbound version of sphinxcontrib*, it should be removed after upgrading sphinx>=5 +sphinxcontrib-applehelp<=1.0.4 +sphinxcontrib-devhelp<=1.0.2 +sphinxcontrib-htmlhelp<=2.0.1 +sphinxcontrib-jsmath<=1.0.1 +sphinxcontrib-qthelp<=1.0.3 +sphinxcontrib-serializinghtml<=1.1.5 sphinx-plotly-directive sphinx-copybutton docutils<0.18.0 From 35fbedb54dd3cc6c0d8841d3a4f01496baab4e73 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 10 Jul 2024 18:27:40 +0800 Subject: [PATCH 10/64] [SPARK-48854][DOCS] Add missing options in CSV documentation ### What changes were proposed in this pull request? This PR added documents for missing CSV options, including `delimiter` as an alternative to `sep`, `charset` as an alternative to `encoding`, `codec` as an alternative to `compression`, and `timeZone`, excluding `columnPruning` which falls back to an internal SQL config. ### Why are the changes needed? improvement for user guide ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? doc build ![image](https://github.com/apache/spark/assets/8326978/d8ff888b-cafa-44e6-ab74-7bf69702a267) ### Was this patch authored or co-authored using generative AI tooling? no Closes #47278 from yaooqinn/SPARK-48854. Authored-by: Kent Yao Signed-off-by: Kent Yao --- docs/sql-data-sources-csv.md | 18 +++++++++++++++--- docs/sql-data-sources-json.md | 1 - 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/docs/sql-data-sources-csv.md b/docs/sql-data-sources-csv.md index a7bb3633d64cb..97a7065e0598f 100644 --- a/docs/sql-data-sources-csv.md +++ b/docs/sql-data-sources-csv.md @@ -55,13 +55,13 @@ Data source options of CSV can be set via: - + - + @@ -261,10 +261,22 @@ Data source options of CSV can be set via: - + + + + + + +
Property NameDefaultMeaningScope
sepsep
delimiter
, Sets a separator for each field and value. This separator can be one or more characters. read/write
encodingencoding
charset
UTF-8 For reading, decodes the CSV files by the given encoding type. For writing, specifies encoding (charset) of saved CSV files. CSV built-in functions ignore this option. read/writeread
compressioncompression
codec
(none) Compression codec to use when saving to file. This can be one of the known case-insensitive shorten names (none, bzip2, gzip, lz4, snappy and deflate). CSV built-in functions ignore this option. write
timeZone(value of spark.sql.session.timeZone configuration)Sets the string that indicates a time zone ID to be used to format timestamps in the JSON datasources or partition values. The following formats of timeZone are supported:
+
    +
  • Region-based zone ID: It should have the form 'area/city', such as 'America/Los_Angeles'.
  • +
  • Zone offset: It should be in the format '(+|-)HH:mm', for example '-08:00' or '+01:00'. Also 'UTC' and 'Z' are supported as aliases of '+00:00'.
  • +
+ Other short names like 'CST' are not recommended to use because they can be ambiguous. +
read/write
Other generic options can be found in Generic File Source Options. diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md index 4ade5170a6d81..18288bfc1654e 100644 --- a/docs/sql-data-sources-json.md +++ b/docs/sql-data-sources-json.md @@ -112,7 +112,6 @@ Data source options of JSON can be set via: - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
Property NameDefaultMeaningScope
timeZone (value of spark.sql.session.timeZone configuration) Sets the string that indicates a time zone ID to be used to format timestamps in the JSON datasources or partition values. The following formats of timeZone are supported:
From a39f70d0c8e85c9911d9a15445fd2a136a66ae4b Mon Sep 17 00:00:00 2001 From: Nemanja Boric Date: Wed, 10 Jul 2024 20:39:00 +0800 Subject: [PATCH 11/64] [SPARK-48843] Prevent infinite loop with BindParameters ### What changes were proposed in this pull request? In order to resolve the named parameters on the subtree, BindParameters recurses into the subtrees and tries to match the pattern with the named parameters. If there's no named parameter in the current level, the rule tries to return the unchanged plan. However, instead of returning the current plan object, the rule always returns the captured root plan node, leading into the infinite recursion. ### Why are the changes needed? Infinite recursion with the named parameters and the global limit. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Added unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47271 from nemanja-boric-databricks/fix-bind. Lead-authored-by: Nemanja Boric Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests/connect/test_connect_basic.py | 7 +++++++ .../apache/spark/sql/catalyst/analysis/parameters.scala | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests/connect/test_connect_basic.py b/python/pyspark/sql/tests/connect/test_connect_basic.py index 598c76a5b25fe..35baf5cca61b3 100755 --- a/python/pyspark/sql/tests/connect/test_connect_basic.py +++ b/python/pyspark/sql/tests/connect/test_connect_basic.py @@ -630,6 +630,13 @@ def test_sql_with_named_args(self): df2 = self.spark.sql(sqlText, args={"minId": 7, "m": SF.create_map(SF.lit("a"), SF.lit(1))}) self.assert_eq(df.toPandas(), df2.toPandas()) + def test_namedargs_with_global_limit(self): + sqlText = """SELECT * FROM VALUES (TIMESTAMP('2022-12-25 10:30:00'), 1) as tab(date, val) + where val = :val""" + df = self.connect.sql(sqlText, args={"val": 1}) + df2 = self.spark.sql(sqlText, args={"val": 1}) + self.assert_eq(df.toPandas(), df2.toPandas()) + def test_sql_with_pos_args(self): sqlText = "SELECT *, element_at(?, 1) FROM range(10) WHERE id > ?" df = self.connect.sql(sqlText, args=[CF.array(CF.lit(1)), 7]) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala index 5b365a0d49aea..30619f21bb8f4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/parameters.scala @@ -212,7 +212,7 @@ object BindParameters extends ParameterizedQueryProcessor with QueryErrorsBase { args(posToIndex(pos)) } - case _ => plan + case other => other } } } From cd2b60dde23db91c91f32355caad0850b2016873 Mon Sep 17 00:00:00 2001 From: Yi Wu Date: Wed, 10 Jul 2024 20:41:48 +0800 Subject: [PATCH 12/64] [SPARK-48791][CORE] Fix perf regression caused by the accumulators registration overhead using CopyOnWriteArrayList ### What changes were proposed in this pull request? This PR proposes to use the `ArrayBuffer` together with the read/write lock rather than `CopyOnWriteArrayList` for `TaskMetrics._externalAccums`. ### Why are the changes needed? Fix the perf regression that caused by the accumulators registration overhead using `CopyOnWriteArrayList`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Manually tested. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47197 from Ngone51/SPARK-48791. Authored-by: Yi Wu Signed-off-by: Wenchen Fan --- .../apache/spark/executor/TaskMetrics.scala | 43 ++++++++++++++----- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 263de0121f7c7..227e7d84654dd 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,7 +17,7 @@ package org.apache.spark.executor -import java.util.concurrent.CopyOnWriteArrayList +import java.util.concurrent.locks.ReentrantReadWriteLock import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import scala.jdk.CollectionConverters._ @@ -29,6 +29,7 @@ import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util._ +import org.apache.spark.util.ArrayImplicits._ /** @@ -150,6 +151,11 @@ class TaskMetrics private[spark] () extends Serializable { private[spark] def setUpdatedBlockStatuses(v: Seq[(BlockId, BlockStatus)]): Unit = _updatedBlockStatuses.setValue(v.asJava) + private val (readLock, writeLock) = { + val lock = new ReentrantReadWriteLock() + (lock.readLock(), lock.writeLock()) + } + /** * Metrics related to reading data from a [[org.apache.spark.rdd.HadoopRDD]] or from persisted * data, defined only in tasks with input. @@ -264,12 +270,32 @@ class TaskMetrics private[spark] () extends Serializable { /** * External accumulators registered with this task. */ - @transient private[spark] lazy val _externalAccums = new CopyOnWriteArrayList[AccumulatorV2[_, _]] + @transient private[spark] lazy val _externalAccums = new ArrayBuffer[AccumulatorV2[_, _]] - private[spark] def externalAccums = _externalAccums.asScala + private[spark] def externalAccums: Seq[AccumulatorV2[_, _]] = withReadLock { + _externalAccums.toArray.toImmutableArraySeq + } + + private def withReadLock[B](fn: => B): B = { + readLock.lock() + try { + fn + } finally { + readLock.unlock() + } + } + + private def withWriteLock[B](fn: => B): B = { + writeLock.lock() + try { + fn + } finally { + writeLock.unlock() + } + } - private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = { - _externalAccums.add(a) + private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = withWriteLock { + _externalAccums += a } private[spark] def accumulators(): Seq[AccumulatorV2[_, _]] = internalAccums ++ externalAccums @@ -328,19 +354,16 @@ private[spark] object TaskMetrics extends Logging { */ def fromAccumulators(accums: Seq[AccumulatorV2[_, _]]): TaskMetrics = { val tm = new TaskMetrics - val externalAccums = new java.util.ArrayList[AccumulatorV2[Any, Any]]() for (acc <- accums) { val name = acc.name - val tmpAcc = acc.asInstanceOf[AccumulatorV2[Any, Any]] if (name.isDefined && tm.nameToAccums.contains(name.get)) { val tmAcc = tm.nameToAccums(name.get).asInstanceOf[AccumulatorV2[Any, Any]] tmAcc.metadata = acc.metadata - tmAcc.merge(tmpAcc) + tmAcc.merge(acc.asInstanceOf[AccumulatorV2[Any, Any]]) } else { - externalAccums.add(tmpAcc) + tm._externalAccums += acc } } - tm._externalAccums.addAll(externalAccums) tm } } From 708bce39db44b75e3f1bd37e24bb7ac35ce05177 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 10 Jul 2024 22:25:50 +0800 Subject: [PATCH 13/64] [SPARK-48857][SQL] Restrict charsets in CSVOptions ### What changes were proposed in this pull request? SPARK-46115 SPARK-46220 started the work for building a consistent charset list for spark, the PR brings it to CSV options ### Why are the changes needed? To make the charset list consistent across different platforms/JDKs ### Does this PR introduce _any_ user-facing change? Yes, legacyCharsets is provided ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47280 from yaooqinn/SPARK-48857. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../spark/sql/catalyst/csv/CSVOptions.scala | 5 +- .../expressions/stringExpressions.scala | 76 ++------ .../sql/catalyst/util/CharsetProvider.scala | 78 ++++++++ .../execution/datasources/csv/CSVSuite.scala | 168 ++++++++++-------- 4 files changed, 191 insertions(+), 136 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharsetProvider.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 13b0b8077128e..5a23d6f7a3ccb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -105,8 +105,9 @@ class CSVOptions( parameters.getOrElse(SEP, parameters.getOrElse(DELIMITER, ","))) val parseMode: ParseMode = parameters.get(MODE).map(ParseMode.fromString).getOrElse(PermissiveMode) - val charset = parameters.getOrElse(ENCODING, - parameters.getOrElse(CHARSET, StandardCharsets.UTF_8.name())) + val charset = parameters.get(ENCODING).orElse(parameters.get(CHARSET)) + .map(CharsetProvider.forName(_, SQLConf.get.legacyJavaCharsets, caller = "CSVOptions")) + .getOrElse(StandardCharsets.UTF_8).name() val quote = getChar(QUOTE, '\"') val escape = getChar(ESCAPE, '\\') diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index a0c796274f761..f25f58731c8cd 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -18,10 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.{ByteBuffer, CharBuffer} -import java.nio.charset.{CharacterCodingException, Charset, CodingErrorAction, IllegalCharsetNameException, UnsupportedCharsetException} +import java.nio.charset.CharacterCodingException import java.text.{BreakIterator, DecimalFormat, DecimalFormatSymbols} -import java.util.{Base64 => JBase64} -import java.util.{HashMap, Locale, Map => JMap} +import java.util.{Base64 => JBase64, HashMap, Locale, Map => JMap} import scala.collection.mutable.ArrayBuffer @@ -36,7 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.objects.{Invoke, StaticInvoke} import org.apache.spark.sql.catalyst.trees.{BinaryLike, UnaryLike} import org.apache.spark.sql.catalyst.trees.TreePattern.{TreePattern, UPPER_OR_LOWER} -import org.apache.spark.sql.catalyst.util.{ArrayData, CollationFactory, CollationSupport, GenericArrayData, TypeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayData, CharsetProvider, CollationFactory, CollationSupport, GenericArrayData, TypeUtils} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.types.{AbstractArrayType, StringTypeAnyCollation, StringTypeBinaryLcase} @@ -2922,32 +2921,13 @@ object StringDecode { legacyCharsets: Boolean, legacyErrorAction: Boolean): UTF8String = { val fromCharset = charset.toString - if (legacyCharsets || Encode.VALID_CHARSETS.contains(fromCharset.toUpperCase(Locale.ROOT))) { - val decoder = try { - val codingErrorAction = if (legacyErrorAction) { - CodingErrorAction.REPLACE - } else { - CodingErrorAction.REPORT - } - Charset.forName(fromCharset) - .newDecoder() - .onMalformedInput(codingErrorAction) - .onUnmappableCharacter(codingErrorAction) - } catch { - case _: IllegalCharsetNameException | - _: UnsupportedCharsetException | - _: IllegalArgumentException => - throw QueryExecutionErrors.invalidCharsetError("decode", fromCharset) - } - try { - val cb = decoder.decode(ByteBuffer.wrap(input)) - UTF8String.fromString(cb.toString) - } catch { - case _: CharacterCodingException => - throw QueryExecutionErrors.malformedCharacterCoding("decode", fromCharset) - } - } else { - throw QueryExecutionErrors.invalidCharsetError("decode", fromCharset) + val decoder = CharsetProvider.newDecoder(fromCharset, legacyCharsets, legacyErrorAction) + try { + val cb = decoder.decode(ByteBuffer.wrap(input)) + UTF8String.fromString(cb.toString) + } catch { + case _: CharacterCodingException => + throw QueryExecutionErrors.malformedCharacterCoding("decode", fromCharset) } } } @@ -3004,9 +2984,6 @@ case class Encode( object Encode { def apply(value: Expression, charset: Expression): Encode = new Encode(value, charset) - private[expressions] final lazy val VALID_CHARSETS = - Set("US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16", "UTF-32") - def encode( input: UTF8String, charset: UTF8String, @@ -3016,32 +2993,13 @@ object Encode { if (input.numBytes == 0 || "UTF-8".equalsIgnoreCase(toCharset)) { return input.getBytes } - if (legacyCharsets || VALID_CHARSETS.contains(toCharset.toUpperCase(Locale.ROOT))) { - val encoder = try { - val codingErrorAction = if (legacyErrorAction) { - CodingErrorAction.REPLACE - } else { - CodingErrorAction.REPORT - } - Charset.forName(toCharset) - .newEncoder() - .onMalformedInput(codingErrorAction) - .onUnmappableCharacter(codingErrorAction) - } catch { - case _: IllegalCharsetNameException | - _: UnsupportedCharsetException | - _: IllegalArgumentException => - throw QueryExecutionErrors.invalidCharsetError("encode", toCharset) - } - try { - val bb = encoder.encode(CharBuffer.wrap(input.toString)) - JavaUtils.bufferToArray(bb) - } catch { - case _: CharacterCodingException => - throw QueryExecutionErrors.malformedCharacterCoding("encode", toCharset) - } - } else { - throw QueryExecutionErrors.invalidCharsetError("encode", toCharset) + val encoder = CharsetProvider.newEncoder(toCharset, legacyCharsets, legacyErrorAction) + try { + val bb = encoder.encode(CharBuffer.wrap(input.toString)) + JavaUtils.bufferToArray(bb) + } catch { + case _: CharacterCodingException => + throw QueryExecutionErrors.malformedCharacterCoding("encode", toCharset) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharsetProvider.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharsetProvider.scala new file mode 100644 index 0000000000000..3dbf5970aae77 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/CharsetProvider.scala @@ -0,0 +1,78 @@ +/* + * 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.util + import java.nio.charset.{Charset, CharsetDecoder, CharsetEncoder, CodingErrorAction, IllegalCharsetNameException, UnsupportedCharsetException} + import java.util.Locale + + import org.apache.spark.sql.errors.QueryExecutionErrors + +private[sql] object CharsetProvider { + + final lazy val VALID_CHARSETS = + Set("US-ASCII", "ISO-8859-1", "UTF-8", "UTF-16BE", "UTF-16LE", "UTF-16", "UTF-32") + + def forName( + charset: String, + legacyCharsets: Boolean, + caller: String = ""): Charset = { + if (legacyCharsets || VALID_CHARSETS.contains(charset.toUpperCase(Locale.ROOT))) { + try { + Charset.forName(charset) + } catch { + case _: IllegalCharsetNameException | + _: UnsupportedCharsetException | + _: IllegalArgumentException => + throw QueryExecutionErrors.invalidCharsetError(caller, charset) + } + } else { + throw QueryExecutionErrors.invalidCharsetError(caller, charset) + } + } + + def newEncoder(charset: String, + legacyCharsets: Boolean, + legacyErrorAction: Boolean, + caller: String = "encode"): CharsetEncoder = { + val codingErrorAction = if (legacyErrorAction) { + CodingErrorAction.REPLACE + } else { + CodingErrorAction.REPORT + } + + forName(charset, legacyCharsets, caller) + .newEncoder() + .onMalformedInput(codingErrorAction) + .onUnmappableCharacter(codingErrorAction) + } + + def newDecoder(charset: String, + legacyCharsets: Boolean, + legacyErrorAction: Boolean, + caller: String = "decode"): CharsetDecoder = { + val codingErrorAction = if (legacyErrorAction) { + CodingErrorAction.REPLACE + } else { + CodingErrorAction.REPORT + } + + forName(charset, legacyCharsets, caller) + .newDecoder() + .onMalformedInput(codingErrorAction) + .onUnmappableCharacter(codingErrorAction) + } + +} 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 497e9c15f2292..21c59c4771b86 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 @@ -18,11 +18,11 @@ package org.apache.spark.sql.execution.datasources.csv import java.io.{EOFException, File} -import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException} +import java.nio.charset.{Charset, StandardCharsets} import java.nio.file.{Files, StandardOpenOption} import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.time.{Duration, Instant, LocalDate, LocalDateTime, Period} +import java.time._ import java.util.Locale import scala.jdk.CollectionConverters._ @@ -35,11 +35,12 @@ import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.GzipCodec import org.apache.logging.log4j.Level -import org.apache.spark.{SparkConf, SparkException, SparkIllegalArgumentException, SparkRuntimeException, SparkUpgradeException, TestUtils} -import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Encoders, QueryTest, Row} +import org.apache.spark._ +import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.catalyst.expressions.ToStringBase import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, HadoopCompressionCodec} +import org.apache.spark.sql.errors.QueryExecutionErrors.toSQLId import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.internal.SQLConf.BinaryOutputStyle @@ -260,15 +261,17 @@ abstract class CSVSuite } test("bad encoding name") { - val exception = intercept[UnsupportedCharsetException] { - spark - .read - .format("csv") - .option("charset", "1-9588-osi") - .load(testFile(carsFile8859)) - } - - assert(exception.getMessage.contains("1-9588-osi")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + spark.read.format("csv").option("charset", "1-9588-osi") + .load(testFile(carsFile8859)) + }, + errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + parameters = Map( + "charset" -> "1-9588-osi", + "functionName" -> toSQLId("CSVOptions"), + "parameter" -> toSQLId("charset")) + ) } test("test different encoding") { @@ -620,31 +623,38 @@ abstract class CSVSuite Seq("iso-8859-1", "utf-8", "utf-16", "utf-32", "windows-1250").foreach { encoding => withTempPath { path => - val csvDir = new File(path, "csv") - Seq(content).toDF().write - .option("encoding", encoding) - .csv(csvDir.getCanonicalPath) - - csvDir.listFiles().filter(_.getName.endsWith("csv")).foreach({ csvFile => - val readback = Files.readAllBytes(csvFile.toPath) - val expected = (content + Properties.lineSeparator).getBytes(Charset.forName(encoding)) - assert(readback === expected) - }) + withSQLConf(SQLConf.LEGACY_JAVA_CHARSETS.key -> "true") { + val csvDir = new File(path, "csv") + Seq(content).toDF().write + .option("encoding", encoding) + .csv(csvDir.getCanonicalPath) + + csvDir.listFiles().filter(_.getName.endsWith("csv")).foreach({ csvFile => + val readback = Files.readAllBytes(csvFile.toPath) + val expected = (content + Properties.lineSeparator).getBytes(Charset.forName(encoding)) + assert(readback === expected) + }) + } } } } test("SPARK-19018: error handling for unsupported charsets") { - val exception = intercept[SparkException] { - withTempPath { path => - val csvDir = new File(path, "csv").getCanonicalPath - Seq("a,A,c,A,b,B").toDF().write - .option("encoding", "1-9588-osi") - .csv(csvDir) - } - } - - assert(exception.getCause.getMessage.contains("1-9588-osi")) + checkError( + exception = intercept[SparkIllegalArgumentException] { + withTempPath { path => + val csvDir = new File(path, "csv").getCanonicalPath + Seq("a,A,c,A,b,B").toDF().write + .option("encoding", "1-9588-osi") + .csv(csvDir) + } + }, + errorClass = "INVALID_PARAMETER_VALUE.CHARSET", + parameters = Map( + "charset" -> "1-9588-osi", + "functionName" -> toSQLId("CSVOptions"), + "parameter" -> toSQLId("charset")) + ) } test("commented lines in CSV data") { @@ -2303,18 +2313,20 @@ abstract class CSVSuite val df = spark.range(3).toDF() Seq("UTF-8", "ISO-8859-1", "CP1251", "US-ASCII", "UTF-16BE", "UTF-32LE").foreach { encoding => Seq(true, false).foreach { header => - withTempPath { path => - df.write - .option("encoding", encoding) - .option("header", header) - .csv(path.getCanonicalPath) - val readback = spark.read - .option("multiLine", true) - .option("encoding", encoding) - .option("inferSchema", true) - .option("header", header) - .csv(path.getCanonicalPath) - checkAnswer(readback, df) + withSQLConf(SQLConf.LEGACY_JAVA_CHARSETS.key -> "true") { + withTempPath { path => + df.write + .option("encoding", encoding) + .option("header", header) + .csv(path.getCanonicalPath) + val readback = spark.read + .option("multiLine", true) + .option("encoding", encoding) + .option("inferSchema", true) + .option("header", header) + .csv(path.getCanonicalPath) + checkAnswer(readback, df) + } } } } @@ -2350,48 +2362,54 @@ abstract class CSVSuite val expected = Seq(("a", 1), ("\nc", 2), ("\nd", 3)) .toDF("_c0", "_c1") - Seq(false, true).foreach { multiLine => - val reader = spark - .read - .option("lineSep", lineSep) - .option("multiLine", multiLine) - .option("encoding", encoding) - val df = if (inferSchema) { - reader.option("inferSchema", true).csv(path.getAbsolutePath) - } else { - reader.schema(schema).csv(path.getAbsolutePath) + withSQLConf(SQLConf.LEGACY_JAVA_CHARSETS.key -> "true") { + Seq(false, true).foreach { multiLine => + val reader = spark + .read + .option("lineSep", lineSep) + .option("multiLine", multiLine) + .option("encoding", encoding) + val df = if (inferSchema) { + reader.option("inferSchema", true).csv(path.getAbsolutePath) + } else { + reader.schema(schema).csv(path.getAbsolutePath) + } + checkAnswer(df, expected) } - checkAnswer(df, expected) } } } // Write withTempPath { path => - Seq("a", "b", "c").toDF("value").coalesce(1) - .write - .option("lineSep", lineSep) - .option("encoding", encoding) - .csv(path.getAbsolutePath) - val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head - val readBack = new String(Files.readAllBytes(partFile.toPath), encoding) - assert( - readBack === s"a${lineSep}b${lineSep}c${lineSep}") + withSQLConf(SQLConf.LEGACY_JAVA_CHARSETS.key -> "true") { + Seq("a", "b", "c").toDF("value").coalesce(1) + .write + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + val partFile = + TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), encoding) + assert(readBack === s"a${lineSep}b${lineSep}c${lineSep}") + } } // Roundtrip withTempPath { path => val df = Seq("a", "b", "c").toDF() - df.write - .option("lineSep", lineSep) - .option("encoding", encoding) - .csv(path.getAbsolutePath) - val readBack = spark - .read - .option("lineSep", lineSep) - .option("encoding", encoding) - .csv(path.getAbsolutePath) - checkAnswer(df, readBack) + withSQLConf(SQLConf.LEGACY_JAVA_CHARSETS.key -> "true") { + df.write + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + val readBack = spark + .read + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + checkAnswer(df, readBack) + } } } } From 06e0e44c26ce1268b2647d01b72fea3d27a12fa9 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 10 Jul 2024 22:31:10 +0800 Subject: [PATCH 14/64] [SPARK-48855][K8S][TESTS] Make `ExecutorPodsAllocatorSuite` independent from default allocation batch size ### What changes were proposed in this pull request? This PR aims to make `ExecutorPodsAllocatorSuite` independent from default allocation batch size. ### Why are the changes needed? To make test assumption explicitly. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47279 from dongjoon-hyun/SPARK-48855. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index f1ef96b942fe1..adad63748abc2 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -60,6 +60,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private val conf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, driverPodName) .set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "10s") + .set(KUBERNETES_ALLOCATION_BATCH_SIZE.key, "5") private val defaultProfile: ResourceProfile = ResourceProfile.getOrCreateDefaultProfile(conf) private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) From 9fa246145f4e36efbcfe781d38f2aabbc98bca8b Mon Sep 17 00:00:00 2001 From: "thomas.hart" Date: Wed, 10 Jul 2024 22:34:32 +0800 Subject: [PATCH 15/64] [MINOR][DOCS] Add example to `countDistinct` ### What changes were proposed in this pull request? This PR adds an example to `countDistinct` docstring demonstrate `count_distinct` and `countDistinct` provide same functionality. ### Why are the changes needed? To improve the documentation. ### Does this PR introduce any user-facing change? No changes in behavior are introduced. ### How was this patch tested? Existing tests. Was this patch authored or co-authored using generative AI tooling? No Closes #47235 from thomhart31/docs-countDistinct. Lead-authored-by: thomas.hart Co-authored-by: Thomas Hart Signed-off-by: Kent Yao --- python/pyspark/sql/functions/builtin.py | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 6fd8fdfec8ea9..c24b9e4378a61 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -5385,6 +5385,24 @@ def countDistinct(col: "ColumnOrName", *cols: "ColumnOrName") -> Column: .. versionchanged:: 3.4.0 Supports Spark Connect. + + Examples + -------- + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame([(1,), (1,), (3,)], ["value"]) + >>> df.select(sf.count_distinct(df.value)).show() + +---------------------+ + |count(DISTINCT value)| + +---------------------+ + | 2| + +---------------------+ + + >>> df.select(sf.countDistinct(df.value)).show() + +---------------------+ + |count(DISTINCT value)| + +---------------------+ + | 2| + +---------------------+ """ return count_distinct(col, *cols) From 8ca1822b4f71145ebf7f61e4235a6fb9bb474805 Mon Sep 17 00:00:00 2001 From: "thomas.hart" Date: Wed, 10 Jul 2024 22:37:08 +0800 Subject: [PATCH 16/64] [SPARK-48823][DOCS] Improve clarity in `lag` docstring ### What changes were proposed in this pull request? This PR edits grammar in `pyspark.sql.functions.lag` docstring. ### Why are the changes needed? To improve the documentation. ### Does this PR introduce any user-facing change? No changes in behavior are introduced. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47236 from thomhart31/docs-lag. Authored-by: thomas.hart Signed-off-by: Kent Yao --- python/pyspark/sql/functions/builtin.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index c24b9e4378a61..fad02a5bcb645 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6899,9 +6899,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| +---+---+ >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("previos_value", lag("c2").over(w)).show() + >>> df.withColumn("previous_value", lag("c2").over(w)).show() +---+---+-------------+ - | c1| c2|previos_value| + | c1| c2|previous_value| +---+---+-------------+ | a| 1| NULL| | a| 2| 1| @@ -6909,9 +6909,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| NULL| | b| 8| 2| +---+---+-------------+ - >>> df.withColumn("previos_value", lag("c2", 1, 0).over(w)).show() + >>> df.withColumn("previous_value", lag("c2", 1, 0).over(w)).show() +---+---+-------------+ - | c1| c2|previos_value| + | c1| c2|previous_value| +---+---+-------------+ | a| 1| 0| | a| 2| 1| @@ -6919,9 +6919,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| 0| | b| 8| 2| +---+---+-------------+ - >>> df.withColumn("previos_value", lag("c2", 2, -1).over(w)).show() + >>> df.withColumn("previous_value", lag("c2", 2, -1).over(w)).show() +---+---+-------------+ - | c1| c2|previos_value| + | c1| c2|previous_value| +---+---+-------------+ | a| 1| -1| | a| 2| -1| From e38383acc121d79e195d95a9009c9eac94841cc2 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Wed, 10 Jul 2024 08:20:45 -0700 Subject: [PATCH 17/64] [SPARK-48844][SQL] USE INVALID_EMPTY_LOCATION instead of UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY when path is empty ### What changes were proposed in this pull request? When running sql on valid datasource files directly, if the given path is an empty string, we currently report UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY, which claims the datasource is invalid. The reason is that the `hadoop.Path` class can not be constructed with empty strings and we wrap `IAE` with UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY. In this PR, we check the path ahead to avoid this ambiguous error message ### Why are the changes needed? trivial bugfix, although this error rarely occurs in REPL environments but might still get a chance to happen when using the query with string interpolation. ### Does this PR introduce _any_ user-facing change? Yes, different error class ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47267 from yaooqinn/SPARK-48844. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../sql/execution/datasources/rules.scala | 6 + .../analyzer-results/sql-on-files.sql.out | 167 ++++++++++++++++ .../sql-tests/inputs/sql-on-files.sql | 19 ++ .../sql-tests/results/sql-on-files.sql.out | 179 ++++++++++++++++++ .../before_1582_date_v2_4.snappy.orc | Bin 0 -> 201 bytes .../src/test/resources/test-data/cars.csv | 7 + .../test-data/dec-in-fixed-len.parquet | Bin 0 -> 460 bytes .../resources/test-data/with-map-fields.json | 5 + 8 files changed, 383 insertions(+) create mode 100644 sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out create mode 100644 sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out create mode 100644 sql/hive-thriftserver/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc create mode 100644 sql/hive-thriftserver/src/test/resources/test-data/cars.csv create mode 100644 sql/hive-thriftserver/src/test/resources/test-data/dec-in-fixed-len.parquet create mode 100644 sql/hive-thriftserver/src/test/resources/test-data/with-map-fields.json 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 2f39a1962d2c0..e4c3cd20dedb9 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 @@ -60,6 +60,12 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { errorClass = "UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY", messageParameters = Map("dataSourceType" -> ident.head)) } + if (isFileFormat && ident.last.isEmpty) { + unresolved.failAnalysis( + errorClass = "INVALID_EMPTY_LOCATION", + messageParameters = Map("location" -> ident.last)) + } + dataSource } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out new file mode 100644 index 0000000000000..78e2a876da861 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out @@ -0,0 +1,167 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM parquet.`` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "parquet.``" + } ] +} + + +-- !query +SELECT * FROM parquet.`/file/not/found` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM parquet.`src/test/resources/test-data/dec-in-fixed-len.parquet` LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [fixed_len_dec#x] + +- Relation [fixed_len_dec#x] parquet + + +-- !query +SELECT * FROM orc.`` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "orc.``" + } ] +} + + +-- !query +SELECT * FROM orc.`/file/not/found` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM orc.`src/test/resources/test-data/before_1582_date_v2_4.snappy.orc` LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [dt#x] + +- Relation [dt#x] orc + + +-- !query +SELECT * FROM csv.`` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "csv.``" + } ] +} + + +-- !query +SELECT * FROM csv.`/file/not/found` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM csv.`src/test/resources/test-data/cars.csv` LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [_c0#x, _c1#x, _c2#x, _c3#x, _c4#x] + +- Relation [_c0#x,_c1#x,_c2#x,_c3#x,_c4#x] csv + + +-- !query +SELECT * FROM json.`` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "json.``" + } ] +} + + +-- !query +SELECT * FROM json.`/file/not/found` +-- !query analysis +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM json.`src/test/resources/test-data/with-map-fields.json` LIMIT 1 +-- !query analysis +GlobalLimit 1 ++- LocalLimit 1 + +- Project [id#xL, intervals#x] + +- Relation [id#xL,intervals#x] json diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql new file mode 100644 index 0000000000000..aee8aaa4d195b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql @@ -0,0 +1,19 @@ +-- Parquet +SELECT * FROM parquet.``; +SELECT * FROM parquet.`/file/not/found`; +SELECT * FROM parquet.`src/test/resources/test-data/dec-in-fixed-len.parquet` LIMIT 1; + +-- ORC +SELECT * FROM orc.``; +SELECT * FROM orc.`/file/not/found`; +SELECT * FROM orc.`src/test/resources/test-data/before_1582_date_v2_4.snappy.orc` LIMIT 1; + +-- CSV +SELECT * FROM csv.``; +SELECT * FROM csv.`/file/not/found`; +SELECT * FROM csv.`src/test/resources/test-data/cars.csv` LIMIT 1; + +-- JSON +SELECT * FROM json.``; +SELECT * FROM json.`/file/not/found`; +SELECT * FROM json.`src/test/resources/test-data/with-map-fields.json` LIMIT 1; diff --git a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out new file mode 100644 index 0000000000000..5c1e5697d029d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out @@ -0,0 +1,179 @@ +-- Automatically generated by SQLQueryTestSuite +-- !query +SELECT * FROM parquet.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 24, + "fragment" : "parquet.``" + } ] +} + + +-- !query +SELECT * FROM parquet.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM parquet.`src/test/resources/test-data/dec-in-fixed-len.parquet` LIMIT 1 +-- !query schema +struct +-- !query output +0.00 + + +-- !query +SELECT * FROM orc.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "orc.``" + } ] +} + + +-- !query +SELECT * FROM orc.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM orc.`src/test/resources/test-data/before_1582_date_v2_4.snappy.orc` LIMIT 1 +-- !query schema +struct +-- !query output +1200-01-01 + + +-- !query +SELECT * FROM csv.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 20, + "fragment" : "csv.``" + } ] +} + + +-- !query +SELECT * FROM csv.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM csv.`src/test/resources/test-data/cars.csv` LIMIT 1 +-- !query schema +struct<_c0:string,_c1:string,_c2:string,_c3:string,_c4:string> +-- !query output +year make model comment blank + + +-- !query +SELECT * FROM json.`` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "INVALID_EMPTY_LOCATION", + "sqlState" : "42K05", + "messageParameters" : { + "location" : "" + }, + "queryContext" : [ { + "objectType" : "", + "objectName" : "", + "startIndex" : 15, + "stopIndex" : 21, + "fragment" : "json.``" + } ] +} + + +-- !query +SELECT * FROM json.`/file/not/found` +-- !query schema +struct<> +-- !query output +org.apache.spark.sql.AnalysisException +{ + "errorClass" : "PATH_NOT_FOUND", + "sqlState" : "42K03", + "messageParameters" : { + "path" : "file:/file/not/found" + } +} + + +-- !query +SELECT * FROM json.`src/test/resources/test-data/with-map-fields.json` LIMIT 1 +-- !query schema +struct,b:struct>> +-- !query output +1 {"a":{"endTime":211,"startTime":111},"b":{"endTime":221,"startTime":121}} diff --git a/sql/hive-thriftserver/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc b/sql/hive-thriftserver/src/test/resources/test-data/before_1582_date_v2_4.snappy.orc new file mode 100644 index 0000000000000000000000000000000000000000..ebe01743b2e2043d9374630c5f24586b6e5df3cb GIT binary patch literal 201 zcmeYdau#G@;9?VE;b074&<8R_xtJLk7=-vZ7_B%sX0K8b0O9}!ZU%-t29BxHjv&Px zYyu1t++dnfLJCMT3NT7=2r+Rm05Ov(P>BQ=*gP%)R8t~2Y8eT&8yPVu5U4^0 z3hK5WJW4SNaflr_N}BXrgbo(V<<*j?`)dokQEarvSr7`N-{ZFH k+I`P Date: Wed, 10 Jul 2024 23:46:16 +0800 Subject: [PATCH 18/64] Revert "[SPARK-48823][DOCS] Improve clarity in `lag` docstring" This reverts commit 8ca1822b4f71145ebf7f61e4235a6fb9bb474805. --- python/pyspark/sql/functions/builtin.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index fad02a5bcb645..c24b9e4378a61 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -6899,9 +6899,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| +---+---+ >>> w = Window.partitionBy("c1").orderBy("c2") - >>> df.withColumn("previous_value", lag("c2").over(w)).show() + >>> df.withColumn("previos_value", lag("c2").over(w)).show() +---+---+-------------+ - | c1| c2|previous_value| + | c1| c2|previos_value| +---+---+-------------+ | a| 1| NULL| | a| 2| 1| @@ -6909,9 +6909,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| NULL| | b| 8| 2| +---+---+-------------+ - >>> df.withColumn("previous_value", lag("c2", 1, 0).over(w)).show() + >>> df.withColumn("previos_value", lag("c2", 1, 0).over(w)).show() +---+---+-------------+ - | c1| c2|previous_value| + | c1| c2|previos_value| +---+---+-------------+ | a| 1| 0| | a| 2| 1| @@ -6919,9 +6919,9 @@ def lag(col: "ColumnOrName", offset: int = 1, default: Optional[Any] = None) -> | b| 2| 0| | b| 8| 2| +---+---+-------------+ - >>> df.withColumn("previous_value", lag("c2", 2, -1).over(w)).show() + >>> df.withColumn("previos_value", lag("c2", 2, -1).over(w)).show() +---+---+-------------+ - | c1| c2|previous_value| + | c1| c2|previos_value| +---+---+-------------+ | a| 1| -1| | a| 2| -1| From 4d13c2231cc4852b922079895223104972dc960b Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Thu, 11 Jul 2024 09:15:21 +0900 Subject: [PATCH 19/64] [SPARK-48763][CONNECT][BUILD] Move connect server and common to builtin module ### What changes were proposed in this pull request? This PR proposes to move the connect server to builtin module. From: ``` connector/connect/server connector/connect/common ``` To: ``` connect/server connect/common ``` ### Why are the changes needed? So the end users do not have to specify `--packages` when they start the Spark Connect server. Spark Connect client remains as a separate module. This was also pointed out in https://github.com/apache/spark/pull/39928#issuecomment-1428264541. ### Does this PR introduce _any_ user-facing change? Yes, users don't have to specify `--packages` anymore. ### How was this patch tested? CI in this PR should verify them. Also manually tested several basic commands such as: - Maven build - SBT build - Running basic Scala client commands ```bash cd connector/connect bin/spark-connect bin/spark-connect-scala-client ``` - Running basic PySpark client commands ```bash bin/pyspark --remote local ``` - Connecting to the server launched by `./sbin/start-connect-server.sh` ```bash ./sbin/start-connect-server.sh bin/pyspark --remote "sc://localhost" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47157 from HyukjinKwon/move-connect-server-builtin. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/labeler.yml | 1 + .github/workflows/build_and_test.yml | 2 +- .github/workflows/build_python_connect.yml | 4 +- .github/workflows/build_python_connect35.yml | 2 +- .github/workflows/maven_test.yml | 2 +- assembly/pom.xml | 63 ++++++------ bin/spark-connect-shell | 3 +- connect/common/README.md | 5 + {connector/connect => connect}/common/pom.xml | 2 +- .../common/src/main/buf.gen.yaml | 0 .../common/src/main/buf.work.yaml | 0 .../common/src/main/protobuf/buf.yaml | 0 .../main/protobuf/spark/connect/base.proto | 0 .../main/protobuf/spark/connect/catalog.proto | 0 .../protobuf/spark/connect/commands.proto | 0 .../main/protobuf/spark/connect/common.proto | 0 .../spark/connect/example_plugins.proto | 0 .../protobuf/spark/connect/expressions.proto | 0 .../protobuf/spark/connect/relations.proto | 0 .../main/protobuf/spark/connect/types.proto | 0 .../spark/sql/connect/ConnectProtoUtils.scala | 0 .../sql/connect/client/ArtifactManager.scala | 0 .../sql/connect/client/ClassFinder.scala | 0 .../connect/client/CloseableIterator.scala | 0 .../CustomSparkConnectBlockingStub.scala | 0 .../client/CustomSparkConnectStub.scala | 0 ...cutePlanResponseReattachableIterator.scala | 0 .../client/GrpcExceptionConverter.scala | 0 .../sql/connect/client/GrpcRetryHandler.scala | 0 .../connect/client/ResponseValidator.scala | 0 .../sql/connect/client/RetriesExceeded.scala | 0 .../sql/connect/client/RetryPolicy.scala | 0 .../connect/client/SparkConnectClient.scala | 0 .../client/SparkConnectClientParser.scala | 0 .../client/SparkConnectStubState.scala | 0 .../sql/connect/client/SparkResult.scala | 0 .../client/arrow/ArrowDeserializer.scala | 0 .../client/arrow/ArrowEncoderUtils.scala | 0 .../client/arrow/ArrowSerializer.scala | 0 .../client/arrow/ArrowVectorReader.scala | 0 .../ConcatenatingArrowStreamReader.scala | 0 .../client/arrow/ScalaCollectionUtils.scala | 0 .../spark/sql/connect/client/package.scala | 0 .../sql/connect/common/Abbreviator.scala | 0 .../common/DataTypeProtoConverter.scala | 0 .../connect/common/ForeachWriterPacket.scala | 0 .../sql/connect/common/InvalidPlanInput.scala | 0 .../common/LiteralValueProtoConverter.scala | 0 .../sql/connect/common/ProtoDataTypes.scala | 0 .../spark/sql/connect/common/ProtoUtils.scala | 0 .../common/StorageLevelProtoConverter.scala | 0 .../common/StreamingListenerPacket.scala | 0 .../spark/sql/connect/common/UdfPacket.scala | 0 .../spark/sql/connect/common/UdfUtils.scala | 0 .../connect/common/config/ConnectCommon.scala | 0 .../test/resources/artifact-tests/Hello.class | Bin .../resources/artifact-tests/crc/Hello.txt | 0 .../resources/artifact-tests/crc/README.md | 0 .../artifact-tests/crc/junitLargeJar.txt | 0 .../artifact-tests/crc/smallClassFile.txt | 0 .../artifact-tests/crc/smallClassFileDup.txt | 0 .../resources/artifact-tests/crc/smallJar.txt | 0 .../artifact-tests/junitLargeJar.jar | Bin .../artifact-tests/smallClassFile.class | Bin .../artifact-tests/smallClassFileDup.class | Bin .../resources/artifact-tests/smallJar.jar | Bin .../test/resources/protobuf-tests/common.desc | 0 .../explain-results/alias_string.explain | 0 .../explain-results/alias_symbol.explain | 0 .../query-tests/explain-results/apply.explain | 0 .../explain-results/as_string.explain | 0 .../explain-results/as_symbol.explain | 0 .../explain-results/between_expr.explain | 0 .../explain-results/coalesce.explain | 0 .../query-tests/explain-results/col.explain | 0 .../explain-results/colRegex.explain | 0 .../explain-results/column_add.explain | 0 .../explain-results/column_alias.explain | 0 .../explain-results/column_and.explain | 0 .../explain-results/column_apply.explain | 0 .../explain-results/column_as_multi.explain | 0 .../column_as_with_metadata.explain | 0 .../explain-results/column_asc.explain | 0 .../column_asc_nulls_first.explain | 0 .../column_asc_nulls_last.explain | 0 .../explain-results/column_between.explain | 0 .../explain-results/column_bitwiseAND.explain | 0 .../explain-results/column_bitwiseOR.explain | 0 .../explain-results/column_bitwiseXOR.explain | 0 .../explain-results/column_cast.explain | 0 .../explain-results/column_contains.explain | 0 .../explain-results/column_desc.explain | 0 .../column_desc_nulls_first.explain | 0 .../column_desc_nulls_last.explain | 0 .../explain-results/column_divide.explain | 0 .../explain-results/column_dropFields.explain | 0 .../explain-results/column_endsWith.explain | 0 .../explain-results/column_eqNullSafe.explain | 0 .../explain-results/column_equals.explain | 0 .../explain-results/column_geq.explain | 0 .../explain-results/column_getField.explain | 0 .../explain-results/column_getItem.explain | 0 .../explain-results/column_gt.explain | 0 .../explain-results/column_ilike.explain | 0 .../explain-results/column_isNaN.explain | 0 .../explain-results/column_isNotNull.explain | 0 .../explain-results/column_isNull.explain | 0 .../explain-results/column_isin.explain | 0 .../explain-results/column_leq.explain | 0 .../explain-results/column_like.explain | 0 .../explain-results/column_lt.explain | 0 .../explain-results/column_modulo.explain | 0 .../explain-results/column_multiply.explain | 0 .../explain-results/column_not.explain | 0 .../explain-results/column_not_equals.explain | 0 .../explain-results/column_or.explain | 0 .../explain-results/column_rlike.explain | 0 .../explain-results/column_star.explain | 0 .../column_star_with_target.explain | 0 .../explain-results/column_startsWith.explain | 0 .../explain-results/column_substr.explain | 0 .../explain-results/column_subtract.explain | 0 .../explain-results/column_try_cast.explain | 0 .../column_unary_minus.explain | 0 .../column_when_otherwise.explain | 0 .../explain-results/column_withField.explain | 0 .../explain-results/crossJoin.explain | 0 .../explain-results/crosstab.explain | 0 .../explain-results/csv_from_dataset.explain | 0 .../explain-results/cube_column.explain | 0 .../explain-results/cube_string.explain | 0 .../explain-results/describe.explain | 0 .../explain-results/distinct.explain | 0 .../query-tests/explain-results/drop.explain | 0 .../explain-results/dropDuplicates.explain | 0 .../dropDuplicates_names_array.explain | 0 .../dropDuplicates_names_seq.explain | 0 .../dropDuplicates_varargs.explain | 0 .../drop_multiple_column.explain | 0 .../drop_multiple_strings.explain | 0 .../drop_single_column.explain | 0 .../drop_single_string.explain | 0 .../explain-results/except.explain | 0 .../explain-results/exceptAll.explain | 0 .../expression_extension.explain | 0 .../expression_extension_deprecated.explain | 0 .../query-tests/explain-results/fill.explain | 0 .../explain-results/filter.explain | 0 .../explain-results/filter_expr.explain | 0 .../explain-results/freqItems.explain | 0 .../from_avro_with_options.explain | 0 .../from_avro_without_options.explain | 0 ...obuf_messageClassName_descFilePath.explain | 0 ...sageClassName_descFilePath_options.explain | 0 .../explain-results/function_abs.explain | 0 .../explain-results/function_acos.explain | 0 .../explain-results/function_acosh.explain | 0 .../function_add_months.explain | 0 .../function_aes_decrypt.explain | 0 .../function_aes_decrypt_with_mode.explain | 0 ...tion_aes_decrypt_with_mode_padding.explain | 0 ..._aes_decrypt_with_mode_padding_aad.explain | 0 .../function_aes_encrypt.explain | 0 .../function_aes_encrypt_with_mode.explain | 0 ...tion_aes_encrypt_with_mode_padding.explain | 0 ...n_aes_encrypt_with_mode_padding_iv.explain | 0 ...s_encrypt_with_mode_padding_iv_aad.explain | 0 .../function_aggregate.explain | 0 .../explain-results/function_any.explain | 0 ...nction_any_value_with_ignore_nulls.explain | 0 ...ction_any_value_with_respect_nulls.explain | 0 .../function_approx_count_distinct.explain | 0 ...function_approx_count_distinct_rsd.explain | 0 .../function_approx_percentile.explain | 0 .../explain-results/function_array.explain | 0 .../function_array_agg.explain | 0 .../function_array_append.explain | 0 .../function_array_compact.explain | 0 .../function_array_contains.explain | 0 .../function_array_distinct.explain | 0 .../function_array_except.explain | 0 .../function_array_insert.explain | 0 .../function_array_intersect.explain | 0 .../function_array_join.explain | 0 ...n_array_join_with_null_replacement.explain | 0 .../function_array_max.explain | 0 .../function_array_min.explain | 0 .../function_array_position.explain | 0 .../function_array_prepend.explain | 0 .../function_array_remove.explain | 0 .../function_array_repeat.explain | 0 .../function_array_size.explain | 0 .../function_array_sort.explain | 0 ...unction_array_sort_with_comparator.explain | 0 .../function_array_union.explain | 0 .../function_arrays_overlap.explain | 0 .../function_arrays_zip.explain | 0 .../explain-results/function_asc.explain | 0 .../function_asc_nulls_first.explain | 0 .../function_asc_nulls_last.explain | 0 .../explain-results/function_ascii.explain | 0 .../explain-results/function_asin.explain | 0 .../explain-results/function_asinh.explain | 0 .../function_assert_true_with_message.explain | 0 .../explain-results/function_atan.explain | 0 .../explain-results/function_atan2.explain | 0 .../explain-results/function_atanh.explain | 0 .../explain-results/function_avg.explain | 0 .../explain-results/function_base64.explain | 0 .../explain-results/function_bin.explain | 0 .../explain-results/function_bit_and.explain | 0 .../function_bit_count.explain | 0 .../explain-results/function_bit_get.explain | 0 .../function_bit_length.explain | 0 .../explain-results/function_bit_or.explain | 0 .../explain-results/function_bit_xor.explain | 0 .../function_bitmap_bit_position.explain | 0 .../function_bitmap_bucket_number.explain | 0 .../function_bitmap_construct_agg.explain | 0 .../function_bitmap_count.explain | 0 .../function_bitmap_or_agg.explain | 0 .../function_bitwise_not.explain | 0 .../explain-results/function_bool_and.explain | 0 .../explain-results/function_bool_or.explain | 0 .../explain-results/function_bround.explain | 0 .../explain-results/function_btrim.explain | 0 ...n_btrim_with_specified_trim_string.explain | 0 .../explain-results/function_bucket.explain | 0 .../function_call_function.explain | 0 .../function_cardinality.explain | 0 .../explain-results/function_ceil.explain | 0 .../function_ceil_scale.explain | 0 .../explain-results/function_ceiling.explain | 0 .../function_ceiling_scale.explain | 0 .../explain-results/function_char.explain | 0 .../function_char_length.explain | 0 .../function_character_length.explain | 0 .../explain-results/function_chr.explain | 0 .../explain-results/function_coalesce.explain | 0 .../explain-results/function_col.explain | 0 .../explain-results/function_collate.explain | 0 .../function_collation.explain | 0 .../function_collect_list.explain | 0 .../function_collect_set.explain | 0 .../explain-results/function_concat.explain | 0 .../function_concat_ws.explain | 0 .../explain-results/function_contains.explain | 0 .../explain-results/function_conv.explain | 0 ...ert_timezone_with_source_time_zone.explain | 0 ..._timezone_without_source_time_zone.explain | 0 .../explain-results/function_corr.explain | 0 .../explain-results/function_cos.explain | 0 .../explain-results/function_cosh.explain | 0 .../explain-results/function_cot.explain | 0 .../explain-results/function_count.explain | 0 .../function_countDistinct.explain | 0 .../explain-results/function_count_if.explain | 0 .../function_count_min_sketch.explain | 0 .../function_count_typed.explain | 0 .../function_covar_pop.explain | 0 .../function_covar_samp.explain | 0 .../explain-results/function_crc32.explain | 0 .../explain-results/function_csc.explain | 0 .../function_cume_dist.explain | 0 .../explain-results/function_curdate.explain | 0 .../function_current_catalog.explain | 0 .../function_current_database.explain | 0 .../function_current_date.explain | 0 .../function_current_schema.explain | 0 .../function_current_timestamp.explain | 0 .../function_current_timezone.explain | 0 .../function_current_user.explain | 0 .../explain-results/function_date_add.explain | 0 .../function_date_diff.explain | 0 .../function_date_format.explain | 0 .../function_date_from_unix_date.explain | 0 .../function_date_part.explain | 0 .../explain-results/function_date_sub.explain | 0 .../function_date_trunc.explain | 0 .../explain-results/function_dateadd.explain | 0 .../explain-results/function_datediff.explain | 0 .../explain-results/function_datepart.explain | 0 .../explain-results/function_day.explain | 0 .../explain-results/function_dayname.explain | 0 .../function_dayofmonth.explain | 0 .../function_dayofweek.explain | 0 .../function_dayofyear.explain | 0 .../explain-results/function_days.explain | 0 .../explain-results/function_decode.explain | 0 .../explain-results/function_degrees.explain | 0 .../function_dense_rank.explain | 0 .../explain-results/function_desc.explain | 0 .../function_desc_nulls_first.explain | 0 .../function_desc_nulls_last.explain | 0 .../explain-results/function_e.explain | 0 .../function_element_at.explain | 0 .../explain-results/function_elt.explain | 0 .../explain-results/function_encode.explain | 0 .../explain-results/function_endswith.explain | 0 .../function_equal_null.explain | 0 .../explain-results/function_every.explain | 0 .../explain-results/function_exists.explain | 0 .../explain-results/function_exp.explain | 0 .../explain-results/function_explode.explain | 0 .../function_explode_outer.explain | 0 .../explain-results/function_expm1.explain | 0 .../explain-results/function_expr.explain | 0 .../explain-results/function_extract.explain | 0 .../function_factorial.explain | 0 .../explain-results/function_filter.explain | 0 .../function_filter_with_pair_input.explain | 0 .../function_find_in_set.explain | 0 ...tion_first_value_with_ignore_nulls.explain | 0 ...ion_first_value_with_respect_nulls.explain | 0 .../function_first_with_ignore_nulls.explain | 0 .../function_first_with_respect_nulls.explain | 0 .../explain-results/function_flatten.explain | 0 .../explain-results/function_floor.explain | 0 .../function_floor_scale.explain | 0 .../explain-results/function_forall.explain | 0 .../function_format_number.explain | 0 .../explain-results/function_from_csv.explain | 0 .../function_from_json.explain | 0 .../function_from_unixtime.explain | 0 .../function_from_utc_timestamp.explain | 0 .../explain-results/function_get.explain | 0 .../function_get_json_object.explain | 0 .../explain-results/function_getbit.explain | 0 .../explain-results/function_greatest.explain | 0 .../explain-results/function_hash.explain | 0 .../explain-results/function_hex.explain | 0 .../function_histogram_numeric.explain | 0 .../explain-results/function_hour.explain | 0 .../explain-results/function_hours.explain | 0 .../explain-results/function_hypot.explain | 0 .../explain-results/function_ifnull.explain | 0 .../explain-results/function_ilike.explain | 0 .../function_ilike_with_escape.explain | 0 .../explain-results/function_initcap.explain | 0 .../explain-results/function_inline.explain | 0 .../function_inline_outer.explain | 0 .../function_input_file_block_length.explain | 0 .../function_input_file_block_start.explain | 0 .../function_input_file_name.explain | 0 .../function_is_variant_null.explain | 0 .../explain-results/function_isnan.explain | 0 .../function_isnotnull.explain | 0 .../explain-results/function_isnull.explain | 0 .../function_java_method.explain | 0 .../function_json_array_length.explain | 0 .../function_json_object_keys.explain | 0 .../function_json_tuple.explain | 0 .../explain-results/function_kurtosis.explain | 0 .../explain-results/function_lag.explain | 0 .../explain-results/function_last_day.explain | 0 ...ction_last_value_with_ignore_nulls.explain | 0 ...tion_last_value_with_respect_nulls.explain | 0 .../function_last_with_ignore_nulls.explain | 0 .../function_last_with_respect_nulls.explain | 0 .../explain-results/function_lcase.explain | 0 .../explain-results/function_lead.explain | 0 .../explain-results/function_least.explain | 0 .../explain-results/function_left.explain | 0 .../explain-results/function_len.explain | 0 .../explain-results/function_length.explain | 0 .../function_levenshtein.explain | 0 ...unction_levenshtein_with_threshold.explain | 0 .../explain-results/function_like.explain | 0 .../function_like_with_escape.explain | 0 .../explain-results/function_lit.explain | 0 .../function_lit_array.explain | 0 .../explain-results/function_ln.explain | 0 .../function_localtimestamp.explain | 0 .../explain-results/function_locate.explain | 0 .../function_locate_with_pos.explain | 0 .../explain-results/function_log.explain | 0 .../explain-results/function_log10.explain | 0 .../explain-results/function_log1p.explain | 0 .../explain-results/function_log2.explain | 0 .../function_log_with_base.explain | 0 .../explain-results/function_lower.explain | 0 .../explain-results/function_lpad.explain | 0 .../function_lpad_binary.explain | 0 .../explain-results/function_ltrim.explain | 0 .../function_ltrim_with_pattern.explain | 0 .../function_make_date.explain | 0 .../function_make_dt_interval.explain | 0 .../function_make_dt_interval_days.explain | 0 ...nction_make_dt_interval_days_hours.explain | 0 ...n_make_dt_interval_days_hours_mins.explain | 0 ...e_dt_interval_days_hours_mins_secs.explain | 0 .../function_make_interval.explain | 0 .../function_make_interval_years.explain | 0 ...unction_make_interval_years_months.explain | 0 ...n_make_interval_years_months_weeks.explain | 0 ...e_interval_years_months_weeks_days.explain | 0 ...rval_years_months_weeks_days_hours.explain | 0 ...years_months_weeks_days_hours_mins.explain | 0 ..._months_weeks_days_hours_mins_secs.explain | 0 ...n_make_timestamp_ltz_with_timezone.explain | 0 ...ake_timestamp_ltz_without_timezone.explain | 0 .../function_make_timestamp_ntz.explain | 0 ...ction_make_timestamp_with_timezone.explain | 0 ...on_make_timestamp_without_timezone.explain | 0 .../function_make_ym_interval.explain | 0 .../function_make_ym_interval_years.explain | 0 ...tion_make_ym_interval_years_months.explain | 0 .../explain-results/function_map.explain | 0 .../function_map_concat.explain | 0 .../function_map_contains_key.explain | 0 .../function_map_entries.explain | 0 .../function_map_filter.explain | 0 .../function_map_from_arrays.explain | 0 .../function_map_from_entries.explain | 0 .../explain-results/function_map_keys.explain | 0 .../function_map_values.explain | 0 .../function_map_zip_with.explain | 0 .../explain-results/function_mask.explain | 0 ...ction_mask_with_specific_upperChar.explain | 0 ..._with_specific_upperChar_lowerChar.explain | 0 ...ific_upperChar_lowerChar_digitChar.explain | 0 ...Char_lowerChar_digitChar_otherChar.explain | 0 .../explain-results/function_max.explain | 0 .../explain-results/function_max_by.explain | 0 .../explain-results/function_md5.explain | 0 .../explain-results/function_median.explain | 0 .../explain-results/function_min.explain | 0 .../explain-results/function_min_by.explain | 0 .../explain-results/function_minute.explain | 0 .../explain-results/function_mode.explain | 0 ...nction_monotonically_increasing_id.explain | 0 .../explain-results/function_month.explain | 0 .../function_monthname.explain | 0 .../explain-results/function_months.explain | 0 .../function_months_between.explain | 0 ...ction_months_between_with_roundoff.explain | 0 .../function_named_struct.explain | 0 .../explain-results/function_nanvl.explain | 0 .../explain-results/function_negate.explain | 0 .../explain-results/function_negative.explain | 0 .../explain-results/function_next_day.explain | 0 .../explain-results/function_now.explain | 0 .../function_nth_value.explain | 0 .../explain-results/function_ntile.explain | 0 .../explain-results/function_nullif.explain | 0 .../explain-results/function_nvl.explain | 0 .../explain-results/function_nvl2.explain | 0 .../function_octet_length.explain | 0 .../explain-results/function_overlay.explain | 0 .../function_overlay_with_len.explain | 0 .../function_parse_json.explain | 0 .../function_parse_url.explain | 0 .../function_parse_url_with_key.explain | 0 .../function_percent_rank.explain | 0 .../function_percentile_approx.explain | 0 ...function_percentile_with_frequency.explain | 0 ...ction_percentile_without_frequency.explain | 0 .../explain-results/function_pi.explain | 0 .../explain-results/function_pmod.explain | 0 .../function_posexplode.explain | 0 .../function_posexplode_outer.explain | 0 .../explain-results/function_position.explain | 0 .../function_position_with_start.explain | 0 .../explain-results/function_positive.explain | 0 .../explain-results/function_pow.explain | 0 .../explain-results/function_power.explain | 0 .../explain-results/function_printf.explain | 0 .../explain-results/function_product.explain | 0 .../explain-results/function_quarter.explain | 0 .../explain-results/function_radians.explain | 0 .../function_raise_error.explain | 0 .../function_rand_with_seed.explain | 0 .../function_randn_with_seed.explain | 0 .../function_random_with_seed.explain | 0 .../explain-results/function_rank.explain | 0 .../explain-results/function_reduce.explain | 0 .../explain-results/function_reflect.explain | 0 .../explain-results/function_regexp.explain | 0 .../function_regexp_count.explain | 0 .../function_regexp_extract.explain | 0 ...extract_all_with_regex_group_index.explain | 0 ...ract_all_without_regex_group_index.explain | 0 ...egexp_instr_with_regex_group_index.explain | 0 ...xp_instr_without_regex_group_index.explain | 0 .../function_regexp_like.explain | 0 .../function_regexp_replace.explain | 0 .../function_regexp_substr.explain | 0 .../function_regr_avgx.explain | 0 .../function_regr_avgy.explain | 0 .../function_regr_count.explain | 0 .../function_regr_intercept.explain | 0 .../explain-results/function_regr_r2.explain | 0 .../function_regr_slope.explain | 0 .../explain-results/function_regr_sxx.explain | 0 .../explain-results/function_regr_sxy.explain | 0 .../explain-results/function_regr_syy.explain | 0 .../explain-results/function_replace.explain | 0 ...tion_replace_with_specified_string.explain | 0 .../explain-results/function_reverse.explain | 0 .../explain-results/function_right.explain | 0 .../explain-results/function_rint.explain | 0 .../explain-results/function_rlike.explain | 0 .../explain-results/function_round.explain | 0 .../function_row_number.explain | 0 .../explain-results/function_rpad.explain | 0 .../function_rpad_binary.explain | 0 .../explain-results/function_rtrim.explain | 0 .../function_rtrim_with_pattern.explain | 0 .../function_schema_of_csv.explain | 0 .../function_schema_of_json.explain | 0 ...nction_schema_of_json_with_options.explain | 0 .../function_schema_of_variant.explain | 0 .../function_schema_of_variant_agg.explain | 0 .../explain-results/function_sec.explain | 0 .../explain-results/function_second.explain | 0 .../function_sentences.explain | 0 .../function_sentences_with_locale.explain | 0 .../explain-results/function_sequence.explain | 0 .../function_session_user.explain | 0 .../function_session_window.explain | 0 .../explain-results/function_sha.explain | 0 .../explain-results/function_sha1.explain | 0 .../explain-results/function_sha2.explain | 0 .../function_shiftleft.explain | 0 .../function_shiftright.explain | 0 .../function_shiftrightunsigned.explain | 0 .../explain-results/function_sign.explain | 0 .../explain-results/function_signum.explain | 0 .../explain-results/function_sin.explain | 0 .../explain-results/function_sinh.explain | 0 .../explain-results/function_size.explain | 0 .../explain-results/function_skewness.explain | 0 .../explain-results/function_slice.explain | 0 .../explain-results/function_some.explain | 0 .../function_sort_array.explain | 0 .../function_spark_partition_id.explain | 0 .../explain-results/function_split.explain | 0 .../function_split_part.explain | 0 .../function_split_using_columns.explain | 0 .../function_split_with_limit.explain | 0 ...ion_split_with_limit_using_columns.explain | 0 .../explain-results/function_sqrt.explain | 0 .../explain-results/function_stack.explain | 0 .../function_startswith.explain | 0 .../explain-results/function_std.explain | 0 .../explain-results/function_stddev.explain | 0 .../function_stddev_pop.explain | 0 .../function_stddev_samp.explain | 0 .../function_str_to_map.explain | 0 ...p_with_pair_and_keyValue_delimiter.explain | 0 ...ion_str_to_map_with_pair_delimiter.explain | 0 .../explain-results/function_struct.explain | 0 .../explain-results/function_substr.explain | 0 .../function_substr_with_len.explain | 0 .../function_substring.explain | 0 .../function_substring_index.explain | 0 .../function_substring_using_columns.explain | 0 .../function_substring_with_columns.explain | 0 .../explain-results/function_sum.explain | 0 .../function_sum_distinct.explain | 0 .../explain-results/function_tan.explain | 0 .../explain-results/function_tanh.explain | 0 .../function_timestamp_add.explain | 0 .../function_timestamp_diff.explain | 0 .../function_timestamp_micros.explain | 0 .../function_timestamp_millis.explain | 0 .../function_timestamp_seconds.explain | 0 .../function_to_binary.explain | 0 .../function_to_binary_with_format.explain | 0 .../explain-results/function_to_char.explain | 0 .../explain-results/function_to_csv.explain | 0 .../explain-results/function_to_date.explain | 0 .../function_to_date_with_format.explain | 0 .../explain-results/function_to_json.explain | 0 .../function_to_number.explain | 0 .../function_to_timestamp.explain | 0 .../function_to_timestamp_ltz.explain | 0 ...ction_to_timestamp_ltz_with_format.explain | 0 .../function_to_timestamp_ntz.explain | 0 ...ction_to_timestamp_ntz_with_format.explain | 0 .../function_to_timestamp_with_format.explain | 0 .../function_to_unix_timestamp.explain | 0 ...tion_to_unix_timestamp_with_format.explain | 0 .../function_to_utc_timestamp.explain | 0 .../function_to_varchar.explain | 0 .../function_transform.explain | 0 .../function_transform_keys.explain | 0 .../function_transform_values.explain | 0 .../function_transform_with_index.explain | 0 .../function_translate.explain | 0 .../explain-results/function_trim.explain | 0 .../function_trim_with_pattern.explain | 0 .../explain-results/function_trunc.explain | 0 .../explain-results/function_try_add.explain | 0 .../function_try_aes_decrypt.explain | 0 ...function_try_aes_decrypt_with_mode.explain | 0 ..._try_aes_decrypt_with_mode_padding.explain | 0 ..._aes_decrypt_with_mode_padding_aad.explain | 0 .../explain-results/function_try_avg.explain | 0 .../function_try_divide.explain | 0 .../function_try_element_at_array.explain | 0 .../function_try_element_at_map.explain | 0 .../function_try_multiply.explain | 0 .../function_try_parse_json.explain | 0 .../function_try_reflect.explain | 0 .../function_try_subtract.explain | 0 .../explain-results/function_try_sum.explain | 0 .../function_try_to_binary.explain | 0 ...ction_try_to_binary_without_format.explain | 0 .../function_try_to_number.explain | 0 .../function_try_to_timestamp.explain | 0 ...on_try_to_timestamp_without_format.explain | 0 .../function_try_variant_get.explain | 0 .../explain-results/function_typedLit.explain | 0 .../explain-results/function_typeof.explain | 0 .../explain-results/function_ucase.explain | 0 .../explain-results/function_unbase64.explain | 0 .../explain-results/function_unhex.explain | 0 .../function_unix_date.explain | 0 .../function_unix_micros.explain | 0 .../function_unix_millis.explain | 0 .../function_unix_seconds.explain | 0 .../function_unix_timestamp.explain | 0 ...unction_unix_timestamp_with_format.explain | 0 .../explain-results/function_upper.explain | 0 .../function_url_decode.explain | 0 .../function_url_encode.explain | 0 .../explain-results/function_user.explain | 0 .../explain-results/function_var_pop.explain | 0 .../explain-results/function_var_samp.explain | 0 .../explain-results/function_variance.explain | 0 .../function_variant_get.explain | 0 .../explain-results/function_weekday.explain | 0 .../function_weekofyear.explain | 0 .../explain-results/function_window.explain | 0 .../function_window_time.explain | 0 .../explain-results/function_xpath.explain | 0 .../function_xpath_boolean.explain | 0 .../function_xpath_double.explain | 0 .../function_xpath_float.explain | 0 .../function_xpath_int.explain | 0 .../function_xpath_long.explain | 0 .../function_xpath_number.explain | 0 .../function_xpath_short.explain | 0 .../function_xpath_string.explain | 0 .../explain-results/function_xxhash64.explain | 0 .../explain-results/function_year.explain | 0 .../explain-results/function_years.explain | 0 .../explain-results/function_zip_with.explain | 0 .../explain-results/groupby_agg.explain | 0 .../groupby_agg_columns.explain | 0 .../groupby_agg_string.explain | 0 .../explain-results/groupby_avg.explain | 0 .../explain-results/groupby_count.explain | 0 .../explain-results/groupby_max.explain | 0 .../explain-results/groupby_mean.explain | 0 .../explain-results/groupby_min.explain | 0 .../explain-results/groupby_sum.explain | 0 .../explain-results/groupingSets.explain | 0 .../grouping_and_grouping_id.explain | 0 .../query-tests/explain-results/hint.explain | 0 .../explain-results/hll_sketch_agg.explain | 0 .../hll_sketch_agg_with_columnName.explain | 0 ..._agg_with_columnName_lgConfigK_int.explain | 0 ...l_sketch_agg_with_column_lgConfigK.explain | 0 ...etch_agg_with_column_lgConfigK_int.explain | 0 .../explain-results/hll_union_agg.explain | 0 .../hll_union_agg_with_columnName.explain | 0 ...me_allowDifferentLgConfigK_boolean.explain | 0 ...ith_column_allowDifferentLgConfigK.explain | 0 ...mn_allowDifferentLgConfigK_boolean.explain | 0 .../explain-results/intersect.explain | 0 .../explain-results/intersectAll.explain | 0 .../explain-results/join_condition.explain | 0 .../join_inner_condition.explain | 0 .../join_inner_no_condition.explain | 0 ...oin_inner_using_multiple_col_array.explain | 0 .../join_inner_using_multiple_col_seq.explain | 0 .../join_inner_using_single_col.explain | 0 .../join_using_multiple_col_array.explain | 0 .../join_using_multiple_col_seq.explain | 0 .../join_using_single_col.explain | 0 .../explain-results/json_from_dataset.explain | 0 .../query-tests/explain-results/limit.explain | 0 .../explain-results/melt_no_values.explain | 0 .../explain-results/melt_values.explain | 0 .../explain-results/offset.explain | 0 .../explain-results/orderBy_columns.explain | 0 .../explain-results/orderBy_strings.explain | 0 .../query-tests/explain-results/pivot.explain | 0 .../pivot_without_column_values.explain | 0 .../query-tests/explain-results/range.explain | 0 .../query-tests/explain-results/read.explain | 0 .../explain-results/read_csv.explain | 0 .../explain-results/read_jdbc.explain | 0 .../read_jdbc_with_partition.explain | 0 .../read_jdbc_with_predicates.explain | 0 .../explain-results/read_json.explain | 0 .../explain-results/read_orc.explain | 0 .../explain-results/read_parquet.explain | 0 .../explain-results/read_table.explain | 0 .../explain-results/read_text.explain | 0 .../relation_extension.explain | 0 .../relation_extension_deprecated.explain | 0 .../explain-results/repartition.explain | 0 .../repartitionByRange_expressions.explain | 0 ...ByRange_num_partitions_expressions.explain | 0 .../repartition_expressions.explain | 0 ...rtition_num_partitions_expressions.explain | 0 .../explain-results/replace.explain | 0 .../explain-results/rollup_column.explain | 0 .../explain-results/rollup_string.explain | 0 .../explain-results/sampleBy.explain | 0 .../sample_fraction_seed.explain | 0 ...mple_withReplacement_fraction_seed.explain | 0 .../explain-results/select.explain | 0 .../explain-results/selectExpr.explain | 0 .../select_collated_string.explain | 0 .../explain-results/select_strings.explain | 0 .../select_typed_1-arg.explain | 0 .../select_typed_2-arg.explain | 0 .../select_typed_3-arg.explain | 0 .../select_typed_4-arg.explain | 0 .../select_typed_5-arg.explain | 0 .../sortWithinPartitions_columns.explain | 0 .../sortWithinPartitions_strings.explain | 0 .../explain-results/sort_columns.explain | 0 .../explain-results/sort_strings.explain | 0 .../streaming_table_API_with_options.explain | 0 .../explain-results/summary.explain | 0 .../query-tests/explain-results/table.explain | 0 .../table_API_with_options.explain | 0 .../explain-results/test_broadcast.explain | 0 .../query-tests/explain-results/to.explain | 0 .../query-tests/explain-results/toDF.explain | 0 .../explain-results/toJSON.explain | 0 .../to_avro_with_schema.explain | 0 .../to_avro_without_schema.explain | 0 .../to_protobuf_messageClassName.explain | 0 ...obuf_messageClassName_descFilePath.explain | 0 ...sageClassName_descFilePath_options.explain | 0 ..._protobuf_messageClassName_options.explain | 0 .../query-tests/explain-results/union.explain | 0 .../explain-results/unionAll.explain | 0 .../explain-results/unionByName.explain | 0 .../unionByName_allowMissingColumns.explain | 0 .../explain-results/unpivot_no_values.explain | 0 .../explain-results/unpivot_values.explain | 0 .../explain-results/where_column.explain | 0 .../explain-results/where_expr.explain | 0 .../explain-results/width_bucket.explain | 0 .../explain-results/window.explain | 0 .../withColumnRenamed_java_map.explain | 0 .../withColumnRenamed_scala_map.explain | 0 .../withColumnRenamed_single.explain | 0 .../explain-results/withColumn_single.explain | 0 .../withColumns_java_map.explain | 0 .../withColumns_scala_map.explain | 0 .../explain-results/withMetadata.explain | 0 .../query-tests/queries/alias_string.json | 0 .../queries/alias_string.proto.bin | Bin .../query-tests/queries/alias_symbol.json | 0 .../queries/alias_symbol.proto.bin | Bin .../resources/query-tests/queries/apply.json | 0 .../query-tests/queries/apply.proto.bin | Bin .../query-tests/queries/as_string.json | 0 .../query-tests/queries/as_string.proto.bin | Bin .../query-tests/queries/as_symbol.json | 0 .../query-tests/queries/as_symbol.proto.bin | Bin .../query-tests/queries/between_expr.json | 0 .../queries/between_expr.proto.bin | Bin .../query-tests/queries/coalesce.json | 0 .../query-tests/queries/coalesce.proto.bin | Bin .../resources/query-tests/queries/col.json | 0 .../query-tests/queries/col.proto.bin | Bin .../query-tests/queries/colRegex.json | 0 .../query-tests/queries/colRegex.proto.bin | Bin .../query-tests/queries/column_add.json | 0 .../query-tests/queries/column_add.proto.bin | Bin .../query-tests/queries/column_alias.json | 0 .../queries/column_alias.proto.bin | Bin .../query-tests/queries/column_and.json | 0 .../query-tests/queries/column_and.proto.bin | Bin .../query-tests/queries/column_apply.json | 0 .../queries/column_apply.proto.bin | Bin .../query-tests/queries/column_as_multi.json | 0 .../queries/column_as_multi.proto.bin | Bin .../queries/column_as_with_metadata.json | 0 .../queries/column_as_with_metadata.proto.bin | Bin .../query-tests/queries/column_asc.json | 0 .../query-tests/queries/column_asc.proto.bin | Bin .../queries/column_asc_nulls_first.json | 0 .../queries/column_asc_nulls_first.proto.bin | Bin .../queries/column_asc_nulls_last.json | 0 .../queries/column_asc_nulls_last.proto.bin | Bin .../query-tests/queries/column_between.json | 0 .../queries/column_between.proto.bin | Bin .../queries/column_bitwiseAND.json | 0 .../queries/column_bitwiseAND.proto.bin | Bin .../query-tests/queries/column_bitwiseOR.json | 0 .../queries/column_bitwiseOR.proto.bin | Bin .../queries/column_bitwiseXOR.json | 0 .../queries/column_bitwiseXOR.proto.bin | Bin .../query-tests/queries/column_cast.json | 0 .../query-tests/queries/column_cast.proto.bin | Bin .../query-tests/queries/column_contains.json | 0 .../queries/column_contains.proto.bin | Bin .../query-tests/queries/column_desc.json | 0 .../query-tests/queries/column_desc.proto.bin | Bin .../queries/column_desc_nulls_first.json | 0 .../queries/column_desc_nulls_first.proto.bin | Bin .../queries/column_desc_nulls_last.json | 0 .../queries/column_desc_nulls_last.proto.bin | Bin .../query-tests/queries/column_divide.json | 0 .../queries/column_divide.proto.bin | Bin .../queries/column_dropFields.json | 0 .../queries/column_dropFields.proto.bin | Bin .../query-tests/queries/column_endsWith.json | 0 .../queries/column_endsWith.proto.bin | Bin .../queries/column_eqNullSafe.json | 0 .../queries/column_eqNullSafe.proto.bin | Bin .../query-tests/queries/column_equals.json | 0 .../queries/column_equals.proto.bin | Bin .../query-tests/queries/column_geq.json | 0 .../query-tests/queries/column_geq.proto.bin | Bin .../query-tests/queries/column_getField.json | 0 .../queries/column_getField.proto.bin | Bin .../query-tests/queries/column_getItem.json | 0 .../queries/column_getItem.proto.bin | Bin .../query-tests/queries/column_gt.json | 0 .../query-tests/queries/column_gt.proto.bin | Bin .../query-tests/queries/column_ilike.json | 0 .../queries/column_ilike.proto.bin | Bin .../query-tests/queries/column_isNaN.json | 0 .../queries/column_isNaN.proto.bin | Bin .../query-tests/queries/column_isNotNull.json | 0 .../queries/column_isNotNull.proto.bin | Bin .../query-tests/queries/column_isNull.json | 0 .../queries/column_isNull.proto.bin | Bin .../query-tests/queries/column_isin.json | 0 .../query-tests/queries/column_isin.proto.bin | Bin .../query-tests/queries/column_leq.json | 0 .../query-tests/queries/column_leq.proto.bin | Bin .../query-tests/queries/column_like.json | 0 .../query-tests/queries/column_like.proto.bin | Bin .../query-tests/queries/column_lt.json | 0 .../query-tests/queries/column_lt.proto.bin | Bin .../query-tests/queries/column_modulo.json | 0 .../queries/column_modulo.proto.bin | Bin .../query-tests/queries/column_multiply.json | 0 .../queries/column_multiply.proto.bin | Bin .../query-tests/queries/column_not.json | 0 .../query-tests/queries/column_not.proto.bin | Bin .../queries/column_not_equals.json | 0 .../queries/column_not_equals.proto.bin | Bin .../query-tests/queries/column_or.json | 0 .../query-tests/queries/column_or.proto.bin | Bin .../query-tests/queries/column_rlike.json | 0 .../queries/column_rlike.proto.bin | Bin .../query-tests/queries/column_star.json | 0 .../query-tests/queries/column_star.proto.bin | Bin .../queries/column_star_with_target.json | 0 .../queries/column_star_with_target.proto.bin | Bin .../queries/column_startsWith.json | 0 .../queries/column_startsWith.proto.bin | Bin .../query-tests/queries/column_substr.json | 0 .../queries/column_substr.proto.bin | Bin .../query-tests/queries/column_subtract.json | 0 .../queries/column_subtract.proto.bin | Bin .../query-tests/queries/column_try_cast.json | 0 .../queries/column_try_cast.proto.bin | Bin .../queries/column_unary_minus.json | 0 .../queries/column_unary_minus.proto.bin | Bin .../queries/column_when_otherwise.json | 0 .../queries/column_when_otherwise.proto.bin | Bin .../query-tests/queries/column_withField.json | 0 .../queries/column_withField.proto.bin | Bin .../query-tests/queries/crossJoin.json | 0 .../query-tests/queries/crossJoin.proto.bin | Bin .../query-tests/queries/crosstab.json | 0 .../query-tests/queries/crosstab.proto.bin | Bin .../query-tests/queries/csv_from_dataset.json | 0 .../queries/csv_from_dataset.proto.bin | Bin .../query-tests/queries/cube_column.json | 0 .../query-tests/queries/cube_column.proto.bin | Bin .../query-tests/queries/cube_string.json | 0 .../query-tests/queries/cube_string.proto.bin | Bin .../query-tests/queries/describe.json | 0 .../query-tests/queries/describe.proto.bin | Bin .../query-tests/queries/distinct.json | 0 .../query-tests/queries/distinct.proto.bin | Bin .../resources/query-tests/queries/drop.json | 0 .../query-tests/queries/drop.proto.bin | Bin .../query-tests/queries/dropDuplicates.json | 0 .../queries/dropDuplicates.proto.bin | Bin .../queries/dropDuplicates_names_array.json | 0 .../dropDuplicates_names_array.proto.bin | Bin .../queries/dropDuplicates_names_seq.json | 0 .../dropDuplicates_names_seq.proto.bin | Bin .../queries/dropDuplicates_varargs.json | 0 .../queries/dropDuplicates_varargs.proto.bin | Bin .../queries/drop_multiple_column.json | 0 .../queries/drop_multiple_column.proto.bin | Bin .../queries/drop_multiple_strings.json | 0 .../queries/drop_multiple_strings.proto.bin | Bin .../queries/drop_single_column.json | 0 .../queries/drop_single_column.proto.bin | Bin .../queries/drop_single_string.json | 0 .../queries/drop_single_string.proto.bin | Bin .../resources/query-tests/queries/except.json | 0 .../query-tests/queries/except.proto.bin | Bin .../query-tests/queries/exceptAll.json | 0 .../query-tests/queries/exceptAll.proto.bin | Bin .../queries/expression_extension.json | 0 .../queries/expression_extension.proto.bin | Bin .../expression_extension_deprecated.json | 0 .../expression_extension_deprecated.proto.bin | Bin .../resources/query-tests/queries/fill.json | 0 .../query-tests/queries/fill.proto.bin | Bin .../resources/query-tests/queries/filter.json | 0 .../query-tests/queries/filter.proto.bin | Bin .../query-tests/queries/filter_expr.json | 0 .../query-tests/queries/filter_expr.proto.bin | Bin .../query-tests/queries/freqItems.json | 0 .../query-tests/queries/freqItems.proto.bin | Bin .../queries/from_avro_with_options.json | 0 .../queries/from_avro_with_options.proto.bin | Bin .../queries/from_avro_without_options.json | 0 .../from_avro_without_options.proto.bin | Bin ...rotobuf_messageClassName_descFilePath.json | 0 ...uf_messageClassName_descFilePath.proto.bin | Bin ...messageClassName_descFilePath_options.json | 0 ...geClassName_descFilePath_options.proto.bin | Bin .../query-tests/queries/function_abs.json | 0 .../queries/function_abs.proto.bin | Bin .../query-tests/queries/function_acos.json | 0 .../queries/function_acos.proto.bin | Bin .../query-tests/queries/function_acosh.json | 0 .../queries/function_acosh.proto.bin | Bin .../queries/function_add_months.json | 0 .../queries/function_add_months.proto.bin | Bin .../queries/function_aes_decrypt.json | 0 .../queries/function_aes_decrypt.proto.bin | Bin .../function_aes_decrypt_with_mode.json | 0 .../function_aes_decrypt_with_mode.proto.bin | Bin ...unction_aes_decrypt_with_mode_padding.json | 0 ...on_aes_decrypt_with_mode_padding.proto.bin | Bin ...ion_aes_decrypt_with_mode_padding_aad.json | 0 ...es_decrypt_with_mode_padding_aad.proto.bin | Bin .../queries/function_aes_encrypt.json | 0 .../queries/function_aes_encrypt.proto.bin | Bin .../function_aes_encrypt_with_mode.json | 0 .../function_aes_encrypt_with_mode.proto.bin | Bin ...unction_aes_encrypt_with_mode_padding.json | 0 ...on_aes_encrypt_with_mode_padding.proto.bin | Bin ...tion_aes_encrypt_with_mode_padding_iv.json | 0 ...aes_encrypt_with_mode_padding_iv.proto.bin | Bin ..._aes_encrypt_with_mode_padding_iv_aad.json | 0 ...encrypt_with_mode_padding_iv_aad.proto.bin | Bin .../queries/function_aggregate.json | 0 .../queries/function_aggregate.proto.bin | Bin .../query-tests/queries/function_any.json | 0 .../queries/function_any.proto.bin | Bin .../function_any_value_with_ignore_nulls.json | 0 ...tion_any_value_with_ignore_nulls.proto.bin | Bin ...function_any_value_with_respect_nulls.json | 0 ...ion_any_value_with_respect_nulls.proto.bin | Bin .../function_approx_count_distinct.json | 0 .../function_approx_count_distinct.proto.bin | Bin .../function_approx_count_distinct_rsd.json | 0 ...nction_approx_count_distinct_rsd.proto.bin | Bin .../queries/function_approx_percentile.json | 0 .../function_approx_percentile.proto.bin | Bin .../query-tests/queries/function_array.json | 0 .../queries/function_array.proto.bin | Bin .../queries/function_array_agg.json | 0 .../queries/function_array_agg.proto.bin | Bin .../queries/function_array_append.json | 0 .../queries/function_array_append.proto.bin | Bin .../queries/function_array_compact.json | 0 .../queries/function_array_compact.proto.bin | Bin .../queries/function_array_contains.json | 0 .../queries/function_array_contains.proto.bin | Bin .../queries/function_array_distinct.json | 0 .../queries/function_array_distinct.proto.bin | Bin .../queries/function_array_except.json | 0 .../queries/function_array_except.proto.bin | Bin .../queries/function_array_insert.json | 0 .../queries/function_array_insert.proto.bin | Bin .../queries/function_array_intersect.json | 0 .../function_array_intersect.proto.bin | Bin .../queries/function_array_join.json | 0 .../queries/function_array_join.proto.bin | Bin ...tion_array_join_with_null_replacement.json | 0 ...array_join_with_null_replacement.proto.bin | Bin .../queries/function_array_max.json | 0 .../queries/function_array_max.proto.bin | Bin .../queries/function_array_min.json | 0 .../queries/function_array_min.proto.bin | Bin .../queries/function_array_position.json | 0 .../queries/function_array_position.proto.bin | Bin .../queries/function_array_prepend.json | 0 .../queries/function_array_prepend.proto.bin | Bin .../queries/function_array_remove.json | 0 .../queries/function_array_remove.proto.bin | Bin .../queries/function_array_repeat.json | 0 .../queries/function_array_repeat.proto.bin | Bin .../queries/function_array_size.json | 0 .../queries/function_array_size.proto.bin | Bin .../queries/function_array_sort.json | 0 .../queries/function_array_sort.proto.bin | Bin .../function_array_sort_with_comparator.json | 0 ...ction_array_sort_with_comparator.proto.bin | Bin .../queries/function_array_union.json | 0 .../queries/function_array_union.proto.bin | Bin .../queries/function_arrays_overlap.json | 0 .../queries/function_arrays_overlap.proto.bin | Bin .../queries/function_arrays_zip.json | 0 .../queries/function_arrays_zip.proto.bin | Bin .../query-tests/queries/function_asc.json | 0 .../queries/function_asc.proto.bin | Bin .../queries/function_asc_nulls_first.json | 0 .../function_asc_nulls_first.proto.bin | Bin .../queries/function_asc_nulls_last.json | 0 .../queries/function_asc_nulls_last.proto.bin | Bin .../query-tests/queries/function_ascii.json | 0 .../queries/function_ascii.proto.bin | Bin .../query-tests/queries/function_asin.json | 0 .../queries/function_asin.proto.bin | Bin .../query-tests/queries/function_asinh.json | 0 .../queries/function_asinh.proto.bin | Bin .../function_assert_true_with_message.json | 0 ...unction_assert_true_with_message.proto.bin | Bin .../query-tests/queries/function_atan.json | 0 .../queries/function_atan.proto.bin | Bin .../query-tests/queries/function_atan2.json | 0 .../queries/function_atan2.proto.bin | Bin .../query-tests/queries/function_atanh.json | 0 .../queries/function_atanh.proto.bin | Bin .../query-tests/queries/function_avg.json | 0 .../queries/function_avg.proto.bin | Bin .../query-tests/queries/function_base64.json | 0 .../queries/function_base64.proto.bin | Bin .../query-tests/queries/function_bin.json | 0 .../queries/function_bin.proto.bin | Bin .../query-tests/queries/function_bit_and.json | 0 .../queries/function_bit_and.proto.bin | Bin .../queries/function_bit_count.json | 0 .../queries/function_bit_count.proto.bin | Bin .../query-tests/queries/function_bit_get.json | 0 .../queries/function_bit_get.proto.bin | Bin .../queries/function_bit_length.json | 0 .../queries/function_bit_length.proto.bin | Bin .../query-tests/queries/function_bit_or.json | 0 .../queries/function_bit_or.proto.bin | Bin .../query-tests/queries/function_bit_xor.json | 0 .../queries/function_bit_xor.proto.bin | Bin .../queries/function_bitmap_bit_position.json | 0 .../function_bitmap_bit_position.proto.bin | Bin .../function_bitmap_bucket_number.json | 0 .../function_bitmap_bucket_number.proto.bin | Bin .../function_bitmap_construct_agg.json | 0 .../function_bitmap_construct_agg.proto.bin | Bin .../queries/function_bitmap_count.json | 0 .../queries/function_bitmap_count.proto.bin | Bin .../queries/function_bitmap_or_agg.json | 0 .../queries/function_bitmap_or_agg.proto.bin | Bin .../queries/function_bitwise_not.json | 0 .../queries/function_bitwise_not.proto.bin | Bin .../queries/function_bool_and.json | 0 .../queries/function_bool_and.proto.bin | Bin .../query-tests/queries/function_bool_or.json | 0 .../queries/function_bool_or.proto.bin | Bin .../query-tests/queries/function_bround.json | 0 .../queries/function_bround.proto.bin | Bin .../query-tests/queries/function_btrim.json | 0 .../queries/function_btrim.proto.bin | Bin ...tion_btrim_with_specified_trim_string.json | 0 ...btrim_with_specified_trim_string.proto.bin | Bin .../query-tests/queries/function_bucket.json | 0 .../queries/function_bucket.proto.bin | Bin .../queries/function_call_function.json | 0 .../queries/function_call_function.proto.bin | Bin .../queries/function_cardinality.json | 0 .../queries/function_cardinality.proto.bin | Bin .../query-tests/queries/function_ceil.json | 0 .../queries/function_ceil.proto.bin | Bin .../queries/function_ceil_scale.json | 0 .../queries/function_ceil_scale.proto.bin | Bin .../query-tests/queries/function_ceiling.json | 0 .../queries/function_ceiling.proto.bin | Bin .../queries/function_ceiling_scale.json | 0 .../queries/function_ceiling_scale.proto.bin | Bin .../query-tests/queries/function_char.json | 0 .../queries/function_char.proto.bin | Bin .../queries/function_char_length.json | 0 .../queries/function_char_length.proto.bin | Bin .../queries/function_character_length.json | 0 .../function_character_length.proto.bin | Bin .../query-tests/queries/function_chr.json | 0 .../queries/function_chr.proto.bin | Bin .../queries/function_coalesce.json | 0 .../queries/function_coalesce.proto.bin | Bin .../query-tests/queries/function_col.json | 0 .../queries/function_col.proto.bin | Bin .../query-tests/queries/function_collate.json | 0 .../queries/function_collate.proto.bin | Bin .../queries/function_collation.json | 0 .../queries/function_collation.proto.bin | Bin .../queries/function_collect_list.json | 0 .../queries/function_collect_list.proto.bin | Bin .../queries/function_collect_set.json | 0 .../queries/function_collect_set.proto.bin | Bin .../query-tests/queries/function_concat.json | 0 .../queries/function_concat.proto.bin | Bin .../queries/function_concat_ws.json | 0 .../queries/function_concat_ws.proto.bin | Bin .../queries/function_contains.json | 0 .../queries/function_contains.proto.bin | Bin .../query-tests/queries/function_conv.json | 0 .../queries/function_conv.proto.bin | Bin ...onvert_timezone_with_source_time_zone.json | 0 ...t_timezone_with_source_time_zone.proto.bin | Bin ...ert_timezone_without_source_time_zone.json | 0 ...imezone_without_source_time_zone.proto.bin | Bin .../query-tests/queries/function_corr.json | 0 .../queries/function_corr.proto.bin | Bin .../query-tests/queries/function_cos.json | 0 .../queries/function_cos.proto.bin | Bin .../query-tests/queries/function_cosh.json | 0 .../queries/function_cosh.proto.bin | Bin .../query-tests/queries/function_cot.json | 0 .../queries/function_cot.proto.bin | Bin .../query-tests/queries/function_count.json | 0 .../queries/function_count.proto.bin | Bin .../queries/function_countDistinct.json | 0 .../queries/function_countDistinct.proto.bin | Bin .../queries/function_count_if.json | 0 .../queries/function_count_if.proto.bin | Bin .../queries/function_count_min_sketch.json | 0 .../function_count_min_sketch.proto.bin | Bin .../queries/function_count_typed.json | 0 .../queries/function_count_typed.proto.bin | Bin .../queries/function_covar_pop.json | 0 .../queries/function_covar_pop.proto.bin | Bin .../queries/function_covar_samp.json | 0 .../queries/function_covar_samp.proto.bin | Bin .../query-tests/queries/function_crc32.json | 0 .../queries/function_crc32.proto.bin | Bin .../query-tests/queries/function_csc.json | 0 .../queries/function_csc.proto.bin | Bin .../queries/function_cume_dist.json | 0 .../queries/function_cume_dist.proto.bin | Bin .../query-tests/queries/function_curdate.json | 0 .../queries/function_curdate.proto.bin | Bin .../queries/function_current_catalog.json | 0 .../function_current_catalog.proto.bin | Bin .../queries/function_current_database.json | 0 .../function_current_database.proto.bin | Bin .../queries/function_current_date.json | 0 .../queries/function_current_date.proto.bin | Bin .../queries/function_current_schema.json | 0 .../queries/function_current_schema.proto.bin | Bin .../queries/function_current_timestamp.json | 0 .../function_current_timestamp.proto.bin | Bin .../queries/function_current_timezone.json | 0 .../function_current_timezone.proto.bin | Bin .../queries/function_current_user.json | 0 .../queries/function_current_user.proto.bin | Bin .../queries/function_date_add.json | 0 .../queries/function_date_add.proto.bin | Bin .../queries/function_date_diff.json | 0 .../queries/function_date_diff.proto.bin | Bin .../queries/function_date_format.json | 0 .../queries/function_date_format.proto.bin | Bin .../queries/function_date_from_unix_date.json | 0 .../function_date_from_unix_date.proto.bin | Bin .../queries/function_date_part.json | 0 .../queries/function_date_part.proto.bin | Bin .../queries/function_date_sub.json | 0 .../queries/function_date_sub.proto.bin | Bin .../queries/function_date_trunc.json | 0 .../queries/function_date_trunc.proto.bin | Bin .../query-tests/queries/function_dateadd.json | 0 .../queries/function_dateadd.proto.bin | Bin .../queries/function_datediff.json | 0 .../queries/function_datediff.proto.bin | Bin .../queries/function_datepart.json | 0 .../queries/function_datepart.proto.bin | Bin .../query-tests/queries/function_day.json | 0 .../queries/function_day.proto.bin | Bin .../query-tests/queries/function_dayname.json | 0 .../queries/function_dayname.proto.bin | Bin .../queries/function_dayofmonth.json | 0 .../queries/function_dayofmonth.proto.bin | Bin .../queries/function_dayofweek.json | 0 .../queries/function_dayofweek.proto.bin | Bin .../queries/function_dayofyear.json | 0 .../queries/function_dayofyear.proto.bin | Bin .../query-tests/queries/function_days.json | 0 .../queries/function_days.proto.bin | Bin .../query-tests/queries/function_decode.json | 0 .../queries/function_decode.proto.bin | Bin .../query-tests/queries/function_degrees.json | 0 .../queries/function_degrees.proto.bin | Bin .../queries/function_dense_rank.json | 0 .../queries/function_dense_rank.proto.bin | Bin .../query-tests/queries/function_desc.json | 0 .../queries/function_desc.proto.bin | Bin .../queries/function_desc_nulls_first.json | 0 .../function_desc_nulls_first.proto.bin | Bin .../queries/function_desc_nulls_last.json | 0 .../function_desc_nulls_last.proto.bin | Bin .../query-tests/queries/function_e.json | 0 .../query-tests/queries/function_e.proto.bin | Bin .../queries/function_element_at.json | 0 .../queries/function_element_at.proto.bin | Bin .../query-tests/queries/function_elt.json | 0 .../queries/function_elt.proto.bin | Bin .../query-tests/queries/function_encode.json | 0 .../queries/function_encode.proto.bin | Bin .../queries/function_endswith.json | 0 .../queries/function_endswith.proto.bin | Bin .../queries/function_equal_null.json | 0 .../queries/function_equal_null.proto.bin | Bin .../query-tests/queries/function_every.json | 0 .../queries/function_every.proto.bin | Bin .../query-tests/queries/function_exists.json | 0 .../queries/function_exists.proto.bin | Bin .../query-tests/queries/function_exp.json | 0 .../queries/function_exp.proto.bin | Bin .../query-tests/queries/function_explode.json | 0 .../queries/function_explode.proto.bin | Bin .../queries/function_explode_outer.json | 0 .../queries/function_explode_outer.proto.bin | Bin .../query-tests/queries/function_expm1.json | 0 .../queries/function_expm1.proto.bin | Bin .../query-tests/queries/function_expr.json | 0 .../queries/function_expr.proto.bin | Bin .../query-tests/queries/function_extract.json | 0 .../queries/function_extract.proto.bin | Bin .../queries/function_factorial.json | 0 .../queries/function_factorial.proto.bin | Bin .../query-tests/queries/function_filter.json | 0 .../queries/function_filter.proto.bin | Bin .../function_filter_with_pair_input.json | 0 .../function_filter_with_pair_input.proto.bin | Bin .../queries/function_find_in_set.json | 0 .../queries/function_find_in_set.proto.bin | Bin ...unction_first_value_with_ignore_nulls.json | 0 ...on_first_value_with_ignore_nulls.proto.bin | Bin ...nction_first_value_with_respect_nulls.json | 0 ...n_first_value_with_respect_nulls.proto.bin | Bin .../function_first_with_ignore_nulls.json | 0 ...function_first_with_ignore_nulls.proto.bin | Bin .../function_first_with_respect_nulls.json | 0 ...unction_first_with_respect_nulls.proto.bin | Bin .../query-tests/queries/function_flatten.json | 0 .../queries/function_flatten.proto.bin | Bin .../query-tests/queries/function_floor.json | 0 .../queries/function_floor.proto.bin | Bin .../queries/function_floor_scale.json | 0 .../queries/function_floor_scale.proto.bin | Bin .../query-tests/queries/function_forall.json | 0 .../queries/function_forall.proto.bin | Bin .../queries/function_format_number.json | 0 .../queries/function_format_number.proto.bin | Bin .../queries/function_from_csv.json | 0 .../queries/function_from_csv.proto.bin | Bin .../queries/function_from_json.json | 0 .../queries/function_from_json.proto.bin | Bin .../queries/function_from_unixtime.json | 0 .../queries/function_from_unixtime.proto.bin | Bin .../queries/function_from_utc_timestamp.json | 0 .../function_from_utc_timestamp.proto.bin | Bin .../query-tests/queries/function_get.json | 0 .../queries/function_get.proto.bin | Bin .../queries/function_get_json_object.json | 0 .../function_get_json_object.proto.bin | Bin .../query-tests/queries/function_getbit.json | 0 .../queries/function_getbit.proto.bin | Bin .../queries/function_greatest.json | 0 .../queries/function_greatest.proto.bin | Bin .../query-tests/queries/function_hash.json | 0 .../queries/function_hash.proto.bin | Bin .../query-tests/queries/function_hex.json | 0 .../queries/function_hex.proto.bin | Bin .../queries/function_histogram_numeric.json | 0 .../function_histogram_numeric.proto.bin | Bin .../query-tests/queries/function_hour.json | 0 .../queries/function_hour.proto.bin | Bin .../query-tests/queries/function_hours.json | 0 .../queries/function_hours.proto.bin | Bin .../query-tests/queries/function_hypot.json | 0 .../queries/function_hypot.proto.bin | Bin .../query-tests/queries/function_ifnull.json | 0 .../queries/function_ifnull.proto.bin | Bin .../query-tests/queries/function_ilike.json | 0 .../queries/function_ilike.proto.bin | Bin .../queries/function_ilike_with_escape.json | 0 .../function_ilike_with_escape.proto.bin | Bin .../query-tests/queries/function_initcap.json | 0 .../queries/function_initcap.proto.bin | Bin .../query-tests/queries/function_inline.json | 0 .../queries/function_inline.proto.bin | Bin .../queries/function_inline_outer.json | 0 .../queries/function_inline_outer.proto.bin | Bin .../function_input_file_block_length.json | 0 ...function_input_file_block_length.proto.bin | Bin .../function_input_file_block_start.json | 0 .../function_input_file_block_start.proto.bin | Bin .../queries/function_input_file_name.json | 0 .../function_input_file_name.proto.bin | Bin .../queries/function_is_variant_null.json | 0 .../function_is_variant_null.proto.bin | Bin .../query-tests/queries/function_isnan.json | 0 .../queries/function_isnan.proto.bin | Bin .../queries/function_isnotnull.json | 0 .../queries/function_isnotnull.proto.bin | Bin .../query-tests/queries/function_isnull.json | 0 .../queries/function_isnull.proto.bin | Bin .../queries/function_java_method.json | 0 .../queries/function_java_method.proto.bin | Bin .../queries/function_json_array_length.json | 0 .../function_json_array_length.proto.bin | Bin .../queries/function_json_object_keys.json | 0 .../function_json_object_keys.proto.bin | Bin .../queries/function_json_tuple.json | 0 .../queries/function_json_tuple.proto.bin | Bin .../queries/function_kurtosis.json | 0 .../queries/function_kurtosis.proto.bin | Bin .../query-tests/queries/function_lag.json | 0 .../queries/function_lag.proto.bin | Bin .../queries/function_last_day.json | 0 .../queries/function_last_day.proto.bin | Bin ...function_last_value_with_ignore_nulls.json | 0 ...ion_last_value_with_ignore_nulls.proto.bin | Bin ...unction_last_value_with_respect_nulls.json | 0 ...on_last_value_with_respect_nulls.proto.bin | Bin .../function_last_with_ignore_nulls.json | 0 .../function_last_with_ignore_nulls.proto.bin | Bin .../function_last_with_respect_nulls.json | 0 ...function_last_with_respect_nulls.proto.bin | Bin .../query-tests/queries/function_lcase.json | 0 .../queries/function_lcase.proto.bin | Bin .../query-tests/queries/function_lead.json | 0 .../queries/function_lead.proto.bin | Bin .../query-tests/queries/function_least.json | 0 .../queries/function_least.proto.bin | Bin .../query-tests/queries/function_left.json | 0 .../queries/function_left.proto.bin | Bin .../query-tests/queries/function_len.json | 0 .../queries/function_len.proto.bin | Bin .../query-tests/queries/function_length.json | 0 .../queries/function_length.proto.bin | Bin .../queries/function_levenshtein.json | 0 .../queries/function_levenshtein.proto.bin | Bin .../function_levenshtein_with_threshold.json | 0 ...ction_levenshtein_with_threshold.proto.bin | Bin .../query-tests/queries/function_like.json | 0 .../queries/function_like.proto.bin | Bin .../queries/function_like_with_escape.json | 0 .../function_like_with_escape.proto.bin | Bin .../query-tests/queries/function_lit.json | 0 .../queries/function_lit.proto.bin | Bin .../queries/function_lit_array.json | 0 .../queries/function_lit_array.proto.bin | Bin .../query-tests/queries/function_ln.json | 0 .../query-tests/queries/function_ln.proto.bin | Bin .../queries/function_localtimestamp.json | 0 .../queries/function_localtimestamp.proto.bin | Bin .../query-tests/queries/function_locate.json | 0 .../queries/function_locate.proto.bin | Bin .../queries/function_locate_with_pos.json | 0 .../function_locate_with_pos.proto.bin | Bin .../query-tests/queries/function_log.json | 0 .../queries/function_log.proto.bin | Bin .../query-tests/queries/function_log10.json | 0 .../queries/function_log10.proto.bin | Bin .../query-tests/queries/function_log1p.json | 0 .../queries/function_log1p.proto.bin | Bin .../query-tests/queries/function_log2.json | 0 .../queries/function_log2.proto.bin | Bin .../queries/function_log_with_base.json | 0 .../queries/function_log_with_base.proto.bin | Bin .../query-tests/queries/function_lower.json | 0 .../queries/function_lower.proto.bin | Bin .../query-tests/queries/function_lpad.json | 0 .../queries/function_lpad.proto.bin | Bin .../queries/function_lpad_binary.json | 0 .../queries/function_lpad_binary.proto.bin | Bin .../query-tests/queries/function_ltrim.json | 0 .../queries/function_ltrim.proto.bin | Bin .../queries/function_ltrim_with_pattern.json | 0 .../function_ltrim_with_pattern.proto.bin | Bin .../queries/function_make_date.json | 0 .../queries/function_make_date.proto.bin | Bin .../queries/function_make_dt_interval.json | 0 .../function_make_dt_interval.proto.bin | Bin .../function_make_dt_interval_days.json | 0 .../function_make_dt_interval_days.proto.bin | Bin .../function_make_dt_interval_days_hours.json | 0 ...tion_make_dt_interval_days_hours.proto.bin | Bin ...tion_make_dt_interval_days_hours_mins.json | 0 ...make_dt_interval_days_hours_mins.proto.bin | Bin ...make_dt_interval_days_hours_mins_secs.json | 0 ...dt_interval_days_hours_mins_secs.proto.bin | Bin .../queries/function_make_interval.json | 0 .../queries/function_make_interval.proto.bin | Bin .../queries/function_make_interval_years.json | 0 .../function_make_interval_years.proto.bin | Bin .../function_make_interval_years_months.json | 0 ...ction_make_interval_years_months.proto.bin | Bin ...tion_make_interval_years_months_weeks.json | 0 ...make_interval_years_months_weeks.proto.bin | Bin ...make_interval_years_months_weeks_days.json | 0 ...interval_years_months_weeks_days.proto.bin | Bin ...nterval_years_months_weeks_days_hours.json | 0 ...al_years_months_weeks_days_hours.proto.bin | Bin ...al_years_months_weeks_days_hours_mins.json | 0 ...ars_months_weeks_days_hours_mins.proto.bin | Bin ...ars_months_weeks_days_hours_mins_secs.json | 0 ...onths_weeks_days_hours_mins_secs.proto.bin | Bin ...tion_make_timestamp_ltz_with_timezone.json | 0 ...make_timestamp_ltz_with_timezone.proto.bin | Bin ...n_make_timestamp_ltz_without_timezone.json | 0 ...e_timestamp_ltz_without_timezone.proto.bin | Bin .../queries/function_make_timestamp_ntz.json | 0 .../function_make_timestamp_ntz.proto.bin | Bin ...function_make_timestamp_with_timezone.json | 0 ...ion_make_timestamp_with_timezone.proto.bin | Bin ...ction_make_timestamp_without_timezone.json | 0 ..._make_timestamp_without_timezone.proto.bin | Bin .../queries/function_make_ym_interval.json | 0 .../function_make_ym_interval.proto.bin | Bin .../function_make_ym_interval_years.json | 0 .../function_make_ym_interval_years.proto.bin | Bin ...unction_make_ym_interval_years_months.json | 0 ...on_make_ym_interval_years_months.proto.bin | Bin .../query-tests/queries/function_map.json | 0 .../queries/function_map.proto.bin | Bin .../queries/function_map_concat.json | 0 .../queries/function_map_concat.proto.bin | Bin .../queries/function_map_contains_key.json | 0 .../function_map_contains_key.proto.bin | Bin .../queries/function_map_entries.json | 0 .../queries/function_map_entries.proto.bin | Bin .../queries/function_map_filter.json | 0 .../queries/function_map_filter.proto.bin | Bin .../queries/function_map_from_arrays.json | 0 .../function_map_from_arrays.proto.bin | Bin .../queries/function_map_from_entries.json | 0 .../function_map_from_entries.proto.bin | Bin .../queries/function_map_keys.json | 0 .../queries/function_map_keys.proto.bin | Bin .../queries/function_map_values.json | 0 .../queries/function_map_values.proto.bin | Bin .../queries/function_map_zip_with.json | 0 .../queries/function_map_zip_with.proto.bin | Bin .../query-tests/queries/function_mask.json | 0 .../queries/function_mask.proto.bin | Bin ...function_mask_with_specific_upperChar.json | 0 ...ion_mask_with_specific_upperChar.proto.bin | Bin ...ask_with_specific_upperChar_lowerChar.json | 0 ...ith_specific_upperChar_lowerChar.proto.bin | Bin ...pecific_upperChar_lowerChar_digitChar.json | 0 ...ic_upperChar_lowerChar_digitChar.proto.bin | Bin ...perChar_lowerChar_digitChar_otherChar.json | 0 ...ar_lowerChar_digitChar_otherChar.proto.bin | Bin .../query-tests/queries/function_max.json | 0 .../queries/function_max.proto.bin | Bin .../query-tests/queries/function_max_by.json | 0 .../queries/function_max_by.proto.bin | Bin .../query-tests/queries/function_md5.json | 0 .../queries/function_md5.proto.bin | Bin .../query-tests/queries/function_median.json | 0 .../queries/function_median.proto.bin | Bin .../query-tests/queries/function_min.json | 0 .../queries/function_min.proto.bin | Bin .../query-tests/queries/function_min_by.json | 0 .../queries/function_min_by.proto.bin | Bin .../query-tests/queries/function_minute.json | 0 .../queries/function_minute.proto.bin | Bin .../query-tests/queries/function_mode.json | 0 .../queries/function_mode.proto.bin | Bin .../function_monotonically_increasing_id.json | 0 ...tion_monotonically_increasing_id.proto.bin | Bin .../query-tests/queries/function_month.json | 0 .../queries/function_month.proto.bin | Bin .../queries/function_monthname.json | 0 .../queries/function_monthname.proto.bin | Bin .../query-tests/queries/function_months.json | 0 .../queries/function_months.proto.bin | Bin .../queries/function_months_between.json | 0 .../queries/function_months_between.proto.bin | Bin ...function_months_between_with_roundoff.json | 0 ...ion_months_between_with_roundoff.proto.bin | Bin .../queries/function_named_struct.json | 0 .../queries/function_named_struct.proto.bin | Bin .../query-tests/queries/function_nanvl.json | 0 .../queries/function_nanvl.proto.bin | Bin .../query-tests/queries/function_negate.json | 0 .../queries/function_negate.proto.bin | Bin .../queries/function_negative.json | 0 .../queries/function_negative.proto.bin | Bin .../queries/function_next_day.json | 0 .../queries/function_next_day.proto.bin | Bin .../query-tests/queries/function_now.json | 0 .../queries/function_now.proto.bin | Bin .../queries/function_nth_value.json | 0 .../queries/function_nth_value.proto.bin | Bin .../query-tests/queries/function_ntile.json | 0 .../queries/function_ntile.proto.bin | Bin .../query-tests/queries/function_nullif.json | 0 .../queries/function_nullif.proto.bin | Bin .../query-tests/queries/function_nvl.json | 0 .../queries/function_nvl.proto.bin | Bin .../query-tests/queries/function_nvl2.json | 0 .../queries/function_nvl2.proto.bin | Bin .../queries/function_octet_length.json | 0 .../queries/function_octet_length.proto.bin | Bin .../query-tests/queries/function_overlay.json | 0 .../queries/function_overlay.proto.bin | Bin .../queries/function_overlay_with_len.json | 0 .../function_overlay_with_len.proto.bin | Bin .../queries/function_parse_json.json | 0 .../queries/function_parse_json.proto.bin | Bin .../queries/function_parse_url.json | 0 .../queries/function_parse_url.proto.bin | Bin .../queries/function_parse_url_with_key.json | 0 .../function_parse_url_with_key.proto.bin | Bin .../queries/function_percent_rank.json | 0 .../queries/function_percent_rank.proto.bin | Bin .../queries/function_percentile_approx.json | 0 .../function_percentile_approx.proto.bin | Bin .../function_percentile_with_frequency.json | 0 ...nction_percentile_with_frequency.proto.bin | Bin ...function_percentile_without_frequency.json | 0 ...ion_percentile_without_frequency.proto.bin | Bin .../query-tests/queries/function_pi.json | 0 .../query-tests/queries/function_pi.proto.bin | Bin .../query-tests/queries/function_pmod.json | 0 .../queries/function_pmod.proto.bin | Bin .../queries/function_posexplode.json | 0 .../queries/function_posexplode.proto.bin | Bin .../queries/function_posexplode_outer.json | 0 .../function_posexplode_outer.proto.bin | Bin .../queries/function_position.json | 0 .../queries/function_position.proto.bin | Bin .../queries/function_position_with_start.json | 0 .../function_position_with_start.proto.bin | Bin .../queries/function_positive.json | 0 .../queries/function_positive.proto.bin | Bin .../query-tests/queries/function_pow.json | 0 .../queries/function_pow.proto.bin | Bin .../query-tests/queries/function_power.json | 0 .../queries/function_power.proto.bin | Bin .../query-tests/queries/function_printf.json | 0 .../queries/function_printf.proto.bin | Bin .../query-tests/queries/function_product.json | 0 .../queries/function_product.proto.bin | Bin .../query-tests/queries/function_quarter.json | 0 .../queries/function_quarter.proto.bin | Bin .../query-tests/queries/function_radians.json | 0 .../queries/function_radians.proto.bin | Bin .../queries/function_raise_error.json | 0 .../queries/function_raise_error.proto.bin | Bin .../queries/function_rand_with_seed.json | 0 .../queries/function_rand_with_seed.proto.bin | Bin .../queries/function_randn_with_seed.json | 0 .../function_randn_with_seed.proto.bin | Bin .../queries/function_random_with_seed.json | 0 .../function_random_with_seed.proto.bin | Bin .../query-tests/queries/function_rank.json | 0 .../queries/function_rank.proto.bin | Bin .../query-tests/queries/function_reduce.json | 0 .../queries/function_reduce.proto.bin | Bin .../query-tests/queries/function_reflect.json | 0 .../queries/function_reflect.proto.bin | Bin .../query-tests/queries/function_regexp.json | 0 .../queries/function_regexp.proto.bin | Bin .../queries/function_regexp_count.json | 0 .../queries/function_regexp_count.proto.bin | Bin .../queries/function_regexp_extract.json | 0 .../queries/function_regexp_extract.proto.bin | Bin ...xp_extract_all_with_regex_group_index.json | 0 ...tract_all_with_regex_group_index.proto.bin | Bin ...extract_all_without_regex_group_index.json | 0 ...ct_all_without_regex_group_index.proto.bin | Bin ...n_regexp_instr_with_regex_group_index.json | 0 ...exp_instr_with_regex_group_index.proto.bin | Bin ...egexp_instr_without_regex_group_index.json | 0 ..._instr_without_regex_group_index.proto.bin | Bin .../queries/function_regexp_like.json | 0 .../queries/function_regexp_like.proto.bin | Bin .../queries/function_regexp_replace.json | 0 .../queries/function_regexp_replace.proto.bin | Bin .../queries/function_regexp_substr.json | 0 .../queries/function_regexp_substr.proto.bin | Bin .../queries/function_regr_avgx.json | 0 .../queries/function_regr_avgx.proto.bin | Bin .../queries/function_regr_avgy.json | 0 .../queries/function_regr_avgy.proto.bin | Bin .../queries/function_regr_count.json | 0 .../queries/function_regr_count.proto.bin | Bin .../queries/function_regr_intercept.json | 0 .../queries/function_regr_intercept.proto.bin | Bin .../query-tests/queries/function_regr_r2.json | 0 .../queries/function_regr_r2.proto.bin | Bin .../queries/function_regr_slope.json | 0 .../queries/function_regr_slope.proto.bin | Bin .../queries/function_regr_sxx.json | 0 .../queries/function_regr_sxx.proto.bin | Bin .../queries/function_regr_sxy.json | 0 .../queries/function_regr_sxy.proto.bin | Bin .../queries/function_regr_syy.json | 0 .../queries/function_regr_syy.proto.bin | Bin .../query-tests/queries/function_replace.json | 0 .../queries/function_replace.proto.bin | Bin ...unction_replace_with_specified_string.json | 0 ...on_replace_with_specified_string.proto.bin | Bin .../query-tests/queries/function_reverse.json | 0 .../queries/function_reverse.proto.bin | Bin .../query-tests/queries/function_right.json | 0 .../queries/function_right.proto.bin | Bin .../query-tests/queries/function_rint.json | 0 .../queries/function_rint.proto.bin | Bin .../query-tests/queries/function_rlike.json | 0 .../queries/function_rlike.proto.bin | Bin .../query-tests/queries/function_round.json | 0 .../queries/function_round.proto.bin | Bin .../queries/function_row_number.json | 0 .../queries/function_row_number.proto.bin | Bin .../query-tests/queries/function_rpad.json | 0 .../queries/function_rpad.proto.bin | Bin .../queries/function_rpad_binary.json | 0 .../queries/function_rpad_binary.proto.bin | Bin .../query-tests/queries/function_rtrim.json | 0 .../queries/function_rtrim.proto.bin | Bin .../queries/function_rtrim_with_pattern.json | 0 .../function_rtrim_with_pattern.proto.bin | Bin .../queries/function_schema_of_csv.json | 0 .../queries/function_schema_of_csv.proto.bin | Bin .../queries/function_schema_of_json.json | 0 .../queries/function_schema_of_json.proto.bin | Bin .../function_schema_of_json_with_options.json | 0 ...tion_schema_of_json_with_options.proto.bin | Bin .../queries/function_schema_of_variant.json | 0 .../function_schema_of_variant.proto.bin | Bin .../function_schema_of_variant_agg.json | 0 .../function_schema_of_variant_agg.proto.bin | Bin .../query-tests/queries/function_sec.json | 0 .../queries/function_sec.proto.bin | Bin .../query-tests/queries/function_second.json | 0 .../queries/function_second.proto.bin | Bin .../queries/function_sentences.json | 0 .../queries/function_sentences.proto.bin | Bin .../function_sentences_with_locale.json | 0 .../function_sentences_with_locale.proto.bin | Bin .../queries/function_sequence.json | 0 .../queries/function_sequence.proto.bin | Bin .../queries/function_session_user.json | 0 .../queries/function_session_user.proto.bin | Bin .../queries/function_session_window.json | 0 .../queries/function_session_window.proto.bin | Bin .../query-tests/queries/function_sha.json | 0 .../queries/function_sha.proto.bin | Bin .../query-tests/queries/function_sha1.json | 0 .../queries/function_sha1.proto.bin | Bin .../query-tests/queries/function_sha2.json | 0 .../queries/function_sha2.proto.bin | Bin .../queries/function_shiftleft.json | 0 .../queries/function_shiftleft.proto.bin | Bin .../queries/function_shiftright.json | 0 .../queries/function_shiftright.proto.bin | Bin .../queries/function_shiftrightunsigned.json | 0 .../function_shiftrightunsigned.proto.bin | Bin .../query-tests/queries/function_sign.json | 0 .../queries/function_sign.proto.bin | Bin .../query-tests/queries/function_signum.json | 0 .../queries/function_signum.proto.bin | Bin .../query-tests/queries/function_sin.json | 0 .../queries/function_sin.proto.bin | Bin .../query-tests/queries/function_sinh.json | 0 .../queries/function_sinh.proto.bin | Bin .../query-tests/queries/function_size.json | 0 .../queries/function_size.proto.bin | Bin .../queries/function_skewness.json | 0 .../queries/function_skewness.proto.bin | Bin .../query-tests/queries/function_slice.json | 0 .../queries/function_slice.proto.bin | Bin .../query-tests/queries/function_some.json | 0 .../queries/function_some.proto.bin | Bin .../queries/function_sort_array.json | 0 .../queries/function_sort_array.proto.bin | Bin .../queries/function_spark_partition_id.json | 0 .../function_spark_partition_id.proto.bin | Bin .../query-tests/queries/function_split.json | 0 .../queries/function_split.proto.bin | Bin .../queries/function_split_part.json | 0 .../queries/function_split_part.proto.bin | Bin .../queries/function_split_using_columns.json | 0 .../function_split_using_columns.proto.bin | Bin .../queries/function_split_with_limit.json | 0 .../function_split_with_limit.proto.bin | Bin ...nction_split_with_limit_using_columns.json | 0 ...n_split_with_limit_using_columns.proto.bin | Bin .../query-tests/queries/function_sqrt.json | 0 .../queries/function_sqrt.proto.bin | Bin .../query-tests/queries/function_stack.json | 0 .../queries/function_stack.proto.bin | Bin .../queries/function_startswith.json | 0 .../queries/function_startswith.proto.bin | Bin .../query-tests/queries/function_std.json | 0 .../queries/function_std.proto.bin | Bin .../query-tests/queries/function_stddev.json | 0 .../queries/function_stddev.proto.bin | Bin .../queries/function_stddev_pop.json | 0 .../queries/function_stddev_pop.proto.bin | Bin .../queries/function_stddev_samp.json | 0 .../queries/function_stddev_samp.proto.bin | Bin .../queries/function_str_to_map.json | 0 .../queries/function_str_to_map.proto.bin | Bin ..._map_with_pair_and_keyValue_delimiter.json | 0 ...with_pair_and_keyValue_delimiter.proto.bin | Bin ...nction_str_to_map_with_pair_delimiter.json | 0 ...n_str_to_map_with_pair_delimiter.proto.bin | Bin .../query-tests/queries/function_struct.json | 0 .../queries/function_struct.proto.bin | Bin .../query-tests/queries/function_substr.json | 0 .../queries/function_substr.proto.bin | Bin .../queries/function_substr_with_len.json | 0 .../function_substr_with_len.proto.bin | Bin .../queries/function_substring.json | 0 .../queries/function_substring.proto.bin | Bin .../queries/function_substring_index.json | 0 .../function_substring_index.proto.bin | Bin .../function_substring_using_columns.json | 0 ...function_substring_using_columns.proto.bin | Bin .../query-tests/queries/function_sum.json | 0 .../queries/function_sum.proto.bin | Bin .../queries/function_sum_distinct.json | 0 .../queries/function_sum_distinct.proto.bin | Bin .../query-tests/queries/function_tan.json | 0 .../queries/function_tan.proto.bin | Bin .../query-tests/queries/function_tanh.json | 0 .../queries/function_tanh.proto.bin | Bin .../queries/function_timestamp_add.json | 0 .../queries/function_timestamp_add.proto.bin | Bin .../queries/function_timestamp_diff.json | 0 .../queries/function_timestamp_diff.proto.bin | Bin .../queries/function_timestamp_micros.json | 0 .../function_timestamp_micros.proto.bin | Bin .../queries/function_timestamp_millis.json | 0 .../function_timestamp_millis.proto.bin | Bin .../queries/function_timestamp_seconds.json | 0 .../function_timestamp_seconds.proto.bin | Bin .../queries/function_to_binary.json | 0 .../queries/function_to_binary.proto.bin | Bin .../function_to_binary_with_format.json | 0 .../function_to_binary_with_format.proto.bin | Bin .../query-tests/queries/function_to_char.json | 0 .../queries/function_to_char.proto.bin | Bin .../query-tests/queries/function_to_csv.json | 0 .../queries/function_to_csv.proto.bin | Bin .../query-tests/queries/function_to_date.json | 0 .../queries/function_to_date.proto.bin | Bin .../queries/function_to_date_with_format.json | 0 .../function_to_date_with_format.proto.bin | Bin .../query-tests/queries/function_to_json.json | 0 .../queries/function_to_json.proto.bin | Bin .../queries/function_to_number.json | 0 .../queries/function_to_number.proto.bin | Bin .../queries/function_to_timestamp.json | 0 .../queries/function_to_timestamp.proto.bin | Bin .../queries/function_to_timestamp_ltz.json | 0 .../function_to_timestamp_ltz.proto.bin | Bin ...function_to_timestamp_ltz_with_format.json | 0 ...ion_to_timestamp_ltz_with_format.proto.bin | Bin .../queries/function_to_timestamp_ntz.json | 0 .../function_to_timestamp_ntz.proto.bin | Bin ...function_to_timestamp_ntz_with_format.json | 0 ...ion_to_timestamp_ntz_with_format.proto.bin | Bin .../function_to_timestamp_with_format.json | 0 ...unction_to_timestamp_with_format.proto.bin | Bin .../queries/function_to_unix_timestamp.json | 0 .../function_to_unix_timestamp.proto.bin | Bin ...unction_to_unix_timestamp_with_format.json | 0 ...on_to_unix_timestamp_with_format.proto.bin | Bin .../queries/function_to_utc_timestamp.json | 0 .../function_to_utc_timestamp.proto.bin | Bin .../queries/function_to_varchar.json | 0 .../queries/function_to_varchar.proto.bin | Bin .../queries/function_transform.json | 0 .../queries/function_transform.proto.bin | Bin .../queries/function_transform_keys.json | 0 .../queries/function_transform_keys.proto.bin | Bin .../queries/function_transform_values.json | 0 .../function_transform_values.proto.bin | Bin .../function_transform_with_index.json | 0 .../function_transform_with_index.proto.bin | Bin .../queries/function_translate.json | 0 .../queries/function_translate.proto.bin | Bin .../query-tests/queries/function_trim.json | 0 .../queries/function_trim.proto.bin | Bin .../queries/function_trim_with_pattern.json | 0 .../function_trim_with_pattern.proto.bin | Bin .../query-tests/queries/function_trunc.json | 0 .../queries/function_trunc.proto.bin | Bin .../query-tests/queries/function_try_add.json | 0 .../queries/function_try_add.proto.bin | Bin .../queries/function_try_aes_decrypt.json | 0 .../function_try_aes_decrypt.proto.bin | Bin .../function_try_aes_decrypt_with_mode.json | 0 ...nction_try_aes_decrypt_with_mode.proto.bin | Bin ...ion_try_aes_decrypt_with_mode_padding.json | 0 ...ry_aes_decrypt_with_mode_padding.proto.bin | Bin ...try_aes_decrypt_with_mode_padding_aad.json | 0 ...es_decrypt_with_mode_padding_aad.proto.bin | Bin .../query-tests/queries/function_try_avg.json | 0 .../queries/function_try_avg.proto.bin | Bin .../queries/function_try_divide.json | 0 .../queries/function_try_divide.proto.bin | Bin .../function_try_element_at_array.json | 0 .../function_try_element_at_array.proto.bin | Bin .../queries/function_try_element_at_map.json | 0 .../function_try_element_at_map.proto.bin | Bin .../queries/function_try_multiply.json | 0 .../queries/function_try_multiply.proto.bin | Bin .../queries/function_try_parse_json.json | 0 .../queries/function_try_parse_json.proto.bin | Bin .../queries/function_try_reflect.json | 0 .../queries/function_try_reflect.proto.bin | Bin .../queries/function_try_subtract.json | 0 .../queries/function_try_subtract.proto.bin | Bin .../query-tests/queries/function_try_sum.json | 0 .../queries/function_try_sum.proto.bin | Bin .../queries/function_try_to_binary.json | 0 .../queries/function_try_to_binary.proto.bin | Bin ...function_try_to_binary_without_format.json | 0 ...ion_try_to_binary_without_format.proto.bin | Bin .../queries/function_try_to_number.json | 0 .../queries/function_try_to_number.proto.bin | Bin .../queries/function_try_to_timestamp.json | 0 .../function_try_to_timestamp.proto.bin | Bin ...ction_try_to_timestamp_without_format.json | 0 ..._try_to_timestamp_without_format.proto.bin | Bin .../queries/function_try_variant_get.json | 0 .../function_try_variant_get.proto.bin | Bin .../queries/function_typedLit.json | 0 .../queries/function_typedLit.proto.bin | Bin .../query-tests/queries/function_typeof.json | 0 .../queries/function_typeof.proto.bin | Bin .../query-tests/queries/function_ucase.json | 0 .../queries/function_ucase.proto.bin | Bin .../queries/function_unbase64.json | 0 .../queries/function_unbase64.proto.bin | Bin .../query-tests/queries/function_unhex.json | 0 .../queries/function_unhex.proto.bin | Bin .../queries/function_unix_date.json | 0 .../queries/function_unix_date.proto.bin | Bin .../queries/function_unix_micros.json | 0 .../queries/function_unix_micros.proto.bin | Bin .../queries/function_unix_millis.json | 0 .../queries/function_unix_millis.proto.bin | Bin .../queries/function_unix_seconds.json | 0 .../queries/function_unix_seconds.proto.bin | Bin .../queries/function_unix_timestamp.json | 0 .../queries/function_unix_timestamp.proto.bin | Bin .../function_unix_timestamp_with_format.json | 0 ...ction_unix_timestamp_with_format.proto.bin | Bin .../query-tests/queries/function_upper.json | 0 .../queries/function_upper.proto.bin | Bin .../queries/function_url_decode.json | 0 .../queries/function_url_decode.proto.bin | Bin .../queries/function_url_encode.json | 0 .../queries/function_url_encode.proto.bin | Bin .../query-tests/queries/function_user.json | 0 .../queries/function_user.proto.bin | Bin .../query-tests/queries/function_var_pop.json | 0 .../queries/function_var_pop.proto.bin | Bin .../queries/function_var_samp.json | 0 .../queries/function_var_samp.proto.bin | Bin .../queries/function_variance.json | 0 .../queries/function_variance.proto.bin | Bin .../queries/function_variant_get.json | 0 .../queries/function_variant_get.proto.bin | Bin .../query-tests/queries/function_weekday.json | 0 .../queries/function_weekday.proto.bin | Bin .../queries/function_weekofyear.json | 0 .../queries/function_weekofyear.proto.bin | Bin .../query-tests/queries/function_window.json | 0 .../queries/function_window.proto.bin | Bin .../queries/function_window_time.json | 0 .../queries/function_window_time.proto.bin | Bin .../query-tests/queries/function_xpath.json | 0 .../queries/function_xpath.proto.bin | Bin .../queries/function_xpath_boolean.json | 0 .../queries/function_xpath_boolean.proto.bin | Bin .../queries/function_xpath_double.json | 0 .../queries/function_xpath_double.proto.bin | Bin .../queries/function_xpath_float.json | 0 .../queries/function_xpath_float.proto.bin | Bin .../queries/function_xpath_int.json | 0 .../queries/function_xpath_int.proto.bin | Bin .../queries/function_xpath_long.json | 0 .../queries/function_xpath_long.proto.bin | Bin .../queries/function_xpath_number.json | 0 .../queries/function_xpath_number.proto.bin | Bin .../queries/function_xpath_short.json | 0 .../queries/function_xpath_short.proto.bin | Bin .../queries/function_xpath_string.json | 0 .../queries/function_xpath_string.proto.bin | Bin .../queries/function_xxhash64.json | 0 .../queries/function_xxhash64.proto.bin | Bin .../query-tests/queries/function_year.json | 0 .../queries/function_year.proto.bin | Bin .../query-tests/queries/function_years.json | 0 .../queries/function_years.proto.bin | Bin .../queries/function_zip_with.json | 0 .../queries/function_zip_with.proto.bin | Bin .../query-tests/queries/groupby_agg.json | 0 .../query-tests/queries/groupby_agg.proto.bin | Bin .../queries/groupby_agg_columns.json | 0 .../queries/groupby_agg_columns.proto.bin | Bin .../queries/groupby_agg_string.json | 0 .../queries/groupby_agg_string.proto.bin | Bin .../query-tests/queries/groupby_avg.json | 0 .../query-tests/queries/groupby_avg.proto.bin | Bin .../query-tests/queries/groupby_count.json | 0 .../queries/groupby_count.proto.bin | Bin .../query-tests/queries/groupby_max.json | 0 .../query-tests/queries/groupby_max.proto.bin | Bin .../query-tests/queries/groupby_mean.json | 0 .../queries/groupby_mean.proto.bin | Bin .../query-tests/queries/groupby_min.json | 0 .../query-tests/queries/groupby_min.proto.bin | Bin .../query-tests/queries/groupby_sum.json | 0 .../query-tests/queries/groupby_sum.proto.bin | Bin .../query-tests/queries/groupingSets.json | 0 .../queries/groupingSets.proto.bin | Bin .../queries/grouping_and_grouping_id.json | 0 .../grouping_and_grouping_id.proto.bin | Bin .../resources/query-tests/queries/hint.json | 0 .../query-tests/queries/hint.proto.bin | Bin .../query-tests/queries/hll_sketch_agg.json | 0 .../queries/hll_sketch_agg.proto.bin | Bin .../hll_sketch_agg_with_columnName.json | 0 .../hll_sketch_agg_with_columnName.proto.bin | Bin ...tch_agg_with_columnName_lgConfigK_int.json | 0 ...gg_with_columnName_lgConfigK_int.proto.bin | Bin .../hll_sketch_agg_with_column_lgConfigK.json | 0 ...sketch_agg_with_column_lgConfigK.proto.bin | Bin ..._sketch_agg_with_column_lgConfigK_int.json | 0 ...ch_agg_with_column_lgConfigK_int.proto.bin | Bin .../query-tests/queries/hll_union_agg.json | 0 .../queries/hll_union_agg.proto.bin | Bin .../hll_union_agg_with_columnName.json | 0 .../hll_union_agg_with_columnName.proto.bin | Bin ...nName_allowDifferentLgConfigK_boolean.json | 0 ..._allowDifferentLgConfigK_boolean.proto.bin | Bin ...g_with_column_allowDifferentLgConfigK.json | 0 ...h_column_allowDifferentLgConfigK.proto.bin | Bin ...olumn_allowDifferentLgConfigK_boolean.json | 0 ..._allowDifferentLgConfigK_boolean.proto.bin | Bin .../query-tests/queries/intersect.json | 0 .../query-tests/queries/intersect.proto.bin | Bin .../query-tests/queries/intersectAll.json | 0 .../queries/intersectAll.proto.bin | Bin .../query-tests/queries/join_condition.json | 0 .../queries/join_condition.proto.bin | Bin .../queries/join_inner_condition.json | 0 .../queries/join_inner_condition.proto.bin | Bin .../queries/join_inner_no_condition.json | 0 .../queries/join_inner_no_condition.proto.bin | Bin .../join_inner_using_multiple_col_array.json | 0 ...n_inner_using_multiple_col_array.proto.bin | Bin .../join_inner_using_multiple_col_seq.json | 0 ...oin_inner_using_multiple_col_seq.proto.bin | Bin .../queries/join_inner_using_single_col.json | 0 .../join_inner_using_single_col.proto.bin | Bin .../join_using_multiple_col_array.json | 0 .../join_using_multiple_col_array.proto.bin | Bin .../queries/join_using_multiple_col_seq.json | 0 .../join_using_multiple_col_seq.proto.bin | Bin .../queries/join_using_single_col.json | 0 .../queries/join_using_single_col.proto.bin | Bin .../queries/json_from_dataset.json | 0 .../queries/json_from_dataset.proto.bin | Bin .../resources/query-tests/queries/limit.json | 0 .../query-tests/queries/limit.proto.bin | Bin .../query-tests/queries/melt_no_values.json | 0 .../queries/melt_no_values.proto.bin | Bin .../query-tests/queries/melt_values.json | 0 .../query-tests/queries/melt_values.proto.bin | Bin .../resources/query-tests/queries/offset.json | 0 .../query-tests/queries/offset.proto.bin | Bin .../query-tests/queries/orderBy_columns.json | 0 .../queries/orderBy_columns.proto.bin | Bin .../query-tests/queries/orderBy_strings.json | 0 .../queries/orderBy_strings.proto.bin | Bin .../resources/query-tests/queries/pivot.json | 0 .../query-tests/queries/pivot.proto.bin | Bin .../queries/pivot_without_column_values.json | 0 .../pivot_without_column_values.proto.bin | Bin .../resources/query-tests/queries/range.json | 0 .../query-tests/queries/range.proto.bin | Bin .../resources/query-tests/queries/read.json | 0 .../query-tests/queries/read.proto.bin | Bin .../query-tests/queries/read_csv.json | 0 .../query-tests/queries/read_csv.proto.bin | Bin .../query-tests/queries/read_jdbc.json | 0 .../query-tests/queries/read_jdbc.proto.bin | Bin .../queries/read_jdbc_with_partition.json | 0 .../read_jdbc_with_partition.proto.bin | Bin .../queries/read_jdbc_with_predicates.json | 0 .../read_jdbc_with_predicates.proto.bin | Bin .../query-tests/queries/read_json.json | 0 .../query-tests/queries/read_json.proto.bin | Bin .../query-tests/queries/read_orc.json | 0 .../query-tests/queries/read_orc.proto.bin | Bin .../query-tests/queries/read_parquet.json | 0 .../queries/read_parquet.proto.bin | Bin .../query-tests/queries/read_table.json | 0 .../query-tests/queries/read_table.proto.bin | Bin .../query-tests/queries/read_text.json | 0 .../query-tests/queries/read_text.proto.bin | Bin .../queries/relation_extension.json | 0 .../queries/relation_extension.proto.bin | Bin .../relation_extension_deprecated.json | 0 .../relation_extension_deprecated.proto.bin | Bin .../query-tests/queries/repartition.json | 0 .../query-tests/queries/repartition.proto.bin | Bin .../repartitionByRange_expressions.json | 0 .../repartitionByRange_expressions.proto.bin | Bin ...ionByRange_num_partitions_expressions.json | 0 ...Range_num_partitions_expressions.proto.bin | Bin .../queries/repartition_expressions.json | 0 .../queries/repartition_expressions.proto.bin | Bin ...epartition_num_partitions_expressions.json | 0 ...ition_num_partitions_expressions.proto.bin | Bin .../query-tests/queries/replace.json | 0 .../query-tests/queries/replace.proto.bin | Bin .../query-tests/queries/rollup_column.json | 0 .../queries/rollup_column.proto.bin | Bin .../query-tests/queries/rollup_string.json | 0 .../queries/rollup_string.proto.bin | Bin .../query-tests/queries/sampleBy.json | 0 .../query-tests/queries/sampleBy.proto.bin | Bin .../queries/sample_fraction_seed.json | 0 .../queries/sample_fraction_seed.proto.bin | Bin .../sample_withReplacement_fraction_seed.json | 0 ...le_withReplacement_fraction_seed.proto.bin | Bin .../resources/query-tests/queries/select.json | 0 .../query-tests/queries/select.proto.bin | Bin .../query-tests/queries/selectExpr.json | 0 .../query-tests/queries/selectExpr.proto.bin | Bin .../queries/select_collated_string.json | 0 .../queries/select_collated_string.proto.bin | Bin .../query-tests/queries/select_strings.json | 0 .../queries/select_strings.proto.bin | Bin .../queries/select_typed_1-arg.json | 0 .../queries/select_typed_1-arg.proto.bin | Bin .../queries/select_typed_2-arg.json | 0 .../queries/select_typed_2-arg.proto.bin | Bin .../queries/select_typed_3-arg.json | 0 .../queries/select_typed_3-arg.proto.bin | Bin .../queries/select_typed_4-arg.json | 0 .../queries/select_typed_4-arg.proto.bin | Bin .../queries/select_typed_5-arg.json | 0 .../queries/select_typed_5-arg.proto.bin | Bin .../queries/sortWithinPartitions_columns.json | 0 .../sortWithinPartitions_columns.proto.bin | Bin .../queries/sortWithinPartitions_strings.json | 0 .../sortWithinPartitions_strings.proto.bin | Bin .../query-tests/queries/sort_columns.json | 0 .../queries/sort_columns.proto.bin | Bin .../query-tests/queries/sort_strings.json | 0 .../queries/sort_strings.proto.bin | Bin .../streaming_table_API_with_options.json | 0 ...streaming_table_API_with_options.proto.bin | Bin .../query-tests/queries/summary.json | 0 .../query-tests/queries/summary.proto.bin | Bin .../resources/query-tests/queries/table.json | 0 .../query-tests/queries/table.proto.bin | Bin .../queries/table_API_with_options.json | 0 .../queries/table_API_with_options.proto.bin | Bin .../query-tests/queries/test_broadcast.json | 0 .../queries/test_broadcast.proto.bin | Bin .../resources/query-tests/queries/to.json | 0 .../query-tests/queries/to.proto.bin | Bin .../resources/query-tests/queries/toDF.json | 0 .../query-tests/queries/toDF.proto.bin | Bin .../resources/query-tests/queries/toJSON.json | 0 .../query-tests/queries/toJSON.proto.bin | Bin .../queries/to_avro_with_schema.json | 0 .../queries/to_avro_with_schema.proto.bin | Bin .../queries/to_avro_without_schema.json | 0 .../queries/to_avro_without_schema.proto.bin | Bin .../queries/to_protobuf_messageClassName.json | 0 .../to_protobuf_messageClassName.proto.bin | Bin ...rotobuf_messageClassName_descFilePath.json | 0 ...uf_messageClassName_descFilePath.proto.bin | Bin ...messageClassName_descFilePath_options.json | 0 ...geClassName_descFilePath_options.proto.bin | Bin .../to_protobuf_messageClassName_options.json | 0 ...rotobuf_messageClassName_options.proto.bin | Bin .../resources/query-tests/queries/union.json | 0 .../query-tests/queries/union.proto.bin | Bin .../query-tests/queries/unionAll.json | 0 .../query-tests/queries/unionAll.proto.bin | Bin .../query-tests/queries/unionByName.json | 0 .../query-tests/queries/unionByName.proto.bin | Bin .../unionByName_allowMissingColumns.json | 0 .../unionByName_allowMissingColumns.proto.bin | Bin .../queries/unpivot_no_values.json | 0 .../queries/unpivot_no_values.proto.bin | Bin .../query-tests/queries/unpivot_values.json | 0 .../queries/unpivot_values.proto.bin | Bin .../query-tests/queries/where_column.json | 0 .../queries/where_column.proto.bin | Bin .../query-tests/queries/where_expr.json | 0 .../query-tests/queries/where_expr.proto.bin | Bin .../query-tests/queries/width_bucket.json | 0 .../queries/width_bucket.proto.bin | Bin .../resources/query-tests/queries/window.json | 0 .../query-tests/queries/window.proto.bin | Bin .../queries/withColumnRenamed_java_map.json | 0 .../withColumnRenamed_java_map.proto.bin | Bin .../queries/withColumnRenamed_scala_map.json | 0 .../withColumnRenamed_scala_map.proto.bin | Bin .../queries/withColumnRenamed_single.json | 0 .../withColumnRenamed_single.proto.bin | Bin .../queries/withColumn_single.json | 0 .../queries/withColumn_single.proto.bin | Bin .../queries/withColumns_java_map.json | 0 .../queries/withColumns_java_map.proto.bin | Bin .../queries/withColumns_scala_map.json | 0 .../queries/withColumns_scala_map.proto.bin | Bin .../query-tests/queries/withMetadata.json | 0 .../queries/withMetadata.proto.bin | Bin .../query-tests/test-data/people.csv | 0 .../query-tests/test-data/people.json | 0 .../query-tests/test-data/people.txt | 0 .../test-data/streaming/csv/people.csv | 0 .../test-data/streaming/txt/people.txt | 0 .../resources/query-tests/test-data/users.orc | Bin .../query-tests/test-data/users.parquet | Bin .../scala/org/apache/spark/sql/TestUDFs.scala | 0 connect/server/README.md | 5 + {connector/connect => connect}/server/pom.xml | 2 +- .../sql/connect/plugin/CommandPlugin.java | 0 .../sql/connect/plugin/ExpressionPlugin.java | 0 .../sql/connect/plugin/RelationPlugin.java | 0 .../connect/SimpleSparkConnectService.scala | 0 .../sql/connect/SparkConnectPlugin.scala | 0 .../spark/sql/connect/config/Connect.scala | 0 .../execution/CachedStreamResponse.scala | 0 .../ConnectProgressExecutionListener.scala | 0 .../execution/ExecuteGrpcResponseSender.scala | 0 .../execution/ExecuteResponseObserver.scala | 0 .../execution/ExecuteThreadRunner.scala | 0 .../execution/SparkConnectPlanExecution.scala | 0 .../LiteralExpressionProtoConverter.scala | 0 .../connect/planner/SaveModeConverter.scala | 0 .../connect/planner/SparkConnectPlanner.scala | 0 ...ConnectStreamingQueryListenerHandler.scala | 0 .../planner/StreamingForeachBatchHelper.scala | 0 .../StreamingQueryListenerHelper.scala | 0 .../planner/TableSaveMethodConverter.scala | 0 .../plugin/SparkConnectPluginRegistry.scala | 0 .../service/ExecuteEventsManager.scala | 0 .../sql/connect/service/ExecuteHolder.scala | 0 .../LocalPropertiesCleanupInterceptor.scala | 0 .../connect/service/LoggingInterceptor.scala | 0 .../service/SessionEventsManager.scala | 0 .../sql/connect/service/SessionHolder.scala | 0 .../SparkConnectAddArtifactsHandler.scala | 0 .../service/SparkConnectAnalyzeHandler.scala | 0 .../SparkConnectArtifactStatusesHandler.scala | 0 .../service/SparkConnectConfigHandler.scala | 0 .../SparkConnectExecutePlanHandler.scala | 0 .../SparkConnectExecutionManager.scala | 0 ...SparkConnectFetchErrorDetailsHandler.scala | 0 .../SparkConnectInterceptorRegistry.scala | 0 .../SparkConnectInterruptHandler.scala | 0 .../SparkConnectListenerBusListener.scala | 0 .../SparkConnectReattachExecuteHandler.scala | 0 .../SparkConnectReleaseExecuteHandler.scala | 0 .../SparkConnectReleaseSessionHandler.scala | 0 .../connect/service/SparkConnectServer.scala | 0 .../connect/service/SparkConnectService.scala | 0 .../service/SparkConnectSessionManager.scala | 0 .../SparkConnectStreamingQueryCache.scala | 0 .../ui/SparkConnectServerAppStatusStore.scala | 0 ...parkConnectServerHistoryServerPlugin.scala | 0 .../ui/SparkConnectServerListener.scala | 0 .../connect/ui/SparkConnectServerPage.scala | 0 .../ui/SparkConnectServerSessionPage.scala | 0 .../connect/ui/SparkConnectServerTab.scala | 0 .../spark/sql/connect/ui/ToolTips.scala | 0 .../spark/sql/connect/utils/ErrorUtils.scala | 0 .../sql/connect/utils/MetricGenerator.scala | 0 .../src/test/resources/log4j2.properties | 0 .../server/src/test/resources/udf | Bin .../server/src/test/resources/udf_noA.jar | Bin .../connect/ProtoToParsedPlanTestSuite.scala | 0 .../spark/sql/connect/ResourceHelper.scala | 16 +-- .../sql/connect/SparkConnectServerTest.scala | 0 .../spark/sql/connect/dsl/package.scala | 0 ...onnectProgressExecutionListenerSuite.scala | 0 .../execution/ReattachableExecuteSuite.scala | 0 .../connect/messages/AbbreviateSuite.scala | 0 .../messages/ConnectProtoMessagesSuite.scala | 0 ...LiteralExpressionProtoConverterSuite.scala | 0 .../planner/SparkConnectPlannerSuite.scala | 0 .../SparkConnectPlannerTestUtils.scala | 0 .../planner/SparkConnectProtoSuite.scala | 0 .../planner/SparkConnectServiceSuite.scala | 0 ...parkConnectWithSessionExtensionSuite.scala | 0 .../StreamingForeachBatchHelperSuite.scala | 0 .../SparkConnectPluginRegistrySuite.scala | 0 .../service/AddArtifactsHandlerSuite.scala | 0 .../ArtifactStatusesHandlerSuite.scala | 0 .../service/ExecuteEventsManagerSuite.scala | 0 .../FetchErrorDetailsHandlerSuite.scala | 0 .../service/InterceptorRegistrySuite.scala | 0 .../service/SessionEventsManagerSuite.scala | 0 ...SparkConnectListenerBusListenerSuite.scala | 0 .../service/SparkConnectServiceE2ESuite.scala | 0 ...arkConnectServiceInternalServerSuite.scala | 0 .../SparkConnectSessionHolderSuite.scala | 0 .../SparkConnectSessionManagerSuite.scala | 0 ...SparkConnectStreamingQueryCacheSuite.scala | 0 .../ui/SparkConnectServerListenerSuite.scala | 0 .../ui/SparkConnectServerPageSuite.scala | 0 connector/connect/bin/spark-connect | 2 +- connector/connect/bin/spark-connect-build | 2 +- connector/connect/bin/spark-connect-shell | 2 +- .../apache/spark/sql/ClientE2ETestSuite.scala | 3 - .../spark/sql/PlanGenerationTestSuite.scala | 10 +- .../streaming/ClientStreamingQuerySuite.scala | 1 - .../spark/sql/test/ConnectFunSuite.scala | 17 +--- .../spark/sql/test/RemoteSparkSession.scala | 8 +- dev/connect-check-protos.py | 2 +- dev/connect-gen-protos.sh | 2 +- dev/lint-scala | 6 +- dev/protobuf-breaking-changes-check.sh | 2 +- dev/sparktestsupport/modules.py | 2 +- docs/building-spark.md | 4 - docs/spark-connect-overview.md | 6 +- .../launcher/AbstractCommandBuilder.java | 5 + pom.xml | 4 +- python/docs/source/development/testing.rst | 9 +- .../getting_started/quickstart_connect.ipynb | 2 +- python/pyspark/sql/connect/session.py | 92 ++---------------- .../scala/org/apache/spark/repl/Main.scala | 5 - 2257 files changed, 95 insertions(+), 198 deletions(-) create mode 100644 connect/common/README.md rename {connector/connect => connect}/common/pom.xml (99%) rename {connector/connect => connect}/common/src/main/buf.gen.yaml (100%) rename {connector/connect => connect}/common/src/main/buf.work.yaml (100%) rename {connector/connect => connect}/common/src/main/protobuf/buf.yaml (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/base.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/catalog.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/commands.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/common.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/example_plugins.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/expressions.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/relations.proto (100%) rename {connector/connect => connect}/common/src/main/protobuf/spark/connect/types.proto (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/ConnectProtoUtils.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/ArtifactManager.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/ClassFinder.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/CloseableIterator.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectBlockingStub.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/CustomSparkConnectStub.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/ExecutePlanResponseReattachableIterator.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcRetryHandler.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/ResponseValidator.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/RetriesExceeded.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/RetryPolicy.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClient.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectClientParser.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/SparkConnectStubState.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/SparkResult.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowDeserializer.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowEncoderUtils.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowSerializer.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ArrowVectorReader.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ConcatenatingArrowStreamReader.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/arrow/ScalaCollectionUtils.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/client/package.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/Abbreviator.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/ForeachWriterPacket.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/InvalidPlanInput.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/LiteralValueProtoConverter.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoDataTypes.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/ProtoUtils.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/StorageLevelProtoConverter.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/StreamingListenerPacket.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/UdfPacket.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/UdfUtils.scala (100%) rename {connector/connect => connect}/common/src/main/scala/org/apache/spark/sql/connect/common/config/ConnectCommon.scala (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/Hello.class (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/Hello.txt (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/README.md (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/junitLargeJar.txt (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/smallClassFile.txt (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/smallClassFileDup.txt (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/crc/smallJar.txt (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/junitLargeJar.jar (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/smallClassFile.class (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/smallClassFileDup.class (100%) rename {connector/connect => connect}/common/src/test/resources/artifact-tests/smallJar.jar (100%) rename {connector/connect => connect}/common/src/test/resources/protobuf-tests/common.desc (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/alias_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/alias_symbol.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/apply.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/as_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/as_symbol.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/between_expr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/coalesce.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/col.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/colRegex.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_add.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_alias.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_and.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_apply.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_as_multi.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_as_with_metadata.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_asc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_asc_nulls_first.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_asc_nulls_last.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_between.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_bitwiseAND.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_bitwiseOR.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_bitwiseXOR.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_cast.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_contains.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_desc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_desc_nulls_first.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_desc_nulls_last.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_divide.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_dropFields.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_endsWith.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_eqNullSafe.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_equals.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_geq.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_getField.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_getItem.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_gt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_ilike.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_isNaN.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_isNotNull.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_isNull.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_isin.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_leq.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_like.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_lt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_modulo.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_multiply.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_not.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_not_equals.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_or.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_rlike.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_star.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_star_with_target.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_startsWith.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_substr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_subtract.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_try_cast.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_unary_minus.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_when_otherwise.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/column_withField.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/crossJoin.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/crosstab.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/csv_from_dataset.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/cube_column.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/cube_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/describe.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/distinct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/drop.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/dropDuplicates.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/dropDuplicates_names_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/dropDuplicates_names_seq.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/dropDuplicates_varargs.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/drop_multiple_column.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/drop_multiple_strings.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/drop_single_column.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/drop_single_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/except.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/exceptAll.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/expression_extension.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/expression_extension_deprecated.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/fill.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/filter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/filter_expr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/freqItems.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/from_avro_with_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/from_avro_without_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/from_protobuf_messageClassName_descFilePath.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/from_protobuf_messageClassName_descFilePath_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_abs.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_acos.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_acosh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_add_months.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_decrypt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_decrypt_with_mode_padding_aad.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_encrypt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_encrypt_with_mode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_encrypt_with_mode_padding.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_encrypt_with_mode_padding_iv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aes_encrypt_with_mode_padding_iv_aad.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_aggregate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_any.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_any_value_with_ignore_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_any_value_with_respect_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_approx_count_distinct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_approx_count_distinct_rsd.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_approx_percentile.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_append.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_compact.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_contains.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_distinct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_except.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_insert.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_intersect.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_join.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_join_with_null_replacement.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_max.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_min.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_position.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_prepend.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_remove.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_repeat.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_size.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_sort.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_sort_with_comparator.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_array_union.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_arrays_overlap.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_arrays_zip.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_asc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_asc_nulls_first.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_asc_nulls_last.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ascii.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_asin.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_asinh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_assert_true_with_message.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_atan.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_atan2.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_atanh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_avg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_base64.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bin.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_and.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_get.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_or.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bit_xor.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitmap_bit_position.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitmap_bucket_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitmap_construct_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitmap_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitmap_or_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bitwise_not.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bool_and.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bool_or.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bround.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_btrim.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_btrim_with_specified_trim_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_bucket.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_call_function.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_cardinality.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ceil.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ceil_scale.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ceiling.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ceiling_scale.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_char.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_char_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_character_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_chr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_coalesce.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_col.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_collate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_collation.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_collect_list.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_collect_set.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_concat.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_concat_ws.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_contains.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_conv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_convert_timezone_with_source_time_zone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_convert_timezone_without_source_time_zone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_corr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_cos.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_cosh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_cot.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_countDistinct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_count_if.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_count_min_sketch.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_count_typed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_covar_pop.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_covar_samp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_crc32.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_csc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_cume_dist.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_curdate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_catalog.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_database.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_date.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_schema.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_timezone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_current_user.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_add.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_diff.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_from_unix_date.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_part.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_sub.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_date_trunc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dateadd.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_datediff.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_datepart.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_day.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dayname.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dayofmonth.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dayofweek.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dayofyear.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_days.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_decode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_degrees.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_dense_rank.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_desc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_desc_nulls_first.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_desc_nulls_last.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_e.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_element_at.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_elt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_encode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_endswith.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_equal_null.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_every.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_exists.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_exp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_explode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_explode_outer.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_expm1.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_expr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_extract.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_factorial.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_filter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_filter_with_pair_input.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_find_in_set.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_first_value_with_ignore_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_first_value_with_respect_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_first_with_ignore_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_first_with_respect_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_flatten.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_floor.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_floor_scale.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_forall.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_format_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_from_csv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_from_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_from_unixtime.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_from_utc_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_get.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_get_json_object.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_getbit.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_greatest.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_hash.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_hex.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_histogram_numeric.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_hour.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_hours.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_hypot.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ifnull.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ilike.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ilike_with_escape.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_initcap.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_inline.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_inline_outer.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_input_file_block_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_input_file_block_start.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_input_file_name.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_is_variant_null.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_isnan.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_isnotnull.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_isnull.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_java_method.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_json_array_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_json_object_keys.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_json_tuple.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_kurtosis.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lag.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_last_day.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_last_value_with_ignore_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_last_value_with_respect_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_last_with_ignore_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_last_with_respect_nulls.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lcase.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lead.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_least.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_left.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_len.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_levenshtein.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_levenshtein_with_threshold.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_like.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_like_with_escape.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lit.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lit_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ln.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_localtimestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_locate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_locate_with_pos.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_log.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_log10.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_log1p.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_log2.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_log_with_base.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lower.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lpad.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_lpad_binary.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ltrim.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ltrim_with_pattern.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_date.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_dt_interval.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_dt_interval_days.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_dt_interval_days_hours.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_dt_interval_days_hours_mins.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_dt_interval_days_hours_mins_secs.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months_weeks.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months_weeks_days.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months_weeks_days_hours.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months_weeks_days_hours_mins.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_interval_years_months_weeks_days_hours_mins_secs.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_timestamp_ltz_with_timezone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_timestamp_ltz_without_timezone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_timestamp_ntz.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_timestamp_with_timezone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_timestamp_without_timezone.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_ym_interval.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_ym_interval_years.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_make_ym_interval_years_months.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_concat.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_contains_key.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_entries.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_filter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_from_arrays.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_from_entries.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_keys.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_map_zip_with.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mask.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mask_with_specific_upperChar.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mask_with_specific_upperChar_lowerChar.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mask_with_specific_upperChar_lowerChar_digitChar.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_max.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_max_by.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_md5.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_median.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_min.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_min_by.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_minute.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_mode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_monotonically_increasing_id.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_month.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_monthname.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_months.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_months_between.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_months_between_with_roundoff.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_named_struct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_nanvl.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_negate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_negative.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_next_day.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_now.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_nth_value.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ntile.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_nullif.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_nvl.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_nvl2.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_octet_length.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_overlay.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_overlay_with_len.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_parse_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_parse_url.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_parse_url_with_key.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_percent_rank.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_percentile_approx.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_percentile_with_frequency.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_percentile_without_frequency.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_pi.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_pmod.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_posexplode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_posexplode_outer.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_position.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_position_with_start.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_positive.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_pow.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_power.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_printf.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_product.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_quarter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_radians.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_raise_error.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rand_with_seed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_randn_with_seed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_random_with_seed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rank.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_reduce.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_reflect.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_extract.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_extract_all_with_regex_group_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_extract_all_without_regex_group_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_instr_with_regex_group_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_instr_without_regex_group_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_like.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_replace.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regexp_substr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_avgx.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_avgy.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_intercept.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_r2.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_slope.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_sxx.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_sxy.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_regr_syy.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_replace.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_replace_with_specified_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_reverse.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_right.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rint.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rlike.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_round.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_row_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rpad.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rpad_binary.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rtrim.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_rtrim_with_pattern.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_schema_of_csv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_schema_of_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_schema_of_json_with_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_schema_of_variant.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_schema_of_variant_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sec.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_second.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sentences.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sentences_with_locale.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sequence.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_session_user.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_session_window.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sha.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sha1.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sha2.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_shiftleft.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_shiftright.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_shiftrightunsigned.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sign.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_signum.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sin.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sinh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_size.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_skewness.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_slice.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_some.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sort_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_spark_partition_id.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_split.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_split_part.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_split_using_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_split_with_limit.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_split_with_limit_using_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sqrt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_stack.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_startswith.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_std.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_stddev.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_stddev_pop.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_stddev_samp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_str_to_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_str_to_map_with_pair_and_keyValue_delimiter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_str_to_map_with_pair_delimiter.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_struct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substr_with_len.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substring.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substring_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substring_using_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_substring_with_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sum.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_sum_distinct.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_tan.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_tanh.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_timestamp_add.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_timestamp_diff.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_timestamp_micros.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_timestamp_millis.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_timestamp_seconds.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_binary.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_binary_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_char.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_csv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_date.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_date_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ltz_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp_ntz_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_timestamp_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_unix_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_unix_timestamp_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_utc_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_to_varchar.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_transform.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_transform_keys.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_transform_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_transform_with_index.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_translate.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_trim.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_trim_with_pattern.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_trunc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_add.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_aes_decrypt.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_aes_decrypt_with_mode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_aes_decrypt_with_mode_padding.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_aes_decrypt_with_mode_padding_aad.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_avg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_divide.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_element_at_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_element_at_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_multiply.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_parse_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_reflect.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_subtract.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_sum.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_to_binary.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_to_binary_without_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_to_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_to_timestamp_without_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_try_variant_get.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_typedLit.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_typeof.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_ucase.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unbase64.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unhex.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_date.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_micros.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_millis.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_seconds.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_timestamp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_unix_timestamp_with_format.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_upper.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_url_decode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_url_encode.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_user.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_var_pop.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_var_samp.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_variance.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_variant_get.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_weekday.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_weekofyear.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_window.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_window_time.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_boolean.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_double.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_float.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_int.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_long.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_number.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_short.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xpath_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_xxhash64.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_year.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_years.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/function_zip_with.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_agg_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_agg_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_avg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_count.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_max.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_mean.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_min.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupby_sum.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/groupingSets.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/grouping_and_grouping_id.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hint.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_sketch_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_sketch_agg_with_columnName.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_sketch_agg_with_columnName_lgConfigK_int.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_sketch_agg_with_column_lgConfigK.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_sketch_agg_with_column_lgConfigK_int.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_union_agg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_union_agg_with_columnName.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_union_agg_with_columnName_allowDifferentLgConfigK_boolean.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_union_agg_with_column_allowDifferentLgConfigK.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/hll_union_agg_with_column_allowDifferentLgConfigK_boolean.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/intersect.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/intersectAll.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_condition.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_inner_condition.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_inner_no_condition.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_inner_using_multiple_col_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_inner_using_multiple_col_seq.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_inner_using_single_col.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_using_multiple_col_array.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_using_multiple_col_seq.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/join_using_single_col.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/json_from_dataset.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/limit.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/melt_no_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/melt_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/offset.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/orderBy_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/orderBy_strings.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/pivot.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/pivot_without_column_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/range.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_csv.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_jdbc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_jdbc_with_partition.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_jdbc_with_predicates.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_json.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_orc.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_parquet.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_table.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/read_text.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/relation_extension.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/relation_extension_deprecated.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/repartition.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/repartitionByRange_expressions.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/repartitionByRange_num_partitions_expressions.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/repartition_expressions.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/repartition_num_partitions_expressions.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/replace.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/rollup_column.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/rollup_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sampleBy.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sample_fraction_seed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sample_withReplacement_fraction_seed.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/selectExpr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_collated_string.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_strings.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_typed_1-arg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_typed_2-arg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_typed_3-arg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_typed_4-arg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/select_typed_5-arg.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sortWithinPartitions_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sortWithinPartitions_strings.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sort_columns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/sort_strings.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/streaming_table_API_with_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/summary.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/table.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/table_API_with_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/test_broadcast.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/toDF.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/toJSON.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_avro_with_schema.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_avro_without_schema.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_protobuf_messageClassName.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_protobuf_messageClassName_descFilePath.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_protobuf_messageClassName_descFilePath_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/to_protobuf_messageClassName_options.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/union.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/unionAll.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/unionByName.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/unionByName_allowMissingColumns.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/unpivot_no_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/unpivot_values.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/where_column.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/where_expr.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/width_bucket.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/window.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumnRenamed_java_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumnRenamed_scala_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumnRenamed_single.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumn_single.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumns_java_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withColumns_scala_map.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/explain-results/withMetadata.explain (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/alias_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/alias_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/alias_symbol.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/alias_symbol.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/apply.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/apply.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/as_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/as_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/as_symbol.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/as_symbol.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/between_expr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/between_expr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/coalesce.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/coalesce.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/col.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/col.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/colRegex.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/colRegex.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_add.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_add.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_alias.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_alias.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_and.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_and.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_apply.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_apply.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_as_multi.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_as_multi.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_as_with_metadata.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_as_with_metadata.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc_nulls_first.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc_nulls_first.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc_nulls_last.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_asc_nulls_last.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_between.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_between.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseAND.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseOR.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseOR.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_cast.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_cast.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_contains.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_contains.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc_nulls_first.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc_nulls_first.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc_nulls_last.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_desc_nulls_last.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_divide.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_divide.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_dropFields.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_dropFields.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_endsWith.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_eqNullSafe.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_equals.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_equals.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_geq.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_geq.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_getField.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_getField.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_getItem.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_getItem.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_gt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_gt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_ilike.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_ilike.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNaN.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNotNull.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNull.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isNull.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isin.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_isin.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_leq.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_leq.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_like.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_like.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_lt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_lt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_modulo.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_modulo.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_multiply.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_multiply.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_not.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_not.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_not_equals.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_or.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_or.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_rlike.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_rlike.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_star.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_star.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_star_with_target.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_star_with_target.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_startsWith.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_substr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_substr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_subtract.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_subtract.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_try_cast.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_try_cast.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_unary_minus.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_when_otherwise.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_withField.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/column_withField.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/crossJoin.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/crossJoin.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/crosstab.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/crosstab.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/csv_from_dataset.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/csv_from_dataset.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/cube_column.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/cube_column.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/cube_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/cube_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/describe.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/describe.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/distinct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/distinct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_names_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_names_seq.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/dropDuplicates_varargs.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_multiple_column.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_multiple_column.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_multiple_strings.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_multiple_strings.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_single_column.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_single_column.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_single_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/drop_single_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/except.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/except.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/exceptAll.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/exceptAll.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/expression_extension.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/expression_extension.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/expression_extension_deprecated.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/expression_extension_deprecated.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/fill.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/fill.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/filter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/filter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/filter_expr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/filter_expr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/freqItems.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/freqItems.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_avro_with_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_avro_with_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_avro_without_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_avro_without_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_abs.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_abs.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_acos.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_acos.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_acosh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_acosh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_add_months.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_add_months.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding_aad.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding_aad.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aggregate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any_value_with_ignore_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any_value_with_ignore_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_percentile.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_approx_percentile.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_append.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_append.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_compact.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_contains.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_distinct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_except.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_except.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_insert.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_intersect.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_join.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_join.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_join_with_null_replacement.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_max.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_max.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_min.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_min.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_position.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_position.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_prepend.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_prepend.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_remove.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_repeat.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_size.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_size.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_sort.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_union.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_array_union.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_arrays_overlap.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_arrays_zip.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc_nulls_first.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc_nulls_first.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc_nulls_last.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asc_nulls_last.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ascii.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ascii.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asin.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asin.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asinh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_asinh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atan.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atan.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atan2.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atan2.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atanh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_atanh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_avg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_avg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_base64.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_base64.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bin.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bin.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_and.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_and.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_get.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_get.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_or.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_or.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_xor.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bit_xor.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitwise_not.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bitwise_not.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bool_and.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bool_and.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bool_or.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bool_or.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bround.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bround.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_btrim.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_btrim.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bucket.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_bucket.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_call_function.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_call_function.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cardinality.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cardinality.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceil.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceil.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceil_scale.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceiling.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceiling.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceiling_scale.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ceiling_scale.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_char.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_char.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_char_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_char_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_character_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_character_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_chr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_chr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_coalesce.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_col.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_col.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collation.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collation.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collect_list.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collect_set.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_concat.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_concat.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_concat_ws.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_contains.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_contains.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_conv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_conv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_corr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_corr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cos.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cos.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cosh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cosh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cot.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cot.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_countDistinct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_if.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_if.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_min_sketch.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_min_sketch.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_typed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_count_typed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_covar_pop.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_covar_samp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_crc32.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_crc32.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_csc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_csc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cume_dist.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_curdate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_curdate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_catalog.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_catalog.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_database.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_database.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_date.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_date.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_schema.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_schema.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_timezone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_timezone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_user.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_current_user.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_add.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_add.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_diff.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_diff.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_from_unix_date.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_from_unix_date.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_part.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_part.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_sub.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_sub.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_trunc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_date_trunc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dateadd.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dateadd.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_datediff.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_datediff.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_datepart.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_datepart.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_day.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_day.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayname.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayname.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofmonth.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofweek.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofyear.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_days.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_days.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_decode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_decode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_degrees.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_degrees.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dense_rank.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_dense_rank.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc_nulls_first.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc_nulls_first.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc_nulls_last.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_desc_nulls_last.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_e.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_e.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_element_at.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_element_at.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_elt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_elt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_encode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_encode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_endswith.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_endswith.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_equal_null.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_equal_null.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_every.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_every.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_exists.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_exists.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_exp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_exp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_explode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_explode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_explode_outer.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_expm1.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_expm1.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_expr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_expr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_extract.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_extract.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_factorial.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_factorial.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_filter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_filter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_find_in_set.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_find_in_set.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_value_with_respect_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_value_with_respect_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_flatten.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_flatten.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_floor.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_floor.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_floor_scale.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_forall.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_forall.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_format_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_format_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_csv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_unixtime.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_from_utc_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_get.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_get.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_get_json_object.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_get_json_object.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_getbit.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_getbit.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_greatest.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_greatest.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hash.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hash.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hex.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hex.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_histogram_numeric.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_histogram_numeric.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hour.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hour.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hours.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hours.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hypot.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_hypot.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ifnull.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ifnull.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ilike.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ilike.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ilike_with_escape.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ilike_with_escape.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_initcap.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_initcap.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_inline.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_inline.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_inline_outer.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_inline_outer.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_block_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_block_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_block_start.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_block_start.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_name.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_is_variant_null.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_is_variant_null.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnan.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnan.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnotnull.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnotnull.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnull.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_isnull.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_java_method.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_java_method.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_array_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_array_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_object_keys.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_object_keys.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_tuple.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_kurtosis.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_kurtosis.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lag.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lag.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_day.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_day.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lcase.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lcase.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lead.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lead.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_least.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_least.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_left.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_left.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_len.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_len.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_levenshtein.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_like.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_like.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_like_with_escape.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_like_with_escape.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lit.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lit.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lit_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lit_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ln.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ln.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_localtimestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_locate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_locate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_locate_with_pos.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log10.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log10.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log1p.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log1p.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log2.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log2.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log_with_base.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lower.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lower.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lpad.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lpad.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lpad_binary.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_lpad_binary.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ltrim.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ltrim.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_date.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_date.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_concat.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_contains_key.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_contains_key.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_entries.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_filter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_from_arrays.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_from_entries.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_keys.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_zip_with.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_max.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_max.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_max_by.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_max_by.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_md5.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_md5.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_median.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_median.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_min.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_min.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_min_by.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_min_by.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_minute.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_minute.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_mode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_month.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_month.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_monthname.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_monthname.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months_between.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months_between.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_named_struct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_named_struct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nanvl.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_negate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_negate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_negative.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_negative.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_next_day.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_next_day.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_now.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_now.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nth_value.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nth_value.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ntile.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ntile.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nullif.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nullif.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nvl.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nvl.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nvl2.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_nvl2.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_octet_length.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_overlay.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_overlay.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_overlay_with_len.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_url.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_url.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_url_with_key.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_parse_url_with_key.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percent_rank.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_approx.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pi.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pi.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pmod.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pmod.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_posexplode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_posexplode_outer.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_position.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_position.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_position_with_start.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_position_with_start.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_positive.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_positive.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pow.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_pow.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_power.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_power.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_printf.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_printf.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_product.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_product.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_quarter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_quarter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_radians.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_radians.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_raise_error.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_raise_error.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rand_with_seed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_randn_with_seed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_random_with_seed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_random_with_seed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rank.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rank.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reduce.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reduce.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reflect.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reflect.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract_all_without_regex_group_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_extract_all_without_regex_group_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_like.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_like.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_replace.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_substr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regexp_substr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_avgx.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_avgx.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_avgy.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_avgy.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_intercept.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_intercept.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_r2.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_r2.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_slope.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_slope.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_sxx.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_sxx.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_sxy.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_sxy.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_syy.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_regr_syy.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_replace.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_replace.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reverse.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_reverse.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_right.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_right.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rint.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rint.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rlike.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rlike.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_round.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_round.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_row_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_row_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rpad.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rpad.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rpad_binary.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rpad_binary.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rtrim.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rtrim.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_csv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_variant.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_variant.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sec.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sec.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_second.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_second.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sentences.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sentences.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sentences_with_locale.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sentences_with_locale.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sequence.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sequence.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_session_user.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_session_user.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_session_window.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_session_window.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha1.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha1.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha2.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sha2.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftleft.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftright.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sign.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sign.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_signum.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_signum.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sin.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sin.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sinh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sinh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_size.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_size.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_skewness.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_skewness.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_slice.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_slice.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_some.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_some.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sort_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_spark_partition_id.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_part.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_part.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_using_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_using_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_with_limit.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sqrt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stack.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stack.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_startswith.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_startswith.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_std.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_std.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev_pop.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev_samp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_delimiter.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_delimiter.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_struct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_struct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substr_with_len.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substr_with_len.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring_using_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_substring_using_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sum.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sum.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sum_distinct.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_tan.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_tan.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_tanh.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_tanh.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_add.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_add.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_diff.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_diff.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_micros.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_micros.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_millis.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_millis.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_binary.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_binary.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_binary_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_binary_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_char.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_char.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_csv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_date.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_date.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_date_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_unix_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_unix_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_unix_timestamp_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_unix_timestamp_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_varchar.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_to_varchar.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_keys.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_keys.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_with_index.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_translate.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_translate.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trim.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trim.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trunc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_trunc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_add.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_add.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_avg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_avg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_divide.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_divide.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_element_at_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_element_at_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_element_at_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_element_at_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_multiply.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_multiply.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_parse_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_parse_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_reflect.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_reflect.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_subtract.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_subtract.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_sum.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_sum.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_binary.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_binary.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_variant_get.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_try_variant_get.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_typedLit.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_typedLit.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_typeof.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_typeof.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ucase.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_ucase.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unbase64.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unhex.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unhex.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_date.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_date.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_micros.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_micros.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_millis.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_millis.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_seconds.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_seconds.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_timestamp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_unix_timestamp_with_format.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_upper.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_upper.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_url_decode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_url_decode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_url_encode.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_url_encode.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_user.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_user.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_var_pop.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_var_pop.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_var_samp.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_variance.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_variance.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_variant_get.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_variant_get.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_weekday.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_weekday.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_weekofyear.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_weekofyear.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_window.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_window.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_window_time.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_window_time.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_boolean.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_boolean.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_double.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_double.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_float.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_float.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_int.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_int.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_long.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_long.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_number.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_number.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_short.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_short.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xpath_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xxhash64.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_year.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_year.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_years.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_years.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_zip_with.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_agg_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_avg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_avg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_count.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_count.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_max.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_max.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_mean.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_mean.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_min.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_min.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_sum.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupby_sum.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupingSets.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/groupingSets.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/grouping_and_grouping_id.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/grouping_and_grouping_id.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hint.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hint.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_columnName.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_columnName.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_columnName_lgConfigK_int.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_columnName_lgConfigK_int.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_column_lgConfigK.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_column_lgConfigK.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_column_lgConfigK_int.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_sketch_agg_with_column_lgConfigK_int.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_columnName.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_columnName.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_columnName_allowDifferentLgConfigK_boolean.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_columnName_allowDifferentLgConfigK_boolean.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_column_allowDifferentLgConfigK.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_column_allowDifferentLgConfigK.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_column_allowDifferentLgConfigK_boolean.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/hll_union_agg_with_column_allowDifferentLgConfigK_boolean.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/intersect.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/intersect.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/intersectAll.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/intersectAll.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_condition.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_condition.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_condition.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_no_condition.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_no_condition.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_multiple_col_seq.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_single_col.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_inner_using_single_col.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_multiple_col_array.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_multiple_col_seq.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_single_col.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/join_using_single_col.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/json_from_dataset.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/json_from_dataset.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/limit.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/limit.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/melt_no_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/melt_no_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/melt_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/melt_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/offset.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/offset.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/orderBy_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/orderBy_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/orderBy_strings.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/orderBy_strings.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/pivot.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/pivot.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/pivot_without_column_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/pivot_without_column_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/range.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/range.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_csv.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_csv.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc_with_partition.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc_with_partition.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc_with_predicates.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_jdbc_with_predicates.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_json.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_json.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_orc.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_orc.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_parquet.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_parquet.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_table.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_table.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_text.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/read_text.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/relation_extension.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/relation_extension.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/relation_extension_deprecated.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/relation_extension_deprecated.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartitionByRange_expressions.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartitionByRange_num_partitions_expressions.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition_expressions.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition_expressions.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/repartition_num_partitions_expressions.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/replace.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/replace.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/rollup_column.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/rollup_column.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/rollup_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/rollup_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sampleBy.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sampleBy.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sample_fraction_seed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sample_fraction_seed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sample_withReplacement_fraction_seed.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/selectExpr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/selectExpr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_collated_string.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_collated_string.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_strings.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_strings.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_1-arg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_1-arg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_2-arg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_2-arg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_3-arg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_3-arg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_4-arg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_4-arg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_5-arg.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/select_typed_5-arg.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sortWithinPartitions_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sortWithinPartitions_strings.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sort_columns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sort_columns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sort_strings.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/sort_strings.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/streaming_table_API_with_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/streaming_table_API_with_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/summary.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/summary.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/table.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/table.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/table_API_with_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/table_API_with_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/test_broadcast.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/test_broadcast.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/toDF.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/toDF.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/toJSON.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/toJSON.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_avro_with_schema.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_avro_with_schema.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_avro_without_schema.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_avro_without_schema.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/union.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/union.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionAll.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionAll.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionByName.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionByName.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unionByName_allowMissingColumns.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unpivot_no_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unpivot_no_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unpivot_values.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/unpivot_values.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/where_column.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/where_column.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/where_expr.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/where_expr.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/width_bucket.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/width_bucket.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/window.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/window.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_java_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_scala_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_single.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumnRenamed_single.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumn_single.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumn_single.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumns_java_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumns_java_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumns_scala_map.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withColumns_scala_map.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withMetadata.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/queries/withMetadata.proto.bin (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/people.csv (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/people.json (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/people.txt (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/streaming/csv/people.csv (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/streaming/txt/people.txt (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/users.orc (100%) rename {connector/connect => connect}/common/src/test/resources/query-tests/test-data/users.parquet (100%) rename {connector/connect => connect}/common/src/test/scala/org/apache/spark/sql/TestUDFs.scala (100%) create mode 100644 connect/server/README.md rename {connector/connect => connect}/server/pom.xml (99%) rename {connector/connect => connect}/server/src/main/java/org/apache/spark/sql/connect/plugin/CommandPlugin.java (100%) rename {connector/connect => connect}/server/src/main/java/org/apache/spark/sql/connect/plugin/ExpressionPlugin.java (100%) rename {connector/connect => connect}/server/src/main/java/org/apache/spark/sql/connect/plugin/RelationPlugin.java (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/SimpleSparkConnectService.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/SparkConnectPlugin.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/config/Connect.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/CachedStreamResponse.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/ConnectProgressExecutionListener.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteGrpcResponseSender.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteResponseObserver.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/ExecuteThreadRunner.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/execution/SparkConnectPlanExecution.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverter.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/SaveModeConverter.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectStreamingQueryListenerHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelper.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/StreamingQueryListenerHelper.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/planner/TableSaveMethodConverter.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistry.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteEventsManager.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/ExecuteHolder.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/LocalPropertiesCleanupInterceptor.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/LoggingInterceptor.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SessionEventsManager.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAddArtifactsHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectArtifactStatusesHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutePlanHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectExecutionManager.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectFetchErrorDetailsHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterceptorRegistry.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectInterruptHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReattachExecuteHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseExecuteHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseSessionHandler.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectServer.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectService.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCache.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerAppStatusStore.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerHistoryServerPlugin.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListener.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerPage.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerSessionPage.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/SparkConnectServerTab.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/ui/ToolTips.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/utils/ErrorUtils.scala (100%) rename {connector/connect => connect}/server/src/main/scala/org/apache/spark/sql/connect/utils/MetricGenerator.scala (100%) rename {connector/connect => connect}/server/src/test/resources/log4j2.properties (100%) rename {connector/connect => connect}/server/src/test/resources/udf (100%) rename {connector/connect => connect}/server/src/test/resources/udf_noA.jar (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/ProtoToParsedPlanTestSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/ResourceHelper.scala (79%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/dsl/package.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/execution/ConnectProgressExecutionListenerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/execution/ReattachableExecuteSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/messages/AbbreviateSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/messages/ConnectProtoMessagesSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/LiteralExpressionProtoConverterSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerTestUtils.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectServiceSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectWithSessionExtensionSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/planner/StreamingForeachBatchHelperSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/plugin/SparkConnectPluginRegistrySuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/AddArtifactsHandlerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/ArtifactStatusesHandlerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/ExecuteEventsManagerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/FetchErrorDetailsHandlerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/InterceptorRegistrySuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SessionEventsManagerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListenerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceInternalServerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionHolderSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManagerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectStreamingQueryCacheSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerListenerSuite.scala (100%) rename {connector/connect => connect}/server/src/test/scala/org/apache/spark/sql/connect/ui/SparkConnectServerPageSuite.scala (100%) mode change 100755 => 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/ConnectFunSuite.scala diff --git a/.github/labeler.yml b/.github/labeler.yml index c5325949889ba..a74b4ab5d5a33 100644 --- a/.github/labeler.yml +++ b/.github/labeler.yml @@ -225,6 +225,7 @@ DEPLOY: CONNECT: - changed-files: - any-glob-to-any-file: [ + 'connect/**/*', 'connector/connect/**/*', 'python/pyspark/sql/**/connect/**/*', 'python/pyspark/ml/**/connect/**/*' diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 4f092d8c04786..95cec52a262f6 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -606,7 +606,7 @@ jobs: - name: Breaking change detection against branch-3.5 uses: bufbuild/buf-breaking-action@v1 with: - input: connector/connect/common/src/main + input: connect/common/src/main against: 'https://github.com/apache/spark.git#branch=branch-3.5,subdir=connector/connect/common/src/main' - name: Install Python 3.9 uses: actions/setup-python@v5 diff --git a/.github/workflows/build_python_connect.yml b/.github/workflows/build_python_connect.yml index 01d9d272d4366..8458cdf771b19 100644 --- a/.github/workflows/build_python_connect.yml +++ b/.github/workflows/build_python_connect.yml @@ -84,7 +84,7 @@ jobs: # Start a Spark Connect server for local PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ - --jars "`find connector/connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" + --jars "`find connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" # Remove Py4J and PySpark zipped library to make sure there is no JVM connection mv python/lib lib.back @@ -104,7 +104,7 @@ jobs: PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --master "local-cluster[2, 4, 1024]" \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ - --jars "`find connector/connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" + --jars "`find connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" # Remove Py4J and PySpark zipped library to make sure there is no JVM connection mv python/lib lib.back diff --git a/.github/workflows/build_python_connect35.yml b/.github/workflows/build_python_connect35.yml index abff471349a22..b00fdddb4b0e0 100644 --- a/.github/workflows/build_python_connect35.yml +++ b/.github/workflows/build_python_connect35.yml @@ -87,7 +87,7 @@ jobs: # Start a Spark Connect server for local PYTHONPATH="python/lib/pyspark.zip:python/lib/py4j-0.10.9.7-src.zip:$PYTHONPATH" ./sbin/start-connect-server.sh \ --driver-java-options "-Dlog4j.configurationFile=file:$GITHUB_WORKSPACE/conf/log4j2.properties" \ - --jars "`find connector/connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" + --jars "`find connect/server/target -name spark-connect-*SNAPSHOT.jar`,`find connector/protobuf/target -name spark-protobuf-*SNAPSHOT.jar`,`find connector/avro/target -name spark-avro*SNAPSHOT.jar`" # Checkout to branch-3.5 to use the tests in branch-3.5. cd .. diff --git a/.github/workflows/maven_test.yml b/.github/workflows/maven_test.yml index d23cea926a274..fa30bd3abc8a5 100644 --- a/.github/workflows/maven_test.yml +++ b/.github/workflows/maven_test.yml @@ -194,7 +194,7 @@ jobs: if [[ "$INCLUDED_TAGS" != "" ]]; then ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pjvm-profiler -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} -Dtest.include.tags="$INCLUDED_TAGS" test -fae elif [[ "$MODULES_TO_TEST" == "connect" ]]; then - ./build/mvn $MAVEN_CLI_OPTS -Dtest.exclude.tags="$EXCLUDED_TAGS" -Djava.version=${JAVA_VERSION/-ea} -pl connector/connect/client/jvm,connector/connect/common,connector/connect/server test -fae + ./build/mvn $MAVEN_CLI_OPTS -Dtest.exclude.tags="$EXCLUDED_TAGS" -Djava.version=${JAVA_VERSION/-ea} -pl connector/connect/client/jvm,connect/common,connect/server test -fae elif [[ "$EXCLUDED_TAGS" != "" ]]; then ./build/mvn $MAVEN_CLI_OPTS -pl "$TEST_MODULES" -Pyarn -Pkubernetes -Pvolcano -Phive -Phive-thriftserver -Phadoop-cloud -Pjvm-profiler -Pspark-ganglia-lgpl -Pkinesis-asl -Djava.version=${JAVA_VERSION/-ea} -Dtest.exclude.tags="$EXCLUDED_TAGS" test -fae elif [[ "$MODULES_TO_TEST" == *"sql#hive-thriftserver"* ]]; then diff --git a/assembly/pom.xml b/assembly/pom.xml index 58e7ae5bb0c7f..9377849cf1cdc 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -74,6 +74,41 @@ spark-repl_${scala.binary.version} ${project.version} + + org.apache.spark + spark-connect_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-connect-common_${scala.binary.version} + + + io.grpc + * + + + com.google.code.gson + gson + + + com.google.guava + failureaccess + + + + + org.apache.spark + spark-avro_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-protobuf_${scala.binary.version} + ${project.version} + provided + diff --git a/python/docs/source/development/testing.rst b/python/docs/source/development/testing.rst index 2dd389e34a44f..c2737371c9b46 100644 --- a/python/docs/source/development/testing.rst +++ b/python/docs/source/development/testing.rst @@ -69,21 +69,16 @@ Running Tests for Python Client ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ In order to test the changes in Protobuf definitions, for example, at -`spark/connector/connect/common/src/main/protobuf/spark/connect `_, +`spark/connect/common/src/main/protobuf/spark/connect `_, you should regenerate Python Protobuf client first by running ``dev/connect-gen-protos.sh``. Running PySpark Shell with Python Client ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~ -For Apache Spark you locally built: +The command below starts Spark Connect server automatically locally, and creates a Spark Connect client connected to the server. .. code-block:: bash bin/pyspark --remote "local[*]" -For the Apache Spark release: - -.. code-block:: bash - - bin/pyspark --remote "local[*]" --packages org.apache.spark:spark-connect_2.13:$SPARK_VERSION diff --git a/python/docs/source/getting_started/quickstart_connect.ipynb b/python/docs/source/getting_started/quickstart_connect.ipynb index 1d994bde907fb..74b77238c67fe 100644 --- a/python/docs/source/getting_started/quickstart_connect.ipynb +++ b/python/docs/source/getting_started/quickstart_connect.ipynb @@ -28,7 +28,7 @@ "metadata": {}, "outputs": [], "source": [ - "!$HOME/sbin/start-connect-server.sh --packages org.apache.spark:spark-connect_2.13:$SPARK_VERSION" + "!$HOME/sbin/start-connect-server.sh" ] }, { diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 1c40206471a5c..b5e76982b3fd1 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -51,7 +51,6 @@ from pyspark.sql.connect.dataframe import DataFrame from pyspark.sql.dataframe import DataFrame as ParentDataFrame -from pyspark.loose_version import LooseVersion from pyspark.sql.connect.client import SparkConnectClient, DefaultChannelBuilder from pyspark.sql.connect.conf import RuntimeConf from pyspark.sql.connect.plan import ( @@ -984,39 +983,13 @@ def _start_connect_server(master: str, opts: Dict[str, Any]) -> None: """ Starts the Spark Connect server given the master (thread-unsafe). - At the high level, there are two cases. The first case is development case, e.g., - you locally build Apache Spark, and run ``SparkSession.builder.remote("local")``: - - 1. This method automatically finds the jars for Spark Connect (because the jars for - Spark Connect are not bundled in the regular Apache Spark release). - - 2. Temporarily remove all states for Spark Connect, for example, ``SPARK_REMOTE`` + 1. Temporarily remove all states for Spark Connect, for example, ``SPARK_REMOTE`` environment variable. - 3. Starts a JVM (without Spark Context) first, and adds the Spark Connect server jars - into the current class loader. Otherwise, Spark Context with ``spark.plugins`` - cannot be initialized because the JVM is already running without the jars in - the classpath before executing this Python process for driver side (in case of - PySpark application submission). - - 4. Starts a regular Spark session that automatically starts a Spark Connect server + 2. Starts a regular Spark session that automatically starts a Spark Connect server via ``spark.plugins`` feature. - - The second case is when you use Apache Spark release: - - 1. Users must specify either the jars or package, e.g., ``--packages - org.apache.spark:spark-connect_2.12:3.4.0``. The jars or packages would be specified - in SparkSubmit automatically. This method does not do anything related to this. - - 2. Temporarily remove all states for Spark Connect, for example, ``SPARK_REMOTE`` - environment variable. It does not do anything for PySpark application submission as - well because jars or packages were already specified before executing this Python - process for driver side. - - 3. Starts a regular Spark session that automatically starts a Spark Connect server - with JVM via ``spark.plugins`` feature. """ - from pyspark import SparkContext, SparkConf, __version__ + from pyspark import SparkContext, SparkConf session = PySparkSession._instantiatedSession if session is None or session._sc._jsc is None: @@ -1033,21 +1006,6 @@ def _start_connect_server(master: str, opts: Dict[str, Any]) -> None: # See also SPARK-42272. overwrite_conf["spark.connect.grpc.binding.port"] = "0" - def create_conf(**kwargs: Any) -> SparkConf: - conf = SparkConf(**kwargs) - for k, v in overwrite_conf.items(): - conf.set(k, v) - for k, v in default_conf.items(): - if not conf.contains(k): - conf.set(k, v) - return conf - - # Check if we're using unreleased version that is in development. - # Also checks SPARK_TESTING for RC versions. - is_dev_mode = ( - "dev" in LooseVersion(__version__).version or "SPARK_TESTING" in os.environ - ) - origin_remote = os.environ.get("SPARK_REMOTE", None) try: if origin_remote is not None: @@ -1055,49 +1013,11 @@ def create_conf(**kwargs: Any) -> SparkConf: # start the regular PySpark session. del os.environ["SPARK_REMOTE"] - SparkContext._ensure_initialized(conf=create_conf(loadDefaults=False)) - - if is_dev_mode: - # Try and catch for a possibility in production because pyspark.testing - # does not exist in the canonical release. - try: - from pyspark.testing.utils import search_jar - - # Note that, in production, spark.jars.packages configuration should be - # set by users. Here we're automatically searching the jars locally built. - connect_jar = search_jar( - "connector/connect/server", "spark-connect-assembly-", "spark-connect" - ) - if connect_jar is None: - warnings.warn( - "Attempted to automatically find the Spark Connect jars because " - "'SPARK_TESTING' environment variable is set, or the current " - f"PySpark version is dev version ({__version__}). However, the jar" - " was not found. Manually locate the jars and specify them, e.g., " - "'spark.jars' configuration." - ) - else: - pyutils = SparkContext._jvm.PythonSQLUtils # type: ignore[union-attr] - pyutils.addJarToCurrentClassLoader(connect_jar) - - # Required for local-cluster testing as their executors need the jars - # to load the Spark plugin for Spark Connect. - if master.startswith("local-cluster"): - if "spark.jars" in overwrite_conf: - overwrite_conf[ - "spark.jars" - ] = f"{overwrite_conf['spark.jars']},{connect_jar}" - else: - overwrite_conf["spark.jars"] = connect_jar - - except ImportError: - pass - # The regular PySpark session is registered as an active session # so would not be garbage-collected. - PySparkSession( - SparkContext.getOrCreate(create_conf(loadDefaults=True, _jvm=SparkContext._jvm)) - ) + conf = SparkConf(loadDefaults=True) + conf.setAll(list(overwrite_conf.items())).setAll(list(default_conf.items())) + PySparkSession(SparkContext.getOrCreate(conf)) # Lastly only keep runtime configurations because other configurations are # disallowed to set in the regular Spark Connect session. diff --git a/repl/src/main/scala/org/apache/spark/repl/Main.scala b/repl/src/main/scala/org/apache/spark/repl/Main.scala index 22b7a8e2a7337..7b126c357271b 100644 --- a/repl/src/main/scala/org/apache/spark/repl/Main.scala +++ b/repl/src/main/scala/org/apache/spark/repl/Main.scala @@ -126,11 +126,6 @@ object Main extends Logging { sparkContext = sparkSession.sparkContext sparkSession } catch { - case e: ClassNotFoundException if isShellSession && e.getMessage.contains( - "org.apache.spark.sql.connect.SparkConnectPlugin") => - logError("Failed to load spark connect plugin.") - logError("You need to build Spark with -Pconnect.") - sys.exit(1) case e: Exception if isShellSession => logError("Failed to initialize Spark session.", e) sys.exit(1) From ddf87b433487b15c8867cf060c3f0508f4249d91 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 11 Jul 2024 10:24:19 +0900 Subject: [PATCH 20/64] [SPARK-48860][TESTS] Update `ui-test` to use `ws` 8.18.0 ### What changes were proposed in this pull request? This is a test dependency update to use `ws` 8.18.0. ### Why are the changes needed? Although Apache Spark binary is not affected by this, this PR aims to resolve this alert which recommends `ws` versions 8.17.1+. - https://github.com/apache/spark/security/dependabot/95 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass the CIs with the new dependency. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47287 from dongjoon-hyun/SPARK-48860. Authored-by: Dongjoon Hyun Signed-off-by: Hyukjin Kwon --- ui-test/package-lock.json | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/ui-test/package-lock.json b/ui-test/package-lock.json index ec870dfa4801c..f6c2adab3b019 100644 --- a/ui-test/package-lock.json +++ b/ui-test/package-lock.json @@ -2632,7 +2632,7 @@ "whatwg-encoding": "^2.0.0", "whatwg-mimetype": "^3.0.0", "whatwg-url": "^11.0.0", - "ws": "^8.11.0", + "ws": "^8.18.0", "xml-name-validator": "^4.0.0" }, "engines": { @@ -4112,10 +4112,11 @@ } }, "node_modules/ws": { - "version": "8.14.2", - "resolved": "https://registry.npmjs.org/ws/-/ws-8.14.2.tgz", - "integrity": "sha512-wEBG1ftX4jcglPxgFCMJmZ2PLtSbJ2Peg6TmpJFTbe9GZYOQCDPdMYu/Tm0/bGZkw8paZnJY45J4K2PZrLYq8g==", + "version": "8.18.0", + "resolved": "https://registry.npmjs.org/ws/-/ws-8.18.0.tgz", + "integrity": "sha512-8VbfWfHLbbwu3+N6OKsOMpBdT4kXPDDB9cJk2bJ6mh9ucxdlnNvH1e+roYkKmN9Nxw2yjz7VzeO9oOz2zJ04Pw==", "dev": true, + "license": "MIT", "engines": { "node": ">=10.0.0" }, From 66769888138df874ee0fdabca66234d89c7d8a13 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Thu, 11 Jul 2024 10:25:02 +0900 Subject: [PATCH 21/64] [SPARK-48862][PYTHON][CONNECT] Avoid calling `_proto_to_string` when INFO level is not enabled ### What changes were proposed in this pull request? Avoid calling `_proto_to_string` when INFO level is not enabled. ### Why are the changes needed? We should avoid `_proto_to_string` as it takes long time, although the result is not used if INFO level is not enabled. E.g., ```py from functools import reduce df = createDataFrame() def project_schema(n=100): return reduce(lambda df, _: df.select(F.col("a"), F.col("b"), F.col("c"), F.col("d")), range(n), df).schema profile(project_schema) ``` Screenshot 2024-07-10 at 17 24 18 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47289 from ueshin/issues/SPARK-48862/logging. Authored-by: Takuya Ueshin Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/client/core.py | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/python/pyspark/sql/connect/client/core.py b/python/pyspark/sql/connect/client/core.py index e91324150cbd8..9ad14ed2c5f4d 100644 --- a/python/pyspark/sql/connect/client/core.py +++ b/python/pyspark/sql/connect/client/core.py @@ -24,6 +24,7 @@ check_dependencies(__name__) +import logging import threading import os import platform @@ -862,7 +863,8 @@ def to_table_as_iterator( """ Return given plan as a PyArrow Table iterator. """ - logger.info(f"Executing plan {self._proto_to_string(plan)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Executing plan {self._proto_to_string(plan)}") req = self._execute_plan_request_with_metadata() req.plan.CopyFrom(plan) with Progress(handlers=self._progress_handlers, operation_id=req.operation_id) as progress: @@ -878,7 +880,8 @@ def to_table( """ Return given plan as a PyArrow Table. """ - logger.info(f"Executing plan {self._proto_to_string(plan)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Executing plan {self._proto_to_string(plan)}") req = self._execute_plan_request_with_metadata() req.plan.CopyFrom(plan) table, schema, metrics, observed_metrics, _ = self._execute_and_fetch(req, observations) @@ -894,7 +897,8 @@ def to_pandas( """ Return given plan as a pandas DataFrame. """ - logger.info(f"Executing plan {self._proto_to_string(plan)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Executing plan {self._proto_to_string(plan)}") req = self._execute_plan_request_with_metadata() req.plan.CopyFrom(plan) (self_destruct_conf,) = self.get_config_with_defaults( @@ -996,7 +1000,8 @@ def schema(self, plan: pb2.Plan) -> StructType: """ Return schema for given plan. """ - logger.info(f"Schema for plan: {self._proto_to_string(plan)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Schema for plan: {self._proto_to_string(plan)}") schema = self._analyze(method="schema", plan=plan).schema assert schema is not None # Server side should populate the struct field which is the schema. @@ -1007,7 +1012,8 @@ def explain_string(self, plan: pb2.Plan, explain_mode: str = "extended") -> str: """ Return explain string for given plan. """ - logger.info(f"Explain (mode={explain_mode}) for plan {self._proto_to_string(plan)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Explain (mode={explain_mode}) for plan {self._proto_to_string(plan)}") result = self._analyze( method="explain", plan=plan, explain_mode=explain_mode ).explain_string @@ -1020,7 +1026,8 @@ def execute_command( """ Execute given command. """ - logger.info(f"Execute command for command {self._proto_to_string(command)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Execute command for command {self._proto_to_string(command)}") req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id @@ -1041,7 +1048,8 @@ def execute_command_as_iterator( """ Execute given command. Similar to execute_command, but the value is returned using yield. """ - logger.info(f"Execute command as iterator for command {self._proto_to_string(command)}") + if logger.isEnabledFor(logging.INFO): + logger.info(f"Execute command as iterator for command {self._proto_to_string(command)}") req = self._execute_plan_request_with_metadata() if self._user_id: req.user_context.user_id = self._user_id From 0d131a023f58bd5c613157ba28e4fa8addd9c5a5 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Thu, 11 Jul 2024 10:28:28 +0900 Subject: [PATCH 22/64] [SPARK-48459][CONNECT][PYTHON][FOLLOWUP] Ignore to_plan from with_origin ### What changes were proposed in this pull request? Ignores `connect.Column.to_plan` from `with_origin`. ### Why are the changes needed? Capturing call site on `connect.Column.to_plan` takes long time when creating proto plans if there are many `connect.Column` objects, although the call sites on `connect.Column.to_plan` are not necessary. E.g., ```py from pyspark.sql import functions as F df = createDataFrame() def schema(): return df.select(*([F.col("a"), F.col("b"), F.col("c"), F.col("d")] * 10)).schema profile(schema) ``` Screenshot 2024-07-10 at 13 40 33 The total function calls / duration for this is: - before ``` 28393570 function calls (28381720 primitive calls) in 3.450 seconds ``` - after ``` 109970 function calls (98120 primitive calls) in 0.184 seconds ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47284 from ueshin/issues/SPARK-48459/query_context. Authored-by: Takuya Ueshin Signed-off-by: Hyukjin Kwon --- python/pyspark/errors/utils.py | 58 +++++++++++++++++++++------- python/pyspark/sql/connect/column.py | 2 +- 2 files changed, 44 insertions(+), 16 deletions(-) diff --git a/python/pyspark/errors/utils.py b/python/pyspark/errors/utils.py index 89721d23c3858..703978e9573cd 100644 --- a/python/pyspark/errors/utils.py +++ b/python/pyspark/errors/utils.py @@ -20,7 +20,19 @@ import inspect import os import threading -from typing import Any, Callable, Dict, Match, TypeVar, Type, Optional, TYPE_CHECKING +from typing import ( + Any, + Callable, + Dict, + List, + Match, + TypeVar, + Type, + Optional, + Union, + TYPE_CHECKING, + overload, +) import pyspark from pyspark.errors.error_classes import ERROR_CLASSES_MAP @@ -251,20 +263,36 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: return wrapper -def with_origin_to_class(cls: Type[T]) -> Type[T]: +@overload +def with_origin_to_class(cls_or_ignores: Type[T], ignores: Optional[List[str]] = None) -> Type[T]: + ... + + +@overload +def with_origin_to_class( + cls_or_ignores: Optional[List[str]] = None, +) -> Callable[[Type[T]], Type[T]]: + ... + + +def with_origin_to_class( + cls_or_ignores: Optional[Union[Type[T], List[str]]] = None, ignores: Optional[List[str]] = None +) -> Union[Type[T], Callable[[Type[T]], Type[T]]]: """ Decorate all methods of a class with `_with_origin` to capture call site information. """ - if os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true": - for name, method in cls.__dict__.items(): - # Excluding Python magic methods that do not utilize JVM functions. - if callable(method) and name not in ( - "__init__", - "__new__", - "__iter__", - "__nonzero__", - "__repr__", - "__bool__", - ): - setattr(cls, name, _with_origin(method)) - return cls + if cls_or_ignores is None or isinstance(cls_or_ignores, list): + ignores = cls_or_ignores or [] + return lambda cls: with_origin_to_class(cls, ignores) + else: + cls = cls_or_ignores + if os.environ.get("PYSPARK_PIN_THREAD", "true").lower() == "true": + skipping = set( + ["__init__", "__new__", "__iter__", "__nonzero__", "__repr__", "__bool__"] + + (ignores or []) + ) + for name, method in cls.__dict__.items(): + # Excluding Python magic methods that do not utilize JVM functions. + if callable(method) and name not in skipping: + setattr(cls, name, _with_origin(method)) + return cls diff --git a/python/pyspark/sql/connect/column.py b/python/pyspark/sql/connect/column.py index ef48091a35b0c..625f5f13f69a3 100644 --- a/python/pyspark/sql/connect/column.py +++ b/python/pyspark/sql/connect/column.py @@ -100,7 +100,7 @@ def _to_expr(v: Any) -> Expression: return v._expr if isinstance(v, Column) else LiteralExpression._from_value(v) -@with_origin_to_class +@with_origin_to_class(["to_plan"]) class Column(ParentColumn): def __new__( cls, From 52d95653ada7386639322a18cb4d0788948fa0a2 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 11 Jul 2024 11:22:33 +0900 Subject: [PATCH 23/64] [SPARK-48763][FOLLOWUP] Make `dev/lint-scala` error message more accurate ### What changes were proposed in this pull request? The pr is followuping https://github.com/apache/spark/pull/47157, to make `dev/lint-scala` error message more accurate. ### Why are the changes needed? After move from: `connector/connect/server` `connector/connect/common` to: `connect/server``connect/common` Our error message in `dev/lint-scala` should be updated synchronously. eg: image image ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. Manually test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47291 from panbingkun/SPARK-48763_FOLLOWUP. Authored-by: panbingkun Signed-off-by: Hyukjin Kwon --- dev/lint-scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/lint-scala b/dev/lint-scala index c3b19db243a8c..d44eb02ef9094 100755 --- a/dev/lint-scala +++ b/dev/lint-scala @@ -36,7 +36,7 @@ ERRORS=$(./build/mvn \ ) if test ! -z "$ERRORS"; then - echo -e "The scalafmt check failed on connector/connect at following occurrences:\n\n$ERRORS\n" + echo -e "The scalafmt check failed on connect or connector/connect at following occurrences:\n\n$ERRORS\n" echo "Before submitting your change, please make sure to format your code using the following command:" echo "./build/mvn scalafmt:format -Dscalafmt.skip=false -Dscalafmt.validateOnly=false -Dscalafmt.changedOnly=false -pl connect/common -pl connect/server -pl connector/connect/client/jvm" exit 1 From 97a0f6449f8fc13eceacea2407f2a5164604f54e Mon Sep 17 00:00:00 2001 From: Eric Marnadi Date: Thu, 11 Jul 2024 11:57:56 +0900 Subject: [PATCH 24/64] [SPARK-48726][SS] Create the StateSchemaV3 file format, and write this out for the TransformWithStateExec operator ### What changes were proposed in this pull request? In this PR, we introduce the `StateSchemaV3` file that is used to keep track of a list of `ColumnFamilySchema` which we write from the `TransformWithState` operator. We collect the Column Family schemas from the driver, and write them out as a part of a planning rule. We will be introducing the OperatorStateMetadataV2 in the following PR: https://github.com/apache/spark/pull/47273 This will integrate with the TransformWithState operator, and rely on the schema file. ### Why are the changes needed? These changes are needed to enable schema evolution for this operator in the future. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit tests and ran existing unit tests ``` [info] Run completed in 11 seconds, 673 milliseconds. [info] Total number of tests run: 4 [info] Suites: completed 1, aborted 0 [info] Tests: succeeded 4, failed 0, canceled 0, ignored 0, pending 0 [info] All tests passed. [success] Total time: 43 s, completed Jun 26, 2024, 10:38:35 AM ``` ### Was this patch authored or co-authored using generative AI tooling? No Closes #47104 from ericm-db/state-schema-tws. Lead-authored-by: Eric Marnadi Co-authored-by: Eric Marnadi <132308037+ericm-db@users.noreply.github.com> Signed-off-by: Jungtaek Lim --- .../resources/error/error-conditions.json | 6 + .../apache/spark/sql/internal/SQLConf.scala | 7 + .../streaming/ColumnFamilySchemaUtils.scala | 75 ++++++++ .../FlatMapGroupsWithStateExec.scala | 5 +- .../streaming/IncrementalExecution.scala | 44 +++-- .../StatefulProcessorHandleImpl.scala | 161 +++++++++++++----- .../StatefulProcessorHandleImplBase.scala | 55 ++++++ .../StreamingSymmetricHashJoinExec.scala | 9 +- .../streaming/TransformWithStateExec.scala | 79 ++++++++- .../streaming/state/SchemaHelper.scala | 56 ++++++ .../StateSchemaCompatibilityChecker.scala | 68 ++++---- .../streaming/state/StateSchemaV3File.scala | 99 +++++++++++ .../streaming/state/StateStore.scala | 42 ++++- .../streaming/state/StateStoreErrors.scala | 19 +++ .../streaming/statefulOperators.scala | 29 +++- .../execution/streaming/streamingLimits.scala | 4 +- .../streaming/state/StateStoreSuite.scala | 26 +++ .../streaming/TransformWithStateSuite.scala | 76 ++++++++- 18 files changed, 753 insertions(+), 107 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ColumnFamilySchemaUtils.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaV3File.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index efb054ef95d7a..02d1e63e380aa 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -3779,6 +3779,12 @@ ], "sqlState" : "42802" }, + "STATE_STORE_COLUMN_FAMILY_SCHEMA_INCOMPATIBLE" : { + "message" : [ + "Incompatible schema transformation with column family=, oldSchema=, newSchema=." + ], + "sqlState" : "42802" + }, "STATE_STORE_HANDLE_NOT_INITIALIZED" : { "message" : [ "The handle has not been initialized for this StatefulProcessor.", 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 9f291bac3bf67..6ca831f99304b 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 @@ -2150,6 +2150,13 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(TimeUnit.MINUTES.toMillis(1)) // 1 minute + val STREAMING_TRANSFORM_WITH_STATE_OP_STATE_SCHEMA_VERSION = + buildConf("spark.sql.streaming.transformWithState.stateSchemaVersion") + .doc("The version of the state schema used by the transformWithState operator") + .version("4.0.0") + .intConf + .createWithDefault(3) + val STATE_STORE_COMPRESSION_CODEC = buildConf("spark.sql.streaming.stateStore.compression.codec") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ColumnFamilySchemaUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ColumnFamilySchemaUtils.scala new file mode 100644 index 0000000000000..68f3fa434389e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ColumnFamilySchemaUtils.scala @@ -0,0 +1,75 @@ +/* + * 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.streaming + +import org.apache.spark.sql.Encoder +import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{COMPOSITE_KEY_ROW_SCHEMA, KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA, VALUE_ROW_SCHEMA_WITH_TTL} +import org.apache.spark.sql.execution.streaming.state.{ColumnFamilySchema, ColumnFamilySchemaV1, NoPrefixKeyStateEncoderSpec, PrefixKeyScanStateEncoderSpec} + +trait ColumnFamilySchemaUtils { + def getValueStateSchema[T](stateName: String, hasTtl: Boolean): ColumnFamilySchema + + def getListStateSchema[T](stateName: String, hasTtl: Boolean): ColumnFamilySchema + + def getMapStateSchema[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + hasTtl: Boolean): ColumnFamilySchema +} + +object ColumnFamilySchemaUtilsV1 extends ColumnFamilySchemaUtils { + + def getValueStateSchema[T](stateName: String, hasTtl: Boolean): ColumnFamilySchemaV1 = { + ColumnFamilySchemaV1( + stateName, + KEY_ROW_SCHEMA, + if (hasTtl) { + VALUE_ROW_SCHEMA_WITH_TTL + } else { + VALUE_ROW_SCHEMA + }, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA)) + } + + def getListStateSchema[T](stateName: String, hasTtl: Boolean): ColumnFamilySchemaV1 = { + ColumnFamilySchemaV1( + stateName, + KEY_ROW_SCHEMA, + if (hasTtl) { + VALUE_ROW_SCHEMA_WITH_TTL + } else { + VALUE_ROW_SCHEMA + }, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA)) + } + + def getMapStateSchema[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + hasTtl: Boolean): ColumnFamilySchemaV1 = { + ColumnFamilySchemaV1( + stateName, + COMPOSITE_KEY_ROW_SCHEMA, + if (hasTtl) { + VALUE_ROW_SCHEMA_WITH_TTL + } else { + VALUE_ROW_SCHEMA + }, + PrefixKeyScanStateEncoderSpec(COMPOSITE_KEY_ROW_SCHEMA, 1), + Some(userKeyEnc.schema)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala index ad1707ce7dad7..e2ef656fce757 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala @@ -189,7 +189,10 @@ trait FlatMapGroupsWithStateExecBase }) } - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, + batchId: Long, + stateSchemaVersion: Int): Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, groupingAttributes.toStructType, stateManager.stateSchema, session.sessionState) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index 61f11749406d8..772c26ac76672 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -85,6 +85,12 @@ class IncrementalExecution( .map(SQLConf.SHUFFLE_PARTITIONS.valueConverter) .getOrElse(sparkSession.sessionState.conf.numShufflePartitions) + /** + * This value dictates which schema format version the state schema should be written in + * for all operators other than TransformWithState. + */ + private val STATE_SCHEMA_DEFAULT_VERSION: Int = 2 + /** * See [SPARK-18339] * Walk the optimized logical plan and replace CurrentBatchTimestamp @@ -187,23 +193,30 @@ class IncrementalExecution( } } - object WriteStatefulOperatorMetadataRule extends SparkPlanPartialRule { - override val rule: PartialFunction[SparkPlan, SparkPlan] = { - case stateStoreWriter: StateStoreWriter if isFirstBatch => - val metadata = stateStoreWriter.operatorStateMetadata() - val metadataWriter = new OperatorStateMetadataWriter(new Path( - checkpointLocation, stateStoreWriter.getStateInfo.operatorId.toString), hadoopConf) - metadataWriter.write(metadata) - stateStoreWriter - } - } - // Planning rule used to record the state schema for the first run and validate state schema // changes across query runs. - object StateSchemaValidationRule extends SparkPlanPartialRule { + object StateSchemaAndOperatorMetadataRule extends SparkPlanPartialRule { override val rule: PartialFunction[SparkPlan, SparkPlan] = { + // In the case of TransformWithStateExec, we want to collect this StateSchema + // filepath, and write this path out in the OperatorStateMetadata file case statefulOp: StatefulOperator if isFirstBatch => - statefulOp.validateAndMaybeEvolveStateSchema(hadoopConf) + val stateSchemaVersion = statefulOp match { + case _: TransformWithStateExec => sparkSession.sessionState.conf. + getConf(SQLConf.STREAMING_TRANSFORM_WITH_STATE_OP_STATE_SCHEMA_VERSION) + case _ => STATE_SCHEMA_DEFAULT_VERSION + } + val stateSchemaPaths = statefulOp. + validateAndMaybeEvolveStateSchema(hadoopConf, currentBatchId, stateSchemaVersion) + // write out the state schema paths to the metadata file + statefulOp match { + case stateStoreWriter: StateStoreWriter => + val metadata = stateStoreWriter.operatorStateMetadata() + // TODO: [SPARK-48849] Populate metadata with stateSchemaPaths if metadata version is v2 + val metadataWriter = new OperatorStateMetadataWriter(new Path( + checkpointLocation, stateStoreWriter.getStateInfo.operatorId.toString), hadoopConf) + metadataWriter.write(metadata) + case _ => + } statefulOp } } @@ -478,10 +491,9 @@ class IncrementalExecution( checkOperatorValidWithMetadata(planWithStateOpId) } - // The two rules below don't change the plan but can cause the side effect that + // The rule below doesn't change the plan but can cause the side effect that // metadata/schema is written in the checkpoint directory of stateful operator. - planWithStateOpId transform StateSchemaValidationRule.rule - planWithStateOpId transform WriteStatefulOperatorMetadataRule.rule + planWithStateOpId transform StateSchemaAndOperatorMetadataRule.rule simulateWatermarkPropagation(planWithStateOpId) planWithStateOpId transform WatermarkPropagationRule.rule diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala index dcc77e94de280..277e1516425d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala @@ -19,14 +19,16 @@ package org.apache.spark.sql.execution.streaming import java.util import java.util.UUID +import scala.collection.mutable + import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.PRE_INIT import org.apache.spark.sql.execution.streaming.state._ -import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, StatefulProcessorHandle, TimeMode, TTLConfig, ValueState} +import org.apache.spark.sql.streaming.{ListState, MapState, QueryInfo, TimeMode, TTLConfig, ValueState} import org.apache.spark.util.Utils /** @@ -48,7 +50,7 @@ object ImplicitGroupingKeyTracker { */ object StatefulProcessorHandleState extends Enumeration { type StatefulProcessorHandleState = Value - val CREATED, INITIALIZED, DATA_PROCESSED, TIMER_PROCESSED, CLOSED = Value + val CREATED, PRE_INIT, INITIALIZED, DATA_PROCESSED, TIMER_PROCESSED, CLOSED = Value } class QueryInfoImpl( @@ -85,7 +87,7 @@ class StatefulProcessorHandleImpl( isStreaming: Boolean = true, batchTimestampMs: Option[Long] = None, metrics: Map[String, SQLMetric] = Map.empty) - extends StatefulProcessorHandle with Logging { + extends StatefulProcessorHandleImplBase(timeMode) with Logging { import StatefulProcessorHandleState._ /** @@ -96,6 +98,8 @@ class StatefulProcessorHandleImpl( private val BATCH_QUERY_ID = "00000000-0000-0000-0000-000000000000" + currState = CREATED + private def buildQueryInfo(): QueryInfo = { val taskCtxOpt = Option(TaskContext.get()) val (queryId, batchId) = if (!isStreaming) { @@ -113,22 +117,14 @@ class StatefulProcessorHandleImpl( private lazy val currQueryInfo: QueryInfo = buildQueryInfo() - private var currState: StatefulProcessorHandleState = CREATED - private def incrementMetric(metricName: String): Unit = { metrics.get(metricName).foreach(_.add(1)) } - def setHandleState(newState: StatefulProcessorHandleState): Unit = { - currState = newState - } - - def getHandleState: StatefulProcessorHandleState = currState - override def getValueState[T]( stateName: String, valEncoder: Encoder[T]): ValueState[T] = { - verifyStateVarOperations("get_value_state") + verifyStateVarOperations("get_value_state", CREATED) incrementMetric("numValueStateVars") val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState @@ -138,7 +134,7 @@ class StatefulProcessorHandleImpl( stateName: String, valEncoder: Encoder[T], ttlConfig: TTLConfig): ValueState[T] = { - verifyStateVarOperations("get_value_state") + verifyStateVarOperations("get_value_state", CREATED) validateTTLConfig(ttlConfig, stateName) assert(batchTimestampMs.isDefined) @@ -153,25 +149,6 @@ class StatefulProcessorHandleImpl( private lazy val timerState = new TimerStateImpl(store, timeMode, keyEncoder) - private def verifyStateVarOperations(operationType: String): Unit = { - if (currState != CREATED) { - throw StateStoreErrors.cannotPerformOperationWithInvalidHandleState(operationType, - currState.toString) - } - } - - private def verifyTimerOperations(operationType: String): Unit = { - if (timeMode == NoTime) { - throw StateStoreErrors.cannotPerformOperationWithInvalidTimeMode(operationType, - timeMode.toString) - } - - if (currState < INITIALIZED || currState >= TIMER_PROCESSED) { - throw StateStoreErrors.cannotPerformOperationWithInvalidHandleState(operationType, - currState.toString) - } - } - /** * Function to register a timer for the given expiryTimestampMs * @param expiryTimestampMs - timestamp in milliseconds for the timer to expire @@ -232,14 +209,14 @@ class StatefulProcessorHandleImpl( * @param stateName - name of the state variable */ override def deleteIfExists(stateName: String): Unit = { - verifyStateVarOperations("delete_if_exists") + verifyStateVarOperations("delete_if_exists", CREATED) if (store.removeColFamilyIfExists(stateName)) { incrementMetric("numDeletedStateVars") } } override def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] = { - verifyStateVarOperations("get_list_state") + verifyStateVarOperations("get_list_state", CREATED) incrementMetric("numListStateVars") val resultState = new ListStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState @@ -265,7 +242,7 @@ class StatefulProcessorHandleImpl( valEncoder: Encoder[T], ttlConfig: TTLConfig): ListState[T] = { - verifyStateVarOperations("get_list_state") + verifyStateVarOperations("get_list_state", CREATED) validateTTLConfig(ttlConfig, stateName) assert(batchTimestampMs.isDefined) @@ -281,7 +258,7 @@ class StatefulProcessorHandleImpl( stateName: String, userKeyEnc: Encoder[K], valEncoder: Encoder[V]): MapState[K, V] = { - verifyStateVarOperations("get_map_state") + verifyStateVarOperations("get_map_state", CREATED) incrementMetric("numMapStateVars") val resultState = new MapStateImpl[K, V](store, stateName, keyEncoder, userKeyEnc, valEncoder) resultState @@ -292,7 +269,7 @@ class StatefulProcessorHandleImpl( userKeyEnc: Encoder[K], valEncoder: Encoder[V], ttlConfig: TTLConfig): MapState[K, V] = { - verifyStateVarOperations("get_map_state") + verifyStateVarOperations("get_map_state", CREATED) validateTTLConfig(ttlConfig, stateName) assert(batchTimestampMs.isDefined) @@ -313,3 +290,111 @@ class StatefulProcessorHandleImpl( } } } + +/** + * This DriverStatefulProcessorHandleImpl is used within TransformWithExec + * on the driver side to collect the columnFamilySchemas before any processing is + * actually done. We need this class because we can only collect the schemas after + * the StatefulProcessor is initialized. + */ +class DriverStatefulProcessorHandleImpl(timeMode: TimeMode) + extends StatefulProcessorHandleImplBase(timeMode) { + + private[sql] val columnFamilySchemaUtils = ColumnFamilySchemaUtilsV1 + + // Because this is only happening on the driver side, there is only + // one task modifying and accessing this map at a time + private[sql] val columnFamilySchemas: mutable.Map[String, ColumnFamilySchema] = + new mutable.HashMap[String, ColumnFamilySchema]() + + def getColumnFamilySchemas: Map[String, ColumnFamilySchema] = columnFamilySchemas.toMap + + override def getValueState[T](stateName: String, valEncoder: Encoder[T]): ValueState[T] = { + verifyStateVarOperations("get_value_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getValueStateSchema(stateName, false) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[ValueState[T]] + } + + override def getValueState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig): ValueState[T] = { + verifyStateVarOperations("get_value_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getValueStateSchema(stateName, true) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[ValueState[T]] + } + + override def getListState[T](stateName: String, valEncoder: Encoder[T]): ListState[T] = { + verifyStateVarOperations("get_list_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getListStateSchema(stateName, false) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[ListState[T]] + } + + override def getListState[T]( + stateName: String, + valEncoder: Encoder[T], + ttlConfig: TTLConfig): ListState[T] = { + verifyStateVarOperations("get_list_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getListStateSchema(stateName, true) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[ListState[T]] + } + + override def getMapState[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V]): MapState[K, V] = { + verifyStateVarOperations("get_map_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getMapStateSchema(stateName, userKeyEnc, false) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[MapState[K, V]] + } + + override def getMapState[K, V]( + stateName: String, + userKeyEnc: Encoder[K], + valEncoder: Encoder[V], + ttlConfig: TTLConfig): MapState[K, V] = { + verifyStateVarOperations("get_map_state", PRE_INIT) + val colFamilySchema = columnFamilySchemaUtils. + getMapStateSchema(stateName, userKeyEnc, true) + columnFamilySchemas.put(stateName, colFamilySchema) + null.asInstanceOf[MapState[K, V]] + } + + /** Function to return queryInfo for currently running task */ + override def getQueryInfo(): QueryInfo = { + new QueryInfoImpl(UUID.randomUUID(), UUID.randomUUID(), 0L) + } + + /** + * Methods that are only included to satisfy the interface. + * These methods will fail if called from the driver side, as the handle + * will be in the PRE_INIT phase, and all these timer operations need to be + * called from the INITIALIZED phase. + */ + override def registerTimer(expiryTimestampMs: Long): Unit = { + verifyTimerOperations("register_timer") + } + + override def deleteTimer(expiryTimestampMs: Long): Unit = { + verifyTimerOperations("delete_timer") + } + + override def listTimers(): Iterator[Long] = { + verifyTimerOperations("list_timers") + Iterator.empty + } + + override def deleteIfExists(stateName: String): Unit = { + verifyStateVarOperations("delete_if_exists", PRE_INIT) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala new file mode 100644 index 0000000000000..3b952967e35d9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImplBase.scala @@ -0,0 +1,55 @@ +/* + * 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.streaming + +import org.apache.spark.sql.catalyst.plans.logical.NoTime +import org.apache.spark.sql.execution.streaming.StatefulProcessorHandleState.{INITIALIZED, PRE_INIT, StatefulProcessorHandleState, TIMER_PROCESSED} +import org.apache.spark.sql.execution.streaming.state.StateStoreErrors +import org.apache.spark.sql.streaming.{StatefulProcessorHandle, TimeMode} + +abstract class StatefulProcessorHandleImplBase(timeMode: TimeMode) + extends StatefulProcessorHandle { + + protected var currState: StatefulProcessorHandleState = PRE_INIT + + def setHandleState(newState: StatefulProcessorHandleState): Unit = { + currState = newState + } + + def getHandleState: StatefulProcessorHandleState = currState + + def verifyTimerOperations(operationType: String): Unit = { + if (timeMode == NoTime) { + throw StateStoreErrors.cannotPerformOperationWithInvalidTimeMode(operationType, + timeMode.toString) + } + + if (currState < INITIALIZED || currState >= TIMER_PROCESSED) { + throw StateStoreErrors.cannotPerformOperationWithInvalidHandleState(operationType, + currState.toString) + } + } + + def verifyStateVarOperations( + operationType: String, + requiredState: StatefulProcessorHandleState): Unit = { + if (currState != requiredState) { + throw StateStoreErrors.cannotPerformOperationWithInvalidHandleState(operationType, + currState.toString) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala index 28365eac6e817..ea275a28780ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingSymmetricHashJoinExec.scala @@ -246,7 +246,10 @@ case class StreamingSymmetricHashJoinExec( watermarkUsedForStateCleanup && watermarkHasChanged } - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, + batchId: Long, + stateSchemaVersion: Int): Array[String] = { var result: Map[String, (StructType, StructType)] = Map.empty // get state schema for state stores on left side of the join result ++= SymmetricHashJoinStateManager.getSchemaForStateStores(LeftSide, @@ -257,10 +260,10 @@ case class StreamingSymmetricHashJoinExec( right.output, rightKeys, stateFormatVersion) // validate and maybe evolve schema for all state stores across both sides of the join - result.foreach { case (stateStoreName, (keySchema, valueSchema)) => + result.iterator.flatMap { case (stateStoreName, (keySchema, valueSchema)) => StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keySchema, valueSchema, session.sessionState, storeName = stateStoreName) - } + }.toArray } protected override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala index c56d7c969d9e7..6af0126efdf89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TransformWithStateExec.scala @@ -20,6 +20,7 @@ import java.util.UUID import java.util.concurrent.TimeUnit.NANOSECONDS import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD @@ -92,6 +93,39 @@ case class TransformWithStateExec( } } + /** + * We initialize this processor handle in the driver to run the init function + * and fetch the schemas of the state variables initialized in this processor. + * @return a new instance of the driver processor handle + */ + private def getDriverProcessorHandle(): DriverStatefulProcessorHandleImpl = { + val driverProcessorHandle = new DriverStatefulProcessorHandleImpl(timeMode) + driverProcessorHandle.setHandleState(StatefulProcessorHandleState.PRE_INIT) + statefulProcessor.setHandle(driverProcessorHandle) + statefulProcessor.init(outputMode, timeMode) + driverProcessorHandle + } + + /** + * Fetching the columnFamilySchemas from the StatefulProcessorHandle + * after init is called. + */ + private def getColFamilySchemas(): Map[String, ColumnFamilySchema] = { + val columnFamilySchemas = getDriverProcessorHandle().getColumnFamilySchemas + closeProcessorHandle() + columnFamilySchemas + } + + /** + * This method is used for the driver-side stateful processor after we + * have collected all the necessary schemas. + * This instance of the stateful processor won't be used again. + */ + private def closeProcessorHandle(): Unit = { + statefulProcessor.close() + statefulProcessor.setHandle(null) + } + /** * Controls watermark propagation to downstream modes. If timeMode is * ProcessingTime, the output rows cannot be interpreted in eventTime, hence @@ -340,11 +374,46 @@ case class TransformWithStateExec( ) } - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { - // TODO: transformWithState is special because we don't have the schema of the state directly - // within the passed args. We need to gather this after running the init function - // within the stateful processor on the driver. This also requires a schema format change - // when recording this information persistently. + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, + batchId: Long, + stateSchemaVersion: Int): Array[String] = { + assert(stateSchemaVersion >= 3) + val newColumnFamilySchemas = getColFamilySchemas() + val schemaFile = new StateSchemaV3File( + hadoopConf, stateSchemaDirPath(StateStoreId.DEFAULT_STORE_NAME).toString) + // TODO: [SPARK-48849] Read the schema path from the OperatorStateMetadata file + // and validate it with the new schema + + // Write the new schema to the schema file + val schemaPath = schemaFile.addWithUUID(batchId, newColumnFamilySchemas.values.toList) + Array(schemaPath.toString) + } + + private def validateSchemas( + oldSchemas: List[ColumnFamilySchema], + newSchemas: Map[String, ColumnFamilySchema]): Unit = { + oldSchemas.foreach { case oldSchema: ColumnFamilySchemaV1 => + newSchemas.get(oldSchema.columnFamilyName).foreach { + case newSchema: ColumnFamilySchemaV1 => + StateSchemaCompatibilityChecker.check( + (oldSchema.keySchema, oldSchema.valueSchema), + (newSchema.keySchema, newSchema.valueSchema), + ignoreValueSchema = false + ) + } + } + } + + private def stateSchemaDirPath(storeName: String): Path = { + assert(storeName == StateStoreId.DEFAULT_STORE_NAME) + def stateInfo = getStateInfo + val stateCheckpointPath = + new Path(getStateInfo.checkpointLocation, + s"${stateInfo.operatorId.toString}") + + val storeNamePath = new Path(stateCheckpointPath, storeName) + new Path(new Path(storeNamePath, "_metadata"), "schema") } override protected def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala index 2eef3d9fc22ed..0a8021ab3de2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/SchemaHelper.scala @@ -20,6 +20,11 @@ package org.apache.spark.sql.execution.streaming.state import java.io.StringReader import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream} +import org.json4s.DefaultFormats +import org.json4s.JsonAST._ +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods +import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.sql.execution.streaming.MetadataVersionUtil import org.apache.spark.sql.types.StructType @@ -28,6 +33,57 @@ import org.apache.spark.util.Utils /** * Helper classes for reading/writing state schema. */ +sealed trait ColumnFamilySchema extends Serializable { + def jsonValue: JValue + + def json: String + + def columnFamilyName: String +} + +case class ColumnFamilySchemaV1( + columnFamilyName: String, + keySchema: StructType, + valueSchema: StructType, + keyStateEncoderSpec: KeyStateEncoderSpec, + userKeyEncoder: Option[StructType] = None) extends ColumnFamilySchema { + def jsonValue: JValue = { + ("columnFamilyName" -> JString(columnFamilyName)) ~ + ("keySchema" -> JString(keySchema.json)) ~ + ("valueSchema" -> JString(valueSchema.json)) ~ + ("keyStateEncoderSpec" -> keyStateEncoderSpec.jsonValue) ~ + ("userKeyEncoder" -> userKeyEncoder.map(s => JString(s.json)).getOrElse(JNothing)) + } + + def json: String = { + compact(render(jsonValue)) + } +} + +object ColumnFamilySchemaV1 { + + /** + * Create a ColumnFamilySchemaV1 object from the Json string + * This function is to read the StateSchemaV3 file + */ + def fromJson(json: String): ColumnFamilySchema = { + implicit val formats: DefaultFormats.type = DefaultFormats + val colFamilyMap = JsonMethods.parse(json).extract[Map[String, Any]] + assert(colFamilyMap.isInstanceOf[Map[_, _]], + s"Expected Map but got ${colFamilyMap.getClass}") + val keySchema = StructType.fromString(colFamilyMap("keySchema").asInstanceOf[String]) + val valueSchema = StructType.fromString(colFamilyMap("valueSchema").asInstanceOf[String]) + ColumnFamilySchemaV1( + colFamilyMap("columnFamilyName").asInstanceOf[String], + keySchema, + valueSchema, + KeyStateEncoderSpec.fromJson(keySchema, colFamilyMap("keyStateEncoderSpec") + .asInstanceOf[Map[String, Any]]), + colFamilyMap.get("userKeyEncoder").map(_.asInstanceOf[String]).map(StructType.fromString) + ) + } +} + object SchemaHelper { sealed trait SchemaReader { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala index e365cc6371f28..8aabc0846fe61 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaCompatibilityChecker.scala @@ -42,36 +42,6 @@ class StateSchemaCompatibilityChecker( fm.mkdirs(schemaFileLocation.getParent) - /** - * Function to check if new state store schema is compatible with the existing schema. - * @param oldSchema - old state schema - * @param newSchema - new state schema - * @param ignoreValueSchema - whether to ignore value schema or not - */ - private def check( - oldSchema: (StructType, StructType), - newSchema: (StructType, StructType), - ignoreValueSchema: Boolean) : Unit = { - val (storedKeySchema, storedValueSchema) = oldSchema - val (keySchema, valueSchema) = newSchema - - if (storedKeySchema.equals(keySchema) && - (ignoreValueSchema || storedValueSchema.equals(valueSchema))) { - // schema is exactly same - } else if (!schemasCompatible(storedKeySchema, keySchema)) { - throw StateStoreErrors.stateStoreKeySchemaNotCompatible(storedKeySchema.toString, - keySchema.toString) - } else if (!ignoreValueSchema && !schemasCompatible(storedValueSchema, valueSchema)) { - throw StateStoreErrors.stateStoreValueSchemaNotCompatible(storedValueSchema.toString, - valueSchema.toString) - } else { - logInfo("Detected schema change which is compatible. Allowing to put rows.") - } - } - - private def schemasCompatible(storedSchema: StructType, schema: StructType): Boolean = - DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema) - def readSchemaFile(): (StructType, StructType) = { val inStream = fm.open(schemaFileLocation) try { @@ -131,7 +101,8 @@ class StateSchemaCompatibilityChecker( createSchemaFile(newKeySchema, newValueSchema) } else { // validate if the new schema is compatible with the existing schema - check(existingSchema.get, (newKeySchema, newValueSchema), ignoreValueSchema) + StateSchemaCompatibilityChecker. + check(existingSchema.get, (newKeySchema, newValueSchema), ignoreValueSchema) } } @@ -139,9 +110,39 @@ class StateSchemaCompatibilityChecker( new Path(new Path(storeCpLocation, "_metadata"), "schema") } -object StateSchemaCompatibilityChecker { +object StateSchemaCompatibilityChecker extends Logging { val VERSION = 2 + /** + * Function to check if new state store schema is compatible with the existing schema. + * @param oldSchema - old state schema + * @param newSchema - new state schema + * @param ignoreValueSchema - whether to ignore value schema or not + */ + def check( + oldSchema: (StructType, StructType), + newSchema: (StructType, StructType), + ignoreValueSchema: Boolean) : Unit = { + val (storedKeySchema, storedValueSchema) = oldSchema + val (keySchema, valueSchema) = newSchema + + if (storedKeySchema.equals(keySchema) && + (ignoreValueSchema || storedValueSchema.equals(valueSchema))) { + // schema is exactly same + } else if (!schemasCompatible(storedKeySchema, keySchema)) { + throw StateStoreErrors.stateStoreKeySchemaNotCompatible(storedKeySchema.toString, + keySchema.toString) + } else if (!ignoreValueSchema && !schemasCompatible(storedValueSchema, valueSchema)) { + throw StateStoreErrors.stateStoreValueSchemaNotCompatible(storedValueSchema.toString, + valueSchema.toString) + } else { + logInfo("Detected schema change which is compatible. Allowing to put rows.") + } + } + + private def schemasCompatible(storedSchema: StructType, schema: StructType): Boolean = + DataType.equalsIgnoreNameAndCompatibleNullability(schema, storedSchema) + private def disallowBinaryInequalityColumn(schema: StructType): Unit = { if (!UnsafeRowUtils.isBinaryStable(schema)) { throw new SparkUnsupportedOperationException( @@ -172,7 +173,7 @@ object StateSchemaCompatibilityChecker { newValueSchema: StructType, sessionState: SessionState, extraOptions: Map[String, String] = Map.empty, - storeName: String = StateStoreId.DEFAULT_STORE_NAME): Unit = { + storeName: String = StateStoreId.DEFAULT_STORE_NAME): Array[String] = { // SPARK-47776: collation introduces the concept of binary (in)equality, which means // in some collation we no longer be able to just compare the binary format of two // UnsafeRows to determine equality. For example, 'aaa' and 'AAA' can be "semantically" @@ -201,5 +202,6 @@ object StateSchemaCompatibilityChecker { if (storeConf.stateSchemaCheckEnabled && result.isDefined) { throw result.get } + Array(checker.schemaFileLocation.toString) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaV3File.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaV3File.scala new file mode 100644 index 0000000000000..38e6484728126 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateSchemaV3File.scala @@ -0,0 +1,99 @@ +/* + * 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.streaming.state + +import java.io.{InputStream, OutputStream} +import java.nio.charset.StandardCharsets.UTF_8 +import java.util.UUID + +import scala.io.{Source => IOSource} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.execution.streaming.CheckpointFileManager +import org.apache.spark.sql.execution.streaming.MetadataVersionUtil.validateVersion + +/** + * The StateSchemaV3File is used to write the schema of multiple column families. + * Right now, this is primarily used for the TransformWithState operator, which supports + * multiple column families to keep the data for multiple state variables. + * We only expect ColumnFamilySchemaV1 to be written and read from this file. + * @param hadoopConf Hadoop configuration that is used to read / write metadata files. + * @param path Path to the directory that will be used for writing metadata. + */ +class StateSchemaV3File( + hadoopConf: Configuration, + path: String) { + + val metadataPath = new Path(path) + + protected val fileManager: CheckpointFileManager = + CheckpointFileManager.create(metadataPath, hadoopConf) + + if (!fileManager.exists(metadataPath)) { + fileManager.mkdirs(metadataPath) + } + + private def deserialize(in: InputStream): List[ColumnFamilySchema] = { + val lines = IOSource.fromInputStream(in, UTF_8.name()).getLines() + + if (!lines.hasNext) { + throw new IllegalStateException("Incomplete log file in the offset commit log") + } + + val version = lines.next().trim + validateVersion(version, StateSchemaV3File.VERSION) + + lines.map(ColumnFamilySchemaV1.fromJson).toList + } + + private def serialize(schemas: List[ColumnFamilySchema], out: OutputStream): Unit = { + out.write(s"v${StateSchemaV3File.VERSION}".getBytes(UTF_8)) + out.write('\n') + out.write(schemas.map(_.json).mkString("\n").getBytes(UTF_8)) + } + + def addWithUUID(batchId: Long, metadata: List[ColumnFamilySchema]): Path = { + val schemaFilePath = new Path(metadataPath, s"${batchId}_${UUID.randomUUID().toString}") + write(schemaFilePath, out => serialize(metadata, out)) + schemaFilePath + } + + def getWithPath(schemaFilePath: Path): List[ColumnFamilySchema] = { + deserialize(fileManager.open(schemaFilePath)) + } + + protected def write( + batchMetadataFile: Path, + fn: OutputStream => Unit): Unit = { + val output = fileManager.createAtomic(batchMetadataFile, overwriteIfPossible = false) + try { + fn(output) + output.close() + } catch { + case e: Throwable => + output.cancel() + throw e + } + } +} + +object StateSchemaV3File { + val VERSION = 3 +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index 0dc5414b7398a..8dcde7927b4b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -27,6 +27,10 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.json4s.{JInt, JString} +import org.json4s.JsonAST.JValue +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.{SparkContext, SparkEnv, SparkException} import org.apache.spark.internal.{Logging, LogKeys, MDC} @@ -278,9 +282,33 @@ case class StateStoreCustomTimingMetric(name: String, desc: String) extends Stat SQLMetrics.createTimingMetric(sparkContext, desc) } -sealed trait KeyStateEncoderSpec +sealed trait KeyStateEncoderSpec { + def jsonValue: JValue + def json: String = compact(render(jsonValue)) +} -case class NoPrefixKeyStateEncoderSpec(keySchema: StructType) extends KeyStateEncoderSpec +object KeyStateEncoderSpec { + def fromJson(keySchema: StructType, m: Map[String, Any]): KeyStateEncoderSpec = { + // match on type + m("keyStateEncoderType").asInstanceOf[String] match { + case "NoPrefixKeyStateEncoderSpec" => + NoPrefixKeyStateEncoderSpec(keySchema) + case "RangeKeyScanStateEncoderSpec" => + val orderingOrdinals = m("orderingOrdinals"). + asInstanceOf[List[_]].map(_.asInstanceOf[BigInt].toInt) + RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) + case "PrefixKeyScanStateEncoderSpec" => + val numColsPrefixKey = m("numColsPrefixKey").asInstanceOf[BigInt].toInt + PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) + } + } +} + +case class NoPrefixKeyStateEncoderSpec(keySchema: StructType) extends KeyStateEncoderSpec { + override def jsonValue: JValue = { + ("keyStateEncoderType" -> JString("NoPrefixKeyStateEncoderSpec")) + } +} case class PrefixKeyScanStateEncoderSpec( keySchema: StructType, @@ -288,6 +316,11 @@ case class PrefixKeyScanStateEncoderSpec( if (numColsPrefixKey == 0 || numColsPrefixKey >= keySchema.length) { throw StateStoreErrors.incorrectNumOrderingColsForPrefixScan(numColsPrefixKey.toString) } + + override def jsonValue: JValue = { + ("keyStateEncoderType" -> JString("PrefixKeyScanStateEncoderSpec")) ~ + ("numColsPrefixKey" -> JInt(numColsPrefixKey)) + } } /** Encodes rows so that they can be range-scanned based on orderingOrdinals */ @@ -297,6 +330,11 @@ case class RangeKeyScanStateEncoderSpec( if (orderingOrdinals.isEmpty || orderingOrdinals.length > keySchema.length) { throw StateStoreErrors.incorrectNumOrderingColsForRangeScan(orderingOrdinals.length.toString) } + + override def jsonValue: JValue = { + ("keyStateEncoderType" -> JString("RangeKeyScanStateEncoderSpec")) ~ + ("orderingOrdinals" -> orderingOrdinals.map(JInt(_))) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala index 0844a87739765..4ac813291c00b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala @@ -150,6 +150,14 @@ object StateStoreErrors { new StateStoreValueSchemaNotCompatible(storedValueSchema, newValueSchema) } + def stateStoreColumnFamilyMismatch( + columnFamilyName: String, + oldColumnFamilySchema: String, + newColumnFamilySchema: String): StateStoreColumnFamilyMismatch = { + new StateStoreColumnFamilyMismatch( + columnFamilyName, oldColumnFamilySchema, newColumnFamilySchema) + } + def stateStoreSnapshotFileNotFound(fileToRead: String, clazz: String): StateStoreSnapshotFileNotFound = { new StateStoreSnapshotFileNotFound(fileToRead, clazz) @@ -194,6 +202,17 @@ class StateStoreUnsupportedOperationException(operationType: String, entity: Str messageParameters = Map("operationType" -> operationType, "entity" -> entity) ) +class StateStoreColumnFamilyMismatch( + columnFamilyName: String, + oldColumnFamilySchema: String, + newColumnFamilySchema: String) + extends SparkUnsupportedOperationException( + errorClass = "STATE_STORE_COLUMN_FAMILY_SCHEMA_INCOMPATIBLE", + messageParameters = Map( + "columnFamilyName" -> columnFamilyName, + "oldColumnFamilySchema" -> oldColumnFamilySchema, + "newColumnFamilySchema" -> newColumnFamilySchema)) + class StatefulProcessorCannotPerformOperationWithInvalidTimeMode( operationType: String, timeMode: String) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index 3d5c418c7f541..94d976b568a5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -75,7 +75,10 @@ trait StatefulOperator extends SparkPlan { // Function used to record state schema for the first time and validate it against proposed // schema changes in the future. Runs as part of a planning rule on the driver. - def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit + // Returns the schema file path for operators that write this to the metadata file, + // otherwise None + def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): Array[String] } /** @@ -430,7 +433,9 @@ case class StateStoreRestoreExec( private[sql] val stateManager = StreamingAggregationStateManager.createStateManager( keyExpressions, child.output, stateFormatVersion) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keyExpressions.toStructType, stateManager.getStateValueSchema, session.sessionState) } @@ -497,7 +502,9 @@ case class StateStoreSaveExec( private[sql] val stateManager = StreamingAggregationStateManager.createStateManager( keyExpressions, child.output, stateFormatVersion) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keyExpressions.toStructType, stateManager.getStateValueSchema, session.sessionState) } @@ -706,7 +713,9 @@ case class SessionWindowStateStoreRestoreExec( private val stateManager = StreamingSessionWindowStateManager.createStateManager( keyWithoutSessionExpressions, sessionExpression, child.output, stateFormatVersion) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, stateManager.getStateKeySchema, stateManager.getStateValueSchema, session.sessionState) } @@ -793,7 +802,9 @@ case class SessionWindowStateStoreSaveExec( private val stateManager = StreamingSessionWindowStateManager.createStateManager( keyWithoutSessionExpressions, sessionExpression, child.output, stateFormatVersion) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, stateManager.getStateKeySchema, stateManager.getStateValueSchema, session.sessionState) } @@ -1106,7 +1117,9 @@ case class StreamingDeduplicateExec( override protected def withNewChildInternal(newChild: SparkPlan): StreamingDeduplicateExec = copy(child = newChild) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keyExpressions.toStructType, schemaForValueRow, session.sessionState, extraOptionOnStateStore) } @@ -1181,7 +1194,9 @@ case class StreamingDeduplicateWithinWatermarkExec( override def shortName: String = "dedupeWithinWatermark" - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keyExpressions.toStructType, schemaForValueRow, session.sessionState, extraOptionOnStateStore) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala index a3cb66f914965..7b3d393ec75d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/streamingLimits.scala @@ -47,7 +47,9 @@ case class StreamingGlobalLimitExec( private val keySchema = StructType(Array(StructField("key", NullType))) private val valueSchema = StructType(Array(StructField("value", LongType))) - override def validateAndMaybeEvolveStateSchema(hadoopConf: Configuration): Unit = { + override def validateAndMaybeEvolveStateSchema( + hadoopConf: Configuration, batchId: Long, stateSchemaVersion: Int): + Array[String] = { StateSchemaCompatibilityChecker.validateAndMaybeEvolveStateSchema(getStateInfo, hadoopConf, keySchema, valueSchema, session.sessionState) } 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 98b2030f1bac4..2c4111ec026ae 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 @@ -30,6 +30,8 @@ import scala.util.Random import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ @@ -1627,6 +1629,30 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] keyRow, keySchema, valueRow, keySchema, storeConf) } + test("test serialization and deserialization of NoPrefixKeyStateEncoderSpec") { + implicit val formats: DefaultFormats.type = DefaultFormats + val encoderSpec = NoPrefixKeyStateEncoderSpec(keySchema) + val jsonMap = JsonMethods.parse(encoderSpec.json).extract[Map[String, Any]] + val deserializedEncoderSpec = KeyStateEncoderSpec.fromJson(keySchema, jsonMap) + assert(encoderSpec == deserializedEncoderSpec) + } + + test("test serialization and deserialization of PrefixKeyScanStateEncoderSpec") { + implicit val formats: DefaultFormats.type = DefaultFormats + val encoderSpec = PrefixKeyScanStateEncoderSpec(keySchema, 1) + val jsonMap = JsonMethods.parse(encoderSpec.json).extract[Map[String, Any]] + val deserializedEncoderSpec = KeyStateEncoderSpec.fromJson(keySchema, jsonMap) + assert(encoderSpec == deserializedEncoderSpec) + } + + test("test serialization and deserialization of RangeKeyScanStateEncoderSpec") { + implicit val formats: DefaultFormats.type = DefaultFormats + val encoderSpec = RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) + val jsonMap = JsonMethods.parse(encoderSpec.json).extract[Map[String, Any]] + val deserializedEncoderSpec = KeyStateEncoderSpec.fromJson(keySchema, jsonMap) + assert(encoderSpec == deserializedEncoderSpec) + } + /** Return a new provider with a random id */ def newStoreProvider(): ProviderClass diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index a7667723cd0ec..f71967d25993a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -25,7 +25,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, Encoders} import org.apache.spark.sql.catalyst.util.stringToFile import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateStoreMultipleColumnFamiliesNotSupportedException} +import org.apache.spark.sql.execution.streaming.TransformWithStateKeyValueRowSchema.{KEY_ROW_SCHEMA, VALUE_ROW_SCHEMA} +import org.apache.spark.sql.execution.streaming.state.{AlsoTestWithChangelogCheckpointingEnabled, ColumnFamilySchemaV1, NoPrefixKeyStateEncoderSpec, RocksDBStateStoreProvider, StatefulProcessorCannotPerformOperationWithInvalidHandleState, StateSchemaV3File, StateStoreMultipleColumnFamiliesNotSupportedException} import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock @@ -785,6 +786,79 @@ class TransformWithStateSuite extends StateStoreMetricsTest } } } + + test("transformWithState - verify StateSchemaV3 serialization and deserialization" + + " works with one batch") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { checkpointDir => + val schema = List(ColumnFamilySchemaV1( + "countState", + KEY_ROW_SCHEMA, + VALUE_ROW_SCHEMA, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), + None + )) + + val schemaFile = new StateSchemaV3File( + spark.sessionState.newHadoopConf(), checkpointDir.getCanonicalPath) + val path = schemaFile.addWithUUID(0, schema) + + assert(schemaFile.getWithPath(path) == schema) + } + } + } + + test("transformWithState - verify StateSchemaV3 serialization and deserialization" + + " works with multiple batches") { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> + classOf[RocksDBStateStoreProvider].getName, + SQLConf.SHUFFLE_PARTITIONS.key -> + TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) { + withTempDir { checkpointDir => + + val schema0 = List(ColumnFamilySchemaV1( + "countState", + KEY_ROW_SCHEMA, + VALUE_ROW_SCHEMA, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), + None + )) + + val schema1 = List( + ColumnFamilySchemaV1( + "countState", + KEY_ROW_SCHEMA, + VALUE_ROW_SCHEMA, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), + None + ), + ColumnFamilySchemaV1( + "mostRecent", + KEY_ROW_SCHEMA, + VALUE_ROW_SCHEMA, + NoPrefixKeyStateEncoderSpec(KEY_ROW_SCHEMA), + None + ) + ) + + val schemaFile = new StateSchemaV3File(spark.sessionState.newHadoopConf(), + checkpointDir.getCanonicalPath) + val path0 = schemaFile.addWithUUID(0, schema0) + + assert(schemaFile.getWithPath(path0) == schema0) + + // test the case where we are trying to add the schema after + // restarting after a few batches + val path1 = schemaFile.addWithUUID(3, schema1) + val latestSchema = schemaFile.getWithPath(path1) + + assert(latestSchema == schema1) + } + } + } } class TransformWithStateValidationSuite extends StateStoreMetricsTest { From 2e3a6fbb342d77a4a97cfe8de3941d1a3bc7adfd Mon Sep 17 00:00:00 2001 From: David Milicevic Date: Thu, 11 Jul 2024 11:12:19 +0800 Subject: [PATCH 25/64] [SPARK-48529][SQL] Introduction of Labels in SQL Scripting ### What changes were proposed in this pull request? Previous [PR1](https://github.com/apache/spark/pull/46665) and [PR2](https://github.com/apache/spark/pull/46665) introduced parser and interpreter changes for SQL Scripting. This PR is a follow-up to introduce the concept of labels for SQL Scripting language and proposes the following changes: - Changes grammar to support labels at start and end of the compound statements. - Updates visitor functions for compound nodes in the syntax tree in AstBuilder to check if labels are present and valid. More details can be found in [Jira item](https://issues.apache.org/jira/browse/SPARK-48529) for this task and its parent (where the design doc is uploaded as well). ### Why are the changes needed? The intent is to add support for various SQL scripting concepts like loops, leave & iterate statements. ### Does this PR introduce any user-facing change? No. This PR is among first PRs in series of PRs that will introduce changes to sql() API to add support for SQL scripting, but for now, the API remains unchanged. In the future, the API will remain the same as well, but it will have new possibility to execute SQL scripts. ### How was this patch tested? There are tests for newly introduced parser changes: SqlScriptingParserSuite - unit tests for execution nodes. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47146 from miland-db/sql_batch_labels. Lead-authored-by: David Milicevic Co-authored-by: Milan Dankovic Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/SqlBaseParser.g4 | 10 +- .../sql/catalyst/parser/AstBuilder.scala | 31 +++++- .../parser/SqlScriptingLogicalOperators.scala | 7 +- .../parser/SqlScriptingParserSuite.scala | 100 +++++++++++++++++- 4 files changed, 140 insertions(+), 8 deletions(-) diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 index 3f6f2772f4961..dc2c2f079461c 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseParser.g4 @@ -52,7 +52,7 @@ singleCompoundStatement ; beginEndCompoundBlock - : BEGIN compoundBody END + : beginLabel? BEGIN compoundBody END endLabel? ; compoundBody @@ -68,6 +68,14 @@ singleStatement : statement SEMICOLON* EOF ; +beginLabel + : multipartIdentifier COLON + ; + +endLabel + : multipartIdentifier + ; + singleExpression : namedExpression EOF ; 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 8e80b9a497c8d..7451d7568b7f2 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 @@ -122,7 +122,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { visit(s).asInstanceOf[CompoundBody] }.getOrElse { val logicalPlan = visitSingleStatement(ctx.singleStatement()) - CompoundBody(Seq(SingleStatement(parsedPlan = logicalPlan))) + CompoundBody(Seq(SingleStatement(parsedPlan = logicalPlan)), + Some(java.util.UUID.randomUUID.toString.toLowerCase(Locale.ROOT))) } } @@ -130,20 +131,40 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging { visit(ctx.beginEndCompoundBlock()).asInstanceOf[CompoundBody] } - private def visitCompoundBodyImpl(ctx: CompoundBodyContext): CompoundBody = { + private def visitCompoundBodyImpl( + ctx: CompoundBodyContext, + label: Option[String]): CompoundBody = { val buff = ListBuffer[CompoundPlanStatement]() ctx.compoundStatements.forEach(compoundStatement => { buff += visit(compoundStatement).asInstanceOf[CompoundPlanStatement] }) - CompoundBody(buff.toSeq) + + CompoundBody(buff.toSeq, label) } override def visitBeginEndCompoundBlock(ctx: BeginEndCompoundBlockContext): CompoundBody = { - visitCompoundBodyImpl(ctx.compoundBody()) + val beginLabelCtx = Option(ctx.beginLabel()) + val endLabelCtx = Option(ctx.endLabel()) + + (beginLabelCtx, endLabelCtx) match { + case (Some(bl: BeginLabelContext), Some(el: EndLabelContext)) + if bl.multipartIdentifier().getText.nonEmpty && + bl.multipartIdentifier().getText.toLowerCase(Locale.ROOT) != + el.multipartIdentifier().getText.toLowerCase(Locale.ROOT) => + throw SparkException.internalError("Both labels should be same.") + case (None, Some(_)) => + throw SparkException.internalError("End label can't exist without begin label.") + case _ => + } + + val labelText = beginLabelCtx. + map(_.multipartIdentifier().getText).getOrElse(java.util.UUID.randomUUID.toString). + toLowerCase(Locale.ROOT) + visitCompoundBodyImpl(ctx.compoundBody(), Some(labelText)) } override def visitCompoundBody(ctx: CompoundBodyContext): CompoundBody = { - visitCompoundBodyImpl(ctx) + visitCompoundBodyImpl(ctx, None) } override def visitCompoundStatement(ctx: CompoundStatementContext): CompoundPlanStatement = diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala index 5b2b6ab95b459..a68898d151e9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SqlScriptingLogicalOperators.scala @@ -51,5 +51,10 @@ case class SingleStatement(parsedPlan: LogicalPlan) /** * Logical operator for a compound body. Contains all statements within the compound body. * @param collection Collection of statements within the compound body. + * @param label Label set to CompoundBody by user or UUID otherwise. + * It can be None in case when CompoundBody is not part of BeginEndCompoundBlock + * for example when CompoundBody is inside loop or conditional block. */ -case class CompoundBody(collection: Seq[CompoundPlanStatement]) extends CompoundPlanStatement +case class CompoundBody( + collection: Seq[CompoundPlanStatement], + label: Option[String]) extends CompoundPlanStatement 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 aa72e409f528b..1f58c861e3fea 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.parser -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.sql.catalyst.plans.SQLHelper class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { @@ -161,6 +161,104 @@ class SqlScriptingParserSuite extends SparkFunSuite with SQLHelper { == "SELECT 3") } + test("compound: beginLabel") { + val sqlScriptText = + """ + |lbl: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 5) + assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) + assert(tree.label.contains("lbl")) + } + + test("compound: beginLabel + endLabel") { + val sqlScriptText = + """ + |lbl: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END lbl""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 5) + assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) + assert(tree.label.contains("lbl")) + } + + test("compound: beginLabel + endLabel with different values") { + val sqlScriptText = + """ + |lbl_begin: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END lbl_end""".stripMargin + val e = intercept[SparkException] { + parseScript(sqlScriptText) + } + assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getMessage.contains("Both labels should be same.")) + } + + test("compound: endLabel") { + val sqlScriptText = + """ + |BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END lbl""".stripMargin + val e = intercept[SparkException] { + parseScript(sqlScriptText) + } + assert(e.getErrorClass === "INTERNAL_ERROR") + assert(e.getMessage.contains("End label can't exist without begin label.")) + } + + test("compound: beginLabel + endLabel with different casing") { + val sqlScriptText = + """ + |LBL: BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END lbl""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 5) + assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) + assert(tree.label.contains("lbl")) + } + + test("compound: no labels provided") { + val sqlScriptText = + """ + |BEGIN + | SELECT 1; + | SELECT 2; + | INSERT INTO A VALUES (a, b, 3); + | SELECT a, b, c FROM T; + | SELECT * FROM T; + |END""".stripMargin + val tree = parseScript(sqlScriptText) + assert(tree.collection.length == 5) + assert(tree.collection.forall(_.isInstanceOf[SingleStatement])) + assert(tree.label.nonEmpty) + } + // Helper methods def cleanupStatementString(statementStr: String): String = { statementStr From 633238ad84b26c8d1fd4dd451d6ca045af5b4936 Mon Sep 17 00:00:00 2001 From: Wei Guo Date: Thu, 11 Jul 2024 12:26:56 +0900 Subject: [PATCH 26/64] [SPARK-48858][PYTHON] Remove deprecated `setDaemon` method call of `Thread` in `log_communication.py` ### What changes were proposed in this pull request? This PR aims to remove deprecated `setDaemon` method call of `Thread` in `log_communication.py`. This is last one used. ### Why are the changes needed? Clean up deprecated apis. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47282 from wayneguow/remove_py_dep. Authored-by: Wei Guo Signed-off-by: Hyukjin Kwon --- python/pyspark/ml/torch/log_communication.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/ml/torch/log_communication.py b/python/pyspark/ml/torch/log_communication.py index 8efa83e62c367..ad1fc810f3ef4 100644 --- a/python/pyspark/ml/torch/log_communication.py +++ b/python/pyspark/ml/torch/log_communication.py @@ -83,7 +83,7 @@ def serve_task(port: int) -> None: self.port = LogStreamingServer._get_free_port(spark_host_address) self.serve_thread = threading.Thread(target=serve_task, args=(self.port,)) - self.serve_thread.setDaemon(True) + self.serve_thread.daemon = True self.serve_thread.start() def shutdown(self) -> None: From 225b6cbf129761705d882b5b68c82ac0d7325e92 Mon Sep 17 00:00:00 2001 From: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Date: Thu, 11 Jul 2024 13:30:59 +0900 Subject: [PATCH 27/64] [SPARK-48280][SQL][FOLLOWUP] Improve collation testing surface area using expression walking ### What changes were proposed in this pull request? Followup: small correction. ### Why are the changes needed? UTF8_BINARY_LCASE no longer exists in Spark. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47216 from uros-db/fix-walker. Authored-by: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/CollationExpressionWalkerSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 d582167478da9..a639367e8ca53 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 @@ -604,7 +604,7 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi "unix_timestamp", "localtimestamp", "now", - // need to skip as plans differ in STRING <-> STRING COLLATE UTF8_BINARY_LCASE + // need to skip as plans differ in STRING <-> STRING COLLATE UTF8_LCASE "current_timezone", "schema_of_variant", // need to skip as result is expected to differ From b4e3c2ad5ac8c924441c7967d032b4a9652eaa88 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 11 Jul 2024 13:19:37 +0800 Subject: [PATCH 28/64] [SPARK-48851][SQL] Change the value of `SCHEMA_NOT_FOUND` from `namespace` to `catalog.namespace` ### What changes were proposed in this pull request? The pr aims to change the value of `SCHEMA_NOT_FOUND` from `namespace` to `catalog.namespace`. ### Why are the changes needed? As discussing https://github.com/apache/spark/pull/47038#discussion_r1670544805, we should provide more friendly and clear prompt error message. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Update existed UT & Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47276 from panbingkun/db_with_catalog. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../client/GrpcExceptionConverter.scala | 4 +- .../apache/spark/sql/ClientE2ETestSuite.scala | 6 +-- .../sql/catalyst/catalog/SessionCatalog.scala | 12 ++--- .../sql/errors/QueryCompilationErrors.scala | 4 +- .../catalog/SessionCatalogSuite.scala | 48 +++++++++---------- .../catalog/InMemoryTableCatalog.scala | 4 +- .../datasources/v2/DropNamespaceExec.scala | 2 +- .../datasources/v2/V2SessionCatalog.scala | 20 ++++---- ...double-quoted-identifiers-disabled.sql.out | 4 +- .../double-quoted-identifiers-enabled.sql.out | 8 ++-- .../double-quoted-identifiers.sql.out | 4 +- .../analyzer-results/show-views.sql.out | 4 +- ...double-quoted-identifiers-disabled.sql.out | 4 +- .../double-quoted-identifiers-enabled.sql.out | 8 ++-- .../results/double-quoted-identifiers.sql.out | 4 +- .../sql-tests/results/show-views.sql.out | 4 +- .../spark/sql/StatisticsCollectionSuite.scala | 2 +- .../sql/connector/DataSourceV2SQLSuite.scala | 10 ++-- .../AlterNamespaceSetLocationSuiteBase.scala | 2 +- ...AlterNamespaceSetPropertiesSuiteBase.scala | 2 +- ...terNamespaceUnsetPropertiesSuiteBase.scala | 2 +- .../command/DescribeNamespaceSuiteBase.scala | 2 +- .../command/DropNamespaceSuiteBase.scala | 4 +- .../command/ShowTablesSuiteBase.scala | 4 +- .../command/v1/DropNamespaceSuite.scala | 2 +- .../command/v1/ShowNamespacesSuite.scala | 2 +- .../v2/V2SessionCatalogSuite.scala | 6 +-- .../spark/sql/hive/MultiDatabaseSuite.scala | 6 +-- .../sql/hive/execution/HiveDDLSuite.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- 30 files changed, 96 insertions(+), 94 deletions(-) diff --git a/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala b/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala index 3247bc6438027..f29291594069d 100644 --- a/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala +++ b/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala @@ -31,7 +31,7 @@ import org.apache.spark.{QueryContext, QueryContextType, SparkArithmeticExceptio import org.apache.spark.connect.proto.{FetchErrorDetailsRequest, FetchErrorDetailsResponse, SparkConnectServiceGrpc, UserContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.trees.Origin import org.apache.spark.sql.streaming.StreamingQueryException @@ -219,6 +219,8 @@ private[client] object GrpcExceptionConverter { params.errorClass.orNull, params.messageParameters, params.cause)), + errorConstructor(params => + new NoSuchNamespaceException(params.errorClass.orNull, params.messageParameters)), errorConstructor(params => new NoSuchTableException(params.errorClass.orNull, params.messageParameters, params.cause)), errorConstructor[NumberFormatException](params => diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index ef0b4607fea6b..95410ab6a7e7d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkArithmeticException, SparkException, SparkUpgradeException} import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION} -import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, TableAlreadyExistsException, TempTableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, TableAlreadyExistsException, TempTableAlreadyExistsException} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.StringEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema import org.apache.spark.sql.catalyst.parser.ParseException @@ -165,8 +165,8 @@ class ClientE2ETestSuite } } - test("throw NoSuchDatabaseException") { - val ex = intercept[NoSuchDatabaseException] { + test("throw NoSuchNamespaceException") { + val ex = intercept[NoSuchNamespaceException] { spark.sql("use database123") } assert(ex.getErrorClass != null) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index 0e0852d0a550d..fa271eee73d02 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -250,7 +250,7 @@ class SessionCatalog( private def requireDbExists(db: String): Unit = { if (!databaseExists(db)) { - throw new NoSuchDatabaseException(db) + throw new NoSuchNamespaceException(Seq(CatalogManager.SESSION_CATALOG_NAME, db)) } } @@ -291,7 +291,8 @@ class SessionCatalog( def dropDatabase(db: String, ignoreIfNotExists: Boolean, cascade: Boolean): Unit = { val dbName = format(db) if (dbName == DEFAULT_DATABASE) { - throw QueryCompilationErrors.cannotDropDefaultDatabaseError(dbName) + throw QueryCompilationErrors.cannotDropDefaultDatabaseError( + Seq(CatalogManager.SESSION_CATALOG_NAME, dbName)) } if (!ignoreIfNotExists) { requireDbExists(dbName) @@ -527,7 +528,7 @@ class SessionCatalog( * We replace char/varchar with "annotated" string type in the table schema, as the query * engine doesn't support char/varchar yet. */ - @throws[NoSuchDatabaseException] + @throws[NoSuchNamespaceException] @throws[NoSuchTableException] def getTableMetadata(name: TableIdentifier): CatalogTable = { val t = getTableRawMetadata(name) @@ -538,7 +539,7 @@ class SessionCatalog( * Retrieve the metadata of an existing permanent table/view. If no database is specified, * assume the table/view is in the current database. */ - @throws[NoSuchDatabaseException] + @throws[NoSuchNamespaceException] @throws[NoSuchTableException] def getTableRawMetadata(name: TableIdentifier): CatalogTable = { val qualifiedIdent = qualifyIdentifier(name) @@ -556,7 +557,7 @@ class SessionCatalog( * For example, if none of the requested tables could be retrieved, an empty list is returned. * There is no guarantee of ordering of the returned tables. */ - @throws[NoSuchDatabaseException] + @throws[NoSuchNamespaceException] def getTablesByName(names: Seq[TableIdentifier]): Seq[CatalogTable] = { if (names.nonEmpty) { val qualifiedIdents = names.map(qualifyIdentifier) @@ -1056,7 +1057,6 @@ class SessionCatalog( getTempViewOrPermanentTableMetadata(ident).tableType == CatalogTableType.VIEW } catch { case _: NoSuchTableException => false - case _: NoSuchDatabaseException => false case _: NoSuchNamespaceException => 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 7e0e2f80505df..7d32bd48f1659 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 @@ -1072,10 +1072,10 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat messageParameters = Map("database" -> database)) } - def cannotDropDefaultDatabaseError(database: String): Throwable = { + def cannotDropDefaultDatabaseError(nameParts: Seq[String]): Throwable = { new AnalysisException( errorClass = "UNSUPPORTED_FEATURE.DROP_DATABASE", - messageParameters = Map("database" -> toSQLId(database))) + messageParameters = Map("database" -> toSQLId(nameParts))) } def cannotUsePreservedDatabaseAsCurrentDatabaseError(database: String): Throwable = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 48f829548bb65..f5f6fac96872f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -229,7 +229,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("get database should throw exception when the database does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.getDatabaseMetadata("db_that_does_not_exist") } } @@ -283,7 +283,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("drop database when the database does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false) } catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = true, cascade = false) @@ -295,7 +295,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { catalog.setCurrentDatabase("db1") assert(catalog.getCurrentDatabase == "db1") catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = true) - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.createTable(newTable("tbl1", "db1"), ignoreIfExists = false) } catalog.setCurrentDatabase("default") @@ -321,7 +321,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("alter database should throw exception when the database does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.alterDatabase(newDb("unknown_db")) } } @@ -332,7 +332,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { assert(catalog.getCurrentDatabase == "default") catalog.setCurrentDatabase("db2") assert(catalog.getCurrentDatabase == "db2") - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.setCurrentDatabase("deebo") } catalog.createDatabase(newDb("deebo"), ignoreIfExists = false) @@ -370,10 +370,10 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("create table when database does not exist") { withBasicCatalog { catalog => // Creating table in non-existent database should always fail - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = false) } - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = true) } // Table already exists @@ -419,11 +419,11 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("drop table when database/table does not exist") { withBasicCatalog { catalog => // Should always throw exception when the database does not exist - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = false, purge = false) } - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true, purge = false) } @@ -494,7 +494,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("rename table when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2")) } intercept[NoSuchTableException] { @@ -543,7 +543,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("alter table when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.alterTable(newTable("tbl1", "unknown_db")) } intercept[NoSuchTableException] { @@ -608,7 +608,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("get table when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.getTableMetadata(TableIdentifier("tbl1", Some("unknown_db"))) } intercept[NoSuchTableException] { @@ -856,7 +856,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl4"), TableIdentifier("tbl1", Some("db2")), TableIdentifier("tbl2", Some("db2")))) - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.listTables("unknown_db") } } @@ -876,7 +876,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { TableIdentifier("tbl2", Some("db2")))) assert(catalog.listTables("db2", "*1").toSet == Set(TableIdentifier("tbl1"), TableIdentifier("tbl1", Some("db2")))) - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.listTables("unknown_db", "*") } } @@ -970,7 +970,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("create partitions when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.createPartitions( TableIdentifier("tbl1", Some("unknown_db")), Seq(), ignoreIfExists = false) } @@ -1077,7 +1077,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("drop partitions when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.dropPartitions( TableIdentifier("tbl1", Some("unknown_db")), Seq(), @@ -1177,7 +1177,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("get partition when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.getPartition(TableIdentifier("tbl1", Some("unknown_db")), part1.spec) } intercept[NoSuchTableException] { @@ -1258,7 +1258,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("rename partitions when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.renamePartitions( TableIdentifier("tbl1", Some("unknown_db")), Seq(part1.spec), Seq(part2.spec)) } @@ -1349,7 +1349,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("alter partitions when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.alterPartitions(TableIdentifier("tbl1", Some("unknown_db")), Seq(part1)) } intercept[NoSuchTableException] { @@ -1497,7 +1497,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("list partitions when database/table does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.listPartitions(TableIdentifier("tbl1", Some("unknown_db"))) } intercept[NoSuchTableException] { @@ -1544,7 +1544,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("create function when database does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.createFunction( newFunc("func5", Some("does_not_exist")), ignoreIfExists = false) } @@ -1687,7 +1687,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("drop function when database/function does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.dropFunction( FunctionIdentifier("something", Some("unknown_db")), ignoreIfNotExists = false) } @@ -1746,7 +1746,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("get function when database/function does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.getFunctionMetadata(FunctionIdentifier("func1", Some("unknown_db"))) } intercept[NoSuchFunctionException] { @@ -1799,7 +1799,7 @@ abstract class SessionCatalogSuite extends AnalysisTest with Eventually { test("list functions when database does not exist") { withBasicCatalog { catalog => - intercept[NoSuchDatabaseException] { + intercept[NoSuchNamespaceException] { catalog.listFunctions("unknown_db", "func*") } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala index 654fa0719cf82..0515237adfae5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/connector/catalog/InMemoryTableCatalog.scala @@ -210,7 +210,7 @@ class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamesp case _ if namespaceExists(namespace) => util.Collections.emptyMap[String, String] case _ => - throw new NoSuchNamespaceException(namespace) + throw new NoSuchNamespaceException(name() +: namespace) } } @@ -256,7 +256,7 @@ class InMemoryTableCatalog extends BasicInMemoryTableCatalog with SupportsNamesp if (namespace.isEmpty || namespaceExists(namespace)) { super.listTables(namespace) } else { - throw new NoSuchNamespaceException(namespace) + throw new NoSuchNamespaceException(name() +: namespace) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala index 5d302055e7d91..2f995ec938147 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropNamespaceExec.scala @@ -45,7 +45,7 @@ case class DropNamespaceExec( throw QueryCompilationErrors.cannotDropNonemptyNamespaceError(namespace) } } else if (!ifExists) { - throw QueryCompilationErrors.noSuchNamespaceError(ns) + throw QueryCompilationErrors.noSuchNamespaceError(catalog.name() +: ns) } Seq.empty diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index e619c59a7540c..bc1e2c92faa83 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -25,7 +25,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.catalyst.{FunctionIdentifier, QualifiedTableName, SQLConfHelper, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, CatalogUtils, ClusterBySpec, SessionCatalog} import org.apache.spark.sql.catalyst.util.TypeUtils._ import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogV2Util, Column, FunctionCatalog, Identifier, NamespaceChange, SupportsNamespaces, Table, TableCatalog, TableCatalogCapability, TableChange, V1Table} @@ -68,7 +68,7 @@ class V2SessionCatalog(catalog: SessionCatalog) .map(ident => Identifier.of(ident.database.map(Array(_)).getOrElse(Array()), ident.table)) .toArray case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } } @@ -123,7 +123,7 @@ class V2SessionCatalog(catalog: SessionCatalog) V1Table(table) } } catch { - case _: NoSuchDatabaseException => + case _: NoSuchNamespaceException => throw QueryCompilationErrors.noSuchTableError(ident) } } @@ -380,7 +380,7 @@ class V2SessionCatalog(catalog: SessionCatalog) case Array(db) if catalog.databaseExists(db) => Array() case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } } @@ -390,12 +390,12 @@ class V2SessionCatalog(catalog: SessionCatalog) try { catalog.getDatabaseMetadata(db).toMetadata } catch { - case _: NoSuchDatabaseException => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + case _: NoSuchNamespaceException => + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } } @@ -430,7 +430,7 @@ class V2SessionCatalog(catalog: SessionCatalog) toCatalogDatabase(db, CatalogV2Util.applyNamespaceChanges(metadata, changes))) case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } } @@ -446,7 +446,7 @@ class V2SessionCatalog(catalog: SessionCatalog) false case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } def isTempView(ident: Identifier): Boolean = { @@ -465,7 +465,7 @@ class V2SessionCatalog(catalog: SessionCatalog) Identifier.of(Array(funcIdent.database.get), funcIdent.identifier) }.toArray case _ => - throw QueryCompilationErrors.noSuchNamespaceError(namespace) + throw QueryCompilationErrors.noSuchNamespaceError(name() +: namespace) } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out index f7b0e3370f9f4..a02bf525f947d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-disabled.sql.out @@ -134,12 +134,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query USE SCHEMA `not_exist` -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out index f241f9bd6867c..22dfeac5fd0b6 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/ansi/double-quoted-identifiers-enabled.sql.out @@ -22,12 +22,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query USE SCHEMA "not_exist" -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } @@ -177,12 +177,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query USE SCHEMA `not_exist` -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out index f7b0e3370f9f4..a02bf525f947d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers.sql.out @@ -134,12 +134,12 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException -- !query USE SCHEMA `not_exist` -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out index ed3690ec5c6a3..d092590b143b5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/show-views.sql.out @@ -103,12 +103,12 @@ ShowViewsCommand showdb, view_*, [namespace#x, viewName#x, isTemporary#x] -- !query SHOW VIEWS IN wrongdb LIKE 'view_*' -- !query analysis -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`wrongdb`" + "schemaName" : "`spark_catalog`.`wrongdb`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out index 861afcc7b1005..81a98a60590f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-disabled.sql.out @@ -154,12 +154,12 @@ USE SCHEMA `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out index 2854e09aab6bd..2444c399a87ec 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/double-quoted-identifiers-enabled.sql.out @@ -26,12 +26,12 @@ USE SCHEMA "not_exist" -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } @@ -197,12 +197,12 @@ USE SCHEMA `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out index 861afcc7b1005..81a98a60590f0 100644 --- a/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/double-quoted-identifiers.sql.out @@ -154,12 +154,12 @@ USE SCHEMA `not_exist` -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`not_exist`" + "schemaName" : "`spark_catalog`.`not_exist`" } } diff --git a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out index bfed13683d9dd..249f5a1d95527 100644 --- a/sql/core/src/test/resources/sql-tests/results/show-views.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/show-views.sql.out @@ -138,12 +138,12 @@ SHOW VIEWS IN wrongdb LIKE 'view_*' -- !query schema struct<> -- !query output -org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException +org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException { "errorClass" : "SCHEMA_NOT_FOUND", "sqlState" : "42704", "messageParameters" : { - "schemaName" : "`wrongdb`" + "schemaName" : "`spark_catalog`.`wrongdb`" } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala index 5f1fa2904e341..919958d304f10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionSuite.scala @@ -850,7 +850,7 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`db_not_exists`")) + parameters = Map("schemaName" -> "`spark_catalog`.`db_not_exists`")) } test("SPARK-43383: Add rowCount statistics to LocalRelation") { 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 e776de7e8222c..51d7f270e1a53 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 @@ -28,7 +28,7 @@ import org.apache.spark.{SparkException, SparkRuntimeException, SparkUnsupported import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchDatabaseException, NoSuchNamespaceException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{CannotReplaceMissingTableException, NoSuchNamespaceException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.ColumnStat import org.apache.spark.sql.catalyst.statsEstimation.StatsEstimationTestBase @@ -1427,12 +1427,12 @@ class DataSourceV2SQLSuiteV1Filter } test("Use: v2 session catalog is used and namespace does not exist") { - val exception = intercept[NoSuchDatabaseException] { + val exception = intercept[AnalysisException] { sql("USE ns1") } checkError(exception, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`ns1`")) + parameters = Map("schemaName" -> "`spark_catalog`.`ns1`")) } test("SPARK-31100: Use: v2 catalog that implements SupportsNamespaces is used " + @@ -2588,7 +2588,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = intercept[AnalysisException](sql("COMMENT ON NAMESPACE abc IS NULL")), errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`abc`")) + parameters = Map("schemaName" -> "`spark_catalog`.`abc`")) // V2 non-session catalog is used. sql("CREATE NAMESPACE testcat.ns1") @@ -2598,7 +2598,7 @@ class DataSourceV2SQLSuiteV1Filter checkError( exception = intercept[AnalysisException](sql("COMMENT ON NAMESPACE testcat.abc IS NULL")), errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`abc`")) + parameters = Map("schemaName" -> "`testcat`.`abc`")) } private def checkNamespaceComment(namespace: String, comment: String): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala index 5b78665d878ef..6427338a6c52e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetLocationSuiteBase.scala @@ -63,7 +63,7 @@ trait AlterNamespaceSetLocationSuiteBase extends QueryTest with DDLCommandTestUt } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`not_exist`")) + parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } // Hive catalog does not support "ALTER NAMESPACE ... SET LOCATION", thus diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala index 7f5b3de4865c9..d2f2d75d86ce9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceSetPropertiesSuiteBase.scala @@ -50,7 +50,7 @@ trait AlterNamespaceSetPropertiesSuiteBase extends QueryTest with DDLCommandTest } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> s"`$ns`")) + parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } test("basic test") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala index 1d43cc5938487..c00f3f99f41f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterNamespaceUnsetPropertiesSuiteBase.scala @@ -57,7 +57,7 @@ trait AlterNamespaceUnsetPropertiesSuiteBase extends QueryTest with DDLCommandTe } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> s"`$ns`")) + parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } test("basic test") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala index 1309ba05b3f19..6945352564e1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeNamespaceSuiteBase.scala @@ -44,7 +44,7 @@ trait DescribeNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`db1`")) + parameters = Map("schemaName" -> s"`$catalog`.`$ns`")) } test("Keep the legacy output schema") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala index 6eb4465124a69..2243517550b2c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DropNamespaceSuiteBase.scala @@ -38,7 +38,7 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { protected def isCasePreserving: Boolean = true protected def namespaceAlias: String = "namespace" - protected def checkNamespace(expected: Seq[String]) = { + protected def checkNamespace(expected: Seq[String]): Unit = { val df = spark.sql(s"SHOW NAMESPACES IN $catalog") assert(df.schema === new StructType().add("namespace", StringType, false)) checkAnswer(df, expected.map(Row(_))) @@ -65,7 +65,7 @@ trait DropNamespaceSuiteBase extends QueryTest with DDLCommandTestUtils { } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`unknown`")) + parameters = Map("schemaName" -> s"`$catalog`.`unknown`")) } test("drop non-empty namespace with a non-cascading mode") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala index d6b91bcf3eb8e..1890726a376ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowTablesSuiteBase.scala @@ -169,7 +169,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"SHOW TABLES IN $catalog.nonexist") }, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`nonexist`")) + parameters = Map("schemaName" -> s"`$catalog`.`nonexist`")) } test("show table extended in a not existing namespace") { @@ -178,7 +178,7 @@ trait ShowTablesSuiteBase extends QueryTest with DDLCommandTestUtils { sql(s"SHOW TABLE EXTENDED IN $catalog.nonexist LIKE '*tbl*'") }, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`nonexist`")) + parameters = Map("schemaName" -> s"`$catalog`.`nonexist`")) } test("show table extended with no matching table") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala index 647247cc833dd..cec72b8855291 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/DropNamespaceSuite.scala @@ -40,7 +40,7 @@ trait DropNamespaceSuiteBase extends command.DropNamespaceSuiteBase sql(s"DROP NAMESPACE default") }, errorClass = "UNSUPPORTED_FEATURE.DROP_DATABASE", - parameters = Map("database" -> "`default`") + parameters = Map("database" -> s"`$catalog`.`default`") ) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala index ee5ac09e00892..85a46cfb93233 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/ShowNamespacesSuite.scala @@ -50,7 +50,7 @@ trait ShowNamespacesSuiteBase extends command.ShowNamespacesSuiteBase { } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`dummy`")) + parameters = Map("schemaName" -> s"`$catalog`.`dummy`")) } } 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 4de74af250006..50988e133005a 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 @@ -28,7 +28,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkUnsupportedOperationException import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchDatabaseException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Column, Identifier, NamespaceChange, SupportsNamespaces, TableCatalog, TableChange, V1Table} @@ -1039,7 +1039,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { assert(catalog.namespaceExists(testNs) === false) - val exc = intercept[NoSuchDatabaseException] { + val exc = intercept[NoSuchNamespaceException] { catalog.createTable(testIdent, columns, emptyTrans, emptyProps) } @@ -1156,7 +1156,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite { assert(catalog.namespaceExists(testNs) === false) - val exc = intercept[NoSuchDatabaseException] { + val exc = intercept[NoSuchNamespaceException] { catalog.alterNamespace(testNs, NamespaceChange.setProperty("property", "value")) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala index 3d5e2851fa7ba..55be6102a8535 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala @@ -273,7 +273,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`d:b`")) + parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } { @@ -282,7 +282,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`d:b`")) + parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } withTempDir { dir => @@ -314,7 +314,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle } checkError(e, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`d:b`")) + parameters = Map("schemaName" -> "`spark_catalog`.`d:b`")) } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index fd437e7dc954f..b959459eb3df8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -1359,7 +1359,7 @@ class HiveDDLSuite sql("DROP DATABASE default") }, errorClass = "UNSUPPORTED_FEATURE.DROP_DATABASE", - parameters = Map("database" -> "`default`") + parameters = Map("database" -> "`spark_catalog`.`default`") ) // SQLConf.CASE_SENSITIVE does not affect the result @@ -1373,7 +1373,7 @@ class HiveDDLSuite case _ => "_LEGACY_ERROR_TEMP_3065" }, parameters = caseSensitive match { - case "false" => Map("database" -> "`default`") + case "false" => Map("database" -> "`spark_catalog`.`default`") case _ => Map( "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", "msg" -> "MetaException(message:Can not drop default database)") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4d23ac0639b3e..5ccb7f0d1f84a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -1394,7 +1394,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sql("USE not_existing_db") }, errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`not_existing_db`")) + parameters = Map("schemaName" -> "`spark_catalog`.`not_existing_db`")) sql(s"USE $currentDatabase") assert(currentDatabase == sql("select current_database()").first().getString(0)) From 31d5ea1c3bc402596e604c335a77476d83f3edee Mon Sep 17 00:00:00 2001 From: Ivan Sadikov Date: Thu, 11 Jul 2024 15:02:57 +0900 Subject: [PATCH 29/64] [SPARK-48863][SQL] Fix ClassCastException when parsing JSON with "spark.sql.json.enablePartialResults" enabled ### What changes were proposed in this pull request? This PR fixes a bug in a corner case of JSON parsing when `spark.sql.json.enablePartialResults` is enabled. When running the following query with the config set to true: ``` select from_json('{"a":"b","c":"d"}', 'array>') ``` the code would fail with ``` org.apache.spark.SparkException: Job aborted due to stage failure: Task 0 in stage 4.0 failed 1 times, most recent failure: Lost task 0.0 in stage 4.0 (TID 4) (ip-10-110-51-101.us-west-2.compute.internal executor driver): java.lang.ClassCastException: class org.apache.spark.unsafe.types.UTF8String cannot be cast to class org.apache.spark.sql.catalyst.util.ArrayData (org.apache.spark.unsafe.types.UTF8String and org.apache.spark.sql.catalyst.util.ArrayData are in unnamed module of loader 'app') at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getArray(rows.scala:53) at org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow.getArray$(rows.scala:53) at org.apache.spark.sql.catalyst.expressions.GenericInternalRow.getArray(rows.scala:172) at org.apache.spark.sql.catalyst.expressions.JsonToStructs.$anonfun$converter$2(jsonExpressions.scala:831) at org.apache.spark.sql.catalyst.expressions.JsonToStructs.nullSafeEval(jsonExpressions.scala:893) ``` The patch fixes the issue by re-throwing PartialArrayDataResultException if parsing fails in this special case. ### Why are the changes needed? Fixes the bug that would prevent users from reading objects as arrays as introduced in SPARK-19595. This is more of a special case but it works with the flag off so it would be good to fix it when the flag is on. ### Does this PR introduce _any_ user-facing change? Yes, but it is a bug fix so it would not have worked without this patch overall. The parsing output will be different due to the partial results improvement: Previously, we would get `null` (the partial results are disabled). With this patch and partial results enabled, this will return `Array([b, null])`. This is not specific to this patch but rather to the partial results feature in general. ### How was this patch tested? I added a unit test. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47292 from sadikovi/SPARK-48863. Authored-by: Ivan Sadikov Signed-off-by: Hyukjin Kwon --- .../sql/catalyst/json/JacksonParser.scala | 13 +++++++- .../apache/spark/sql/JsonFunctionsSuite.scala | 32 +++++++++++++++++++ 2 files changed, 44 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala index b2c302fbbbe31..32a1731a93d4e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JacksonParser.scala @@ -201,7 +201,18 @@ class JacksonParser( // val st = at.elementType.asInstanceOf[StructType] val fieldConverters = st.map(_.dataType).map(makeConverter).toArray - Some(InternalRow(new GenericArrayData(convertObject(parser, st, fieldConverters).toArray))) + + val res = try { + convertObject(parser, st, fieldConverters) + } catch { + case err: PartialResultException => + throw PartialArrayDataResultException( + new GenericArrayData(Seq(err.partialResult)), + err.cause + ) + } + + Some(InternalRow(new GenericArrayData(res.toArray))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 9dca1d091d33b..7dbad885169d1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -1161,6 +1161,38 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession { } } + test("SPARK-48863: parse object as an array with partial results enabled") { + val schema = StructType(StructField("a", StringType) :: StructField("c", IntegerType) :: Nil) + + // Value can be parsed correctly and should return the same result with or without the flag. + Seq(false, true).foreach { enabled => + withSQLConf(SQLConf.JSON_ENABLE_PARTIAL_RESULTS.key -> s"${enabled}") { + checkAnswer( + Seq("""{"a": "b", "c": 1}""").toDF("c0") + .select(from_json($"c0", ArrayType(schema))), + Row(Seq(Row("b", 1))) + ) + } + } + + // Value does not match the schema. + val df = Seq("""{"a": "b", "c": "1"}""").toDF("c0") + + withSQLConf(SQLConf.JSON_ENABLE_PARTIAL_RESULTS.key -> "true") { + checkAnswer( + df.select(from_json($"c0", ArrayType(schema))), + Row(Seq(Row("b", null))) + ) + } + + withSQLConf(SQLConf.JSON_ENABLE_PARTIAL_RESULTS.key -> "false") { + checkAnswer( + df.select(from_json($"c0", ArrayType(schema))), + Row(null) + ) + } + } + test("SPARK-33270: infers schema for JSON field with spaces and pass them to from_json") { val in = Seq("""{"a b": 1}""").toDS() val out = in.select(from_json($"value", schema_of_json("""{"a b": 100}""")) as "parsed") From 261dbf4a9047bc00271137b547341e02351106ed Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Thu, 11 Jul 2024 18:59:10 +0800 Subject: [PATCH 30/64] [SPARK-48866][SQL] Fix hints of valid charset in the error message of INVALID_PARAMETER_VALUE.CHARSET ### What changes were proposed in this pull request? This PR fixes hints at the error message of INVALID_PARAMETER_VALUE.CHARSET. The current error message does not enumerate all valid charsets, e.g. UTF-32. This PR parameterizes it to fix this issue. ### Why are the changes needed? Bugfix, the hint w/ charsets missing is not helpful ### Does this PR introduce _any_ user-facing change? Yes, error message changing ### How was this patch tested? modified tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47295 from yaooqinn/SPARK-48866. Authored-by: Kent Yao Signed-off-by: yangjie01 --- .../utils/src/main/resources/error/error-conditions.json | 2 +- .../apache/spark/sql/errors/QueryExecutionErrors.scala | 5 +++-- .../sql-tests/results/ansi/string-functions.sql.out | 8 ++++++++ .../resources/sql-tests/results/string-functions.sql.out | 8 ++++++++ .../spark/sql/execution/datasources/csv/CSVSuite.scala | 8 +++++--- 5 files changed, 25 insertions(+), 6 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 02d1e63e380aa..7f54a77c94a0f 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -2584,7 +2584,7 @@ }, "CHARSET" : { "message" : [ - "expects one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', but got ." + "expects one of the , but got ." ] }, "DATETIME_UNIT" : { 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 d524742e126e0..bdd53219de404 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 @@ -42,7 +42,7 @@ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.ValueInterval import org.apache.spark.sql.catalyst.trees.{Origin, TreeNode} -import org.apache.spark.sql.catalyst.util.{sideBySide, DateTimeUtils, FailFastMode, MapData} +import org.apache.spark.sql.catalyst.util.{sideBySide, CharsetProvider, DateTimeUtils, FailFastMode, MapData} import org.apache.spark.sql.connector.catalog.{CatalogNotFoundException, Table, TableProvider} import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ import org.apache.spark.sql.connector.expressions.Transform @@ -2742,7 +2742,8 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE messageParameters = Map( "functionName" -> toSQLId(functionName), "parameter" -> toSQLId("charset"), - "charset" -> charset)) + "charset" -> charset, + "charsets" -> CharsetProvider.VALID_CHARSETS.mkString(", "))) } def malformedCharacterCoding(functionName: String, charset: String): RuntimeException = { diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out index da2fa9ca0c18b..d4adec22c50f1 100644 --- a/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/ansi/string-functions.sql.out @@ -846,6 +846,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -863,6 +864,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -880,6 +882,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -897,6 +900,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -1140,6 +1144,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1157,6 +1162,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1206,6 +1212,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1223,6 +1230,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } diff --git a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out index d42c387c8057f..08a7e317c116f 100644 --- a/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/string-functions.sql.out @@ -778,6 +778,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -795,6 +796,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -812,6 +814,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -829,6 +832,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`encode`", "parameter" : "`charset`" } @@ -1072,6 +1076,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1089,6 +1094,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "Windows-xxx", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1138,6 +1144,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } @@ -1155,6 +1162,7 @@ org.apache.spark.SparkIllegalArgumentException "sqlState" : "22023", "messageParameters" : { "charset" : "WINDOWS-1252", + "charsets" : "UTF-16LE, UTF-8, UTF-32, UTF-16BE, UTF-16, US-ASCII, ISO-8859-1", "functionName" : "`decode`", "parameter" : "`charset`" } 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 21c59c4771b86..93828ee649bf0 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 @@ -39,7 +39,7 @@ import org.apache.spark._ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.catalyst.expressions.ToStringBase -import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, HadoopCompressionCodec} +import org.apache.spark.sql.catalyst.util.{CharsetProvider, DateTimeTestUtils, DateTimeUtils, HadoopCompressionCodec} import org.apache.spark.sql.errors.QueryExecutionErrors.toSQLId import org.apache.spark.sql.execution.datasources.CommonFileDataSourceSuite import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} @@ -270,7 +270,8 @@ abstract class CSVSuite parameters = Map( "charset" -> "1-9588-osi", "functionName" -> toSQLId("CSVOptions"), - "parameter" -> toSQLId("charset")) + "parameter" -> toSQLId("charset"), + "charsets" -> CharsetProvider.VALID_CHARSETS.mkString(", ")) ) } @@ -653,7 +654,8 @@ abstract class CSVSuite parameters = Map( "charset" -> "1-9588-osi", "functionName" -> toSQLId("CSVOptions"), - "parameter" -> toSQLId("charset")) + "parameter" -> toSQLId("charset"), + "charsets" -> CharsetProvider.VALID_CHARSETS.mkString(", ")) ) } From 896c15e2a30c3590a7e6d7deb7326aacde85a71e Mon Sep 17 00:00:00 2001 From: Rui Wang Date: Thu, 11 Jul 2024 19:06:07 +0800 Subject: [PATCH 31/64] [SPARK-48773] Document config "spark.default.parallelism" by config builder framework ### What changes were proposed in this pull request? Document config "spark.default.parallelism". This is Spark used config but not documented by config builder framework. This config is already in spark website: https://spark.apache.org/docs/latest/configuration.html. ### Why are the changes needed? Document Spark's config. ### Does this PR introduce _any_ user-facing change? NO. ### How was this patch tested? N/A ### Was this patch authored or co-authored using generative AI tooling? N/A Closes #47171 from amaliujia/document_spark_default_paramllel. Authored-by: Rui Wang Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/Partitioner.scala | 3 ++- .../spark/internal/config/package.scala | 12 ++++++++++++ .../CoarseGrainedSchedulerBackend.scala | 4 ++-- .../local/LocalSchedulerBackend.scala | 2 +- .../scala/org/apache/spark/FileSuite.scala | 2 +- .../org/apache/spark/PartitioningSuite.scala | 7 ++++--- .../SparkContextSchedulerCreationSuite.scala | 3 ++- .../spark/rdd/PairRDDFunctionsSuite.scala | 19 ++++++++++--------- .../CoarseGrainedSchedulerBackendSuite.scala | 2 +- .../scheduler/SchedulerIntegrationSuite.scala | 4 ++-- .../org/apache/spark/graphx/GraphSuite.scala | 3 ++- .../datasources/FileSourceStrategySuite.scala | 3 ++- 12 files changed, 41 insertions(+), 23 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 357e71cdf4457..7d086f34f6983 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -26,6 +26,7 @@ import scala.math.log10 import scala.reflect.ClassTag import scala.util.hashing.byteswap32 +import org.apache.spark.internal.config import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{CollectionsUtils, Utils} @@ -73,7 +74,7 @@ object Partitioner { None } - val defaultNumPartitions = if (rdd.context.conf.contains("spark.default.parallelism")) { + val defaultNumPartitions = if (rdd.context.conf.contains(config.DEFAULT_PARALLELISM.key)) { rdd.context.defaultParallelism } else { rdds.map(_.partitions.length).max diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 9fcd9ba529c16..fc25fe59b8942 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -42,6 +42,18 @@ package object config { private[spark] val SPARK_TASK_PREFIX = "spark.task" private[spark] val LISTENER_BUS_EVENT_QUEUE_PREFIX = "spark.scheduler.listenerbus.eventqueue" + private[spark] val DEFAULT_PARALLELISM = + ConfigBuilder("spark.default.parallelism") + .doc("Default number of partitions in RDDs returned by transformations like " + + "join, reduceByKey, and parallelize when not set by user. " + + "For distributed shuffle operations like reduceByKey and join, the largest number of " + + "partitions in a parent RDD. For operations like parallelize with no parent RDDs, " + + "it depends on the cluster manager. For example in Local mode, it defaults to the " + + "number of cores on the local machine") + .version("0.5.0") + .intConf + .createOptional + private[spark] val RESOURCES_DISCOVERY_PLUGIN = ConfigBuilder("spark.resources.discoveryPlugin") .doc("Comma-separated list of class names implementing" + diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index deaa1b4e47906..4b3a16b4d3f60 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -32,7 +32,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.ExecutorLogUrlHandler -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{config, Logging, MDC} import org.apache.spark.internal.LogKeys import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config._ @@ -707,7 +707,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } override def defaultParallelism(): Int = { - conf.getInt("spark.default.parallelism", math.max(totalCoreCount.get(), 2)) + conf.getInt(config.DEFAULT_PARALLELISM.key, math.max(totalCoreCount.get(), 2)) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 298669327a39c..ddc9287946085 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -154,7 +154,7 @@ private[spark] class LocalSchedulerBackend( } override def defaultParallelism(): Int = - scheduler.conf.getInt("spark.default.parallelism", totalCores) + scheduler.conf.getInt(config.DEFAULT_PARALLELISM.key, totalCores) override def killTask( taskId: Long, executorId: String, interruptThread: Boolean, reason: String): Unit = { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 7750db6020887..5651dc9b2dbdc 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -327,7 +327,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { test("SPARK-22357 test binaryFiles minPartitions") { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local") .set("spark.files.openCostInBytes", "0") - .set("spark.default.parallelism", "1")) + .set(DEFAULT_PARALLELISM.key, "1")) withTempDir { tempDir => val tempDirPath = tempDir.getAbsolutePath diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 3447ba8c1765e..d0423e267baf3 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -22,6 +22,7 @@ import scala.math.abs import org.scalatest.PrivateMethodTester +import org.apache.spark.internal.config._ import org.apache.spark.rdd.RDD import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.StatCounter @@ -286,9 +287,9 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva } test("defaultPartitioner when defaultParallelism is set") { - assert(!sc.conf.contains("spark.default.parallelism")) + assert(!sc.conf.contains(DEFAULT_PARALLELISM.key)) try { - sc.conf.set("spark.default.parallelism", "4") + sc.conf.set(DEFAULT_PARALLELISM.key, "4") val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 150) val rdd2 = sc.parallelize(Seq((1, 2), (2, 3), (2, 4), (3, 4))) @@ -317,7 +318,7 @@ class PartitioningSuite extends SparkFunSuite with SharedSparkContext with Priva assert(partitioner6.numPartitions == sc.defaultParallelism) assert(partitioner7.numPartitions == sc.defaultParallelism) } finally { - sc.conf.remove("spark.default.parallelism") + sc.conf.remove(DEFAULT_PARALLELISM.key) } } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 5dfd6ea23cbbf..f224f06c7d07a 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.scalatest.PrivateMethodTester +import org.apache.spark.internal.config import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend @@ -123,7 +124,7 @@ class SparkContextSchedulerCreationSuite } test("local-default-parallelism") { - val conf = new SparkConf().set("spark.default.parallelism", "16") + val conf = new SparkConf().set(config.DEFAULT_PARALLELISM.key, "16") val sched = createTaskScheduler("local", conf) { sched => sched.backend match { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index c2ada90e7f626..cf7cb29775dc8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -34,6 +34,7 @@ import org.scalatest.Assertions import org.apache.spark._ import org.apache.spark.Partitioner +import org.apache.spark.internal.config._ import org.apache.spark.util.ArrayImplicits._ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { @@ -332,44 +333,44 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("cogroup between multiple RDD when defaultParallelism is set without proper partitioner") { - assert(!sc.conf.contains("spark.default.parallelism")) + assert(!sc.conf.contains(DEFAULT_PARALLELISM.key)) try { - sc.conf.set("spark.default.parallelism", "4") + sc.conf.set(DEFAULT_PARALLELISM.key, "4") val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1)), 10) val joined = rdd1.cogroup(rdd2) assert(joined.getNumPartitions == sc.defaultParallelism) } finally { - sc.conf.remove("spark.default.parallelism") + sc.conf.remove(DEFAULT_PARALLELISM.key) } } test("cogroup between multiple RDD when defaultParallelism is set with proper partitioner") { - assert(!sc.conf.contains("spark.default.parallelism")) + assert(!sc.conf.contains(DEFAULT_PARALLELISM.key)) try { - sc.conf.set("spark.default.parallelism", "4") + sc.conf.set(DEFAULT_PARALLELISM.key, "4") val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 20) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1))) .partitionBy(new HashPartitioner(10)) val joined = rdd1.cogroup(rdd2) assert(joined.getNumPartitions == rdd2.getNumPartitions) } finally { - sc.conf.remove("spark.default.parallelism") + sc.conf.remove(DEFAULT_PARALLELISM.key) } } test("cogroup between multiple RDD when defaultParallelism is set; with huge number of " + "partitions in upstream RDDs") { - assert(!sc.conf.contains("spark.default.parallelism")) + assert(!sc.conf.contains(DEFAULT_PARALLELISM.key)) try { - sc.conf.set("spark.default.parallelism", "4") + sc.conf.set(DEFAULT_PARALLELISM.key, "4") val rdd1 = sc.parallelize((1 to 1000).map(x => (x, x)), 1000) val rdd2 = sc.parallelize(Seq((1, 1), (1, 2), (2, 1), (3, 1))) .partitionBy(new HashPartitioner(10)) val joined = rdd1.cogroup(rdd2) assert(joined.getNumPartitions == rdd2.getNumPartitions) } finally { - sc.conf.remove("spark.default.parallelism") + sc.conf.remove(DEFAULT_PARALLELISM.key) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index a75f470deec33..127e4b2e413ce 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -54,7 +54,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo test("serialized task larger than max RPC message size") { val conf = new SparkConf conf.set(RPC_MESSAGE_MAX_SIZE, 1) - conf.set("spark.default.parallelism", "1") + conf.set(DEFAULT_PARALLELISM.key, "1") sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) val buffer = new SerializableBuffer(java.nio.ByteBuffer.allocate(2 * frameSize)) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index f9d0a035d9ae8..2cde2cc76ba35 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.TaskState._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config.SCHEDULER_REVIVE_INTERVAL import org.apache.spark.rdd.RDD import org.apache.spark.resource.ResourceProfile @@ -445,7 +445,7 @@ private[spark] class SingleCoreMockBackend( val cores = 1 - override def defaultParallelism(): Int = conf.getInt("spark.default.parallelism", cores) + override def defaultParallelism(): Int = conf.getInt(config.DEFAULT_PARALLELISM.key, cores) freeCores = cores val localExecutorId = SparkContext.DRIVER_IDENTIFIER diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 474fa908baa4f..35c256c2e7912 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ +import org.apache.spark.internal.config import org.apache.spark.rdd._ import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -373,7 +374,7 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { val numEdgePartitions = 4 assert(defaultParallelism != numEdgePartitions) val conf = new org.apache.spark.SparkConf() - .set("spark.default.parallelism", defaultParallelism.toString) + .set(config.DEFAULT_PARALLELISM.key, defaultParallelism.toString) val sc = new SparkContext("local", "test", conf) try { val edges = sc.parallelize((1 to n).map(x => (x: VertexId, 0: VertexId)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index 7debc9b32df05..91b2e18c6b209 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -26,6 +26,7 @@ import org.apache.hadoop.fs.{BlockLocation, FileStatus, Path, RawLocalFileSystem import org.apache.hadoop.mapreduce.Job import org.apache.spark.SparkException +import org.apache.spark.internal.config import org.apache.spark.paths.SparkPath.{fromUrlString => sp} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -44,7 +45,7 @@ import org.apache.spark.util.Utils class FileSourceStrategySuite extends QueryTest with SharedSparkSession { import testImplicits._ - protected override def sparkConf = super.sparkConf.set("spark.default.parallelism", "1") + protected override def sparkConf = super.sparkConf.set(config.DEFAULT_PARALLELISM.key, "1") test("unpartitioned table, single partition") { val table = From dbc420f02e817d03b0605ccd5a6195d50a5af07e Mon Sep 17 00:00:00 2001 From: panbingkun Date: Thu, 11 Jul 2024 20:52:06 +0800 Subject: [PATCH 32/64] [SPARK-48793][SQL][TESTS][S] Unify v1 and v2 `ALTER TABLE .. `DROP|RENAME` COLUMN ...` tests ### What changes were proposed in this pull request? The pr aims to: - Move parser tests from `o.a.s.s.c.p.DDLParserSuite` and `o.a.s.s.c.p.ErrorParserSuite` to `AlterTableRenameColumnParserSuite` & `AlterTableDropColumnParserSuite` - Add a test for DSv2 ALTER TABLE .. `DROP|RENAME` to `v2.AlterTableDropColumnSuite` & `v2.AlterTableRenameColumnSuite` (This PR includes the unification of two commands: `DROP COLUMN` & `RENAME COLUMN`) ### Why are the changes needed? - To improve test coverage. - Align with other similar tests, eg: AlterTableRename* ### Does this PR introduce _any_ user-facing change? No, only tests. ### How was this patch tested? - Add new UT - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47199 from panbingkun/alter_table_drop_column. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../sql/catalyst/parser/DDLParserSuite.scala | 51 ----- .../catalyst/parser/ErrorParserSuite.scala | 18 -- .../AlterTableDropColumnParserSuite.scala | 68 +++++++ .../AlterTableDropColumnSuiteBase.scala | 38 ++++ .../AlterTableRenameColumnParserSuite.scala | 56 ++++++ .../AlterTableRenameColumnSuiteBase.scala | 39 ++++ .../v1/AlterTableDropColumnSuite.scala | 57 ++++++ .../v1/AlterTableRenameColumnSuite.scala | 56 ++++++ .../v2/AlterTableDropColumnSuite.scala | 184 +++++++++++++++++ .../v2/AlterTableRenameColumnSuite.scala | 190 ++++++++++++++++++ .../command/AlterTableDropColumnSuite.scala | 26 +++ .../command/AlterTableRenameColumnSuite.scala | 26 +++ 12 files changed, 740 insertions(+), 69 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnParserSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnSuiteBase.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropColumnSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenameColumnSuite.scala 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 f88c516f0019d..d570716c37673 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 @@ -1204,15 +1204,6 @@ class DDLParserSuite extends AnalysisTest { ))) } - test("alter table: rename column") { - comparePlans( - parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"), - RenameColumn( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME COLUMN"), - UnresolvedFieldName(Seq("a", "b", "c")), - "d")) - } - test("alter table: update column type using ALTER") { comparePlans( parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint"), @@ -1322,48 +1313,6 @@ class DDLParserSuite extends AnalysisTest { None)) } - test("alter table: drop column") { - comparePlans( - parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"), - DropColumns( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), - Seq(UnresolvedFieldName(Seq("a", "b", "c"))), - ifExists = false)) - - comparePlans( - parsePlan("ALTER TABLE table_name DROP COLUMN IF EXISTS a.b.c"), - DropColumns( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), - Seq(UnresolvedFieldName(Seq("a", "b", "c"))), - ifExists = true)) - } - - test("alter table: drop multiple columns") { - val sql = "ALTER TABLE table_name DROP COLUMN x, y, a.b.c" - Seq(sql, sql.replace("COLUMN", "COLUMNS")).foreach { drop => - comparePlans( - parsePlan(drop), - DropColumns( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), - Seq(UnresolvedFieldName(Seq("x")), - UnresolvedFieldName(Seq("y")), - UnresolvedFieldName(Seq("a", "b", "c"))), - ifExists = false)) - } - - val sqlIfExists = "ALTER TABLE table_name DROP COLUMN IF EXISTS x, y, a.b.c" - Seq(sqlIfExists, sqlIfExists.replace("COLUMN", "COLUMNS")).foreach { drop => - comparePlans( - parsePlan(drop), - DropColumns( - UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), - Seq(UnresolvedFieldName(Seq("x")), - UnresolvedFieldName(Seq("y")), - UnresolvedFieldName(Seq("a", "b", "c"))), - ifExists = true)) - } - } - test("alter table: hive style change column") { val sql1 = "ALTER TABLE table_name CHANGE COLUMN a.b.c c INT" val sql2 = "ALTER TABLE table_name CHANGE COLUMN a.b.c c INT COMMENT 'new_comment'" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index 0130ae72a03c4..cd1556a2e7916 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -89,24 +89,6 @@ class ErrorParserSuite extends AnalysisTest { """.stripMargin), errorClass = "INVALID_IDENTIFIER", parameters = Map("ident" -> "test-col")) - checkError( - exception = parseException( - """ - |ALTER TABLE t - |RENAME COLUMN - |test-col TO test - """.stripMargin), - errorClass = "INVALID_IDENTIFIER", - parameters = Map("ident" -> "test-col")) - checkError( - exception = parseException( - """ - |ALTER TABLE t - |RENAME COLUMN - |test TO test-col - """.stripMargin), - errorClass = "INVALID_IDENTIFIER", - parameters = Map("ident" -> "test-col")) checkError( exception = parseException( """ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnParserSuite.scala new file mode 100644 index 0000000000000..09095a2f6814d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnParserSuite.scala @@ -0,0 +1,68 @@ +/* + * 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.command + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedFieldName, UnresolvedTable} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.plans.logical.DropColumns +import org.apache.spark.sql.test.SharedSparkSession + +class AlterTableDropColumnParserSuite extends AnalysisTest with SharedSparkSession { + + test("alter table: drop column") { + comparePlans( + parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"), + DropColumns( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), + Seq(UnresolvedFieldName(Seq("a", "b", "c"))), + ifExists = false)) + + comparePlans( + parsePlan("ALTER TABLE table_name DROP COLUMN IF EXISTS a.b.c"), + DropColumns( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), + Seq(UnresolvedFieldName(Seq("a", "b", "c"))), + ifExists = true)) + } + + test("alter table: drop multiple columns") { + val sql = "ALTER TABLE table_name DROP COLUMN x, y, a.b.c" + Seq(sql, sql.replace("COLUMN", "COLUMNS")).foreach { drop => + comparePlans( + parsePlan(drop), + DropColumns( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), + Seq(UnresolvedFieldName(Seq("x")), + UnresolvedFieldName(Seq("y")), + UnresolvedFieldName(Seq("a", "b", "c"))), + ifExists = false)) + } + + val sqlIfExists = "ALTER TABLE table_name DROP COLUMN IF EXISTS x, y, a.b.c" + Seq(sqlIfExists, sqlIfExists.replace("COLUMN", "COLUMNS")).foreach { drop => + comparePlans( + parsePlan(drop), + DropColumns( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... DROP COLUMNS"), + Seq(UnresolvedFieldName(Seq("x")), + UnresolvedFieldName(Seq("y")), + UnresolvedFieldName(Seq("a", "b", "c"))), + ifExists = true)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnSuiteBase.scala new file mode 100644 index 0000000000000..382049e3ec3cb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableDropColumnSuiteBase.scala @@ -0,0 +1,38 @@ +/* + * 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.command + +import org.apache.spark.sql.QueryTest + +/** + * This base suite contains unified tests for the `ALTER TABLE .. DROP (COLUMN | COLUMNS)` command + * that check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are + * located in more specific test suites: + * + * - V2 table catalog tests: + * `org.apache.spark.sql.execution.command.v2.AlterTableDropColumnSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterTableDropColumnSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterTableDropColumnSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableDropColumnSuite` + */ +trait AlterTableDropColumnSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER TABLE .. DROP COLUMN" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala new file mode 100644 index 0000000000000..62ee8aa57a760 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnParserSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.command + +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedFieldName, UnresolvedTable} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan +import org.apache.spark.sql.catalyst.plans.logical.RenameColumn +import org.apache.spark.sql.test.SharedSparkSession + +class AlterTableRenameColumnParserSuite extends AnalysisTest with SharedSparkSession { + + test("alter table: rename column") { + comparePlans( + parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"), + RenameColumn( + UnresolvedTable(Seq("table_name"), "ALTER TABLE ... RENAME COLUMN"), + UnresolvedFieldName(Seq("a", "b", "c")), + "d")) + } + + test("alter table: rename column - hyphen in identifier") { + checkError( + exception = parseException(parsePlan)( + "ALTER TABLE t RENAME COLUMN test-col TO test"), + errorClass = "INVALID_IDENTIFIER", + parameters = Map("ident" -> "test-col")) + checkError( + exception = parseException(parsePlan)( + "ALTER TABLE t RENAME COLUMN test TO test-col"), + errorClass = "INVALID_IDENTIFIER", + parameters = Map("ident" -> "test-col")) + } + + test("alter table: rename nested column") { + checkError( + exception = parseException(parsePlan)( + "ALTER TABLE t RENAME COLUMN point.x to point.y"), + errorClass = "PARSE_SYNTAX_ERROR", + parameters = Map("error" -> "'.'", "hint" -> "")) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnSuiteBase.scala new file mode 100644 index 0000000000000..25d499a427374 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRenameColumnSuiteBase.scala @@ -0,0 +1,39 @@ +/* + * 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.command + +import org.apache.spark.sql.QueryTest + +/** + * This base suite contains unified tests for the `ALTER TABLE ... RENAME COLUMN` command that + * check V1 and V2 table catalogs. The tests that cannot run for all supported catalogs are located + * in more specific test suites: + * + * - V2 table catalog tests: + * `org.apache.spark.sql.execution.command.v2.AlterTableRenameColumnSuite` + * - V1 table catalog tests: + * `org.apache.spark.sql.execution.command.v1.AlterTableRenameColumnSuiteBase` + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterTableRenameColumnSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableRenameColumnSuite` + */ +trait AlterTableRenameColumnSuiteBase extends QueryTest with DDLCommandTestUtils { + override val command = "ALTER TABLE .. RENAME COLUMN" +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala new file mode 100644 index 0000000000000..6370a834746a5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableDropColumnSuite.scala @@ -0,0 +1,57 @@ +/* + * 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.command.v1 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.execution.command + +/** + * This base suite contains unified tests for the `ALTER TABLE .. DROP (COLUMN | COLUMNS)` command + * that check V1 table catalogs. The tests that cannot run for all V1 catalogs are located in more + * specific test suites: + * + * - V1 In-Memory catalog: + * `org.apache.spark.sql.execution.command.v1.AlterTableDropColumnSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableDropColumnSuite` + */ +trait AlterTableDropColumnSuiteBase extends command.AlterTableDropColumnSuiteBase { + + test("not support drop column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + checkError( + exception = intercept[AnalysisException]( + sql(s"ALTER TABLE $t DROP COLUMN id") + ), + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> toSQLId(t), + "operation" -> "DROP COLUMN" + ) + ) + } + } +} + +/** + * The class contains tests for the `ALTER TABLE .. DROP (COLUMN | COLUMNS)` command to check + * V1 In-Memory table catalog. + */ +class AlterTableDropColumnSuite extends AlterTableDropColumnSuiteBase with CommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala new file mode 100644 index 0000000000000..86b34311bfb3d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v1/AlterTableRenameColumnSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.command.v1 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.execution.command + +/** + * This base suite contains unified tests for the `ALTER TABLE .. RENAME COLUMN` + * command that check V1 table catalogs. The tests that cannot run for all V1 catalogs + * are located in more specific test suites: + * + * - V1 In-Memory catalog: `org.apache.spark.sql.execution.command.v1.AlterTableRenameColumnSuite` + * - V1 Hive External catalog: + * `org.apache.spark.sql.hive.execution.command.AlterTableRenameColumnSuite` + */ +trait AlterTableRenameColumnSuiteBase extends command.AlterTableRenameColumnSuiteBase { + + test("not support rename column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") + checkError( + exception = intercept[AnalysisException]( + sql(s"ALTER TABLE $t RENAME COLUMN col1 TO col3") + ), + errorClass = "UNSUPPORTED_FEATURE.TABLE_OPERATION", + parameters = Map( + "tableName" -> toSQLId(t), + "operation" -> "RENAME COLUMN" + ) + ) + } + } +} + +/** + * The class contains tests for the `ALTER TABLE .. RENAME COLUMN` command to check + * V1 In-Memory table catalog. + */ +class AlterTableRenameColumnSuite extends AlterTableRenameColumnSuiteBase with CommandSuiteBase diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala new file mode 100644 index 0000000000000..3adcc42c6be5f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableDropColumnSuite.scala @@ -0,0 +1,184 @@ +/* + * 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.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.connector.catalog.{Column, Identifier, Table} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType} + +/** + * The class contains tests for the `ALTER TABLE .. DROP (COLUMN | COLUMNS)` command to + * check V2 table catalogs. + */ +class AlterTableDropColumnSuite + extends command.AlterTableDropColumnSuiteBase with CommandSuiteBase { + + private def getTableMetadata(tableIndent: TableIdentifier): Table = { + val nameParts = tableIndent.nameParts + val v2Catalog = spark.sessionState.catalogManager.catalog(nameParts.head).asTableCatalog + val namespace = nameParts.drop(1).init.toArray + v2Catalog.loadTable(Identifier.of(namespace, nameParts.last)) + } + + test("table does not exist") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + checkError( + exception = intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist DROP COLUMN id") + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`does_not_exist`"), + context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25) + ) + } + } + + test("drop column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, data string) $defaultUsing") + sql(s"ALTER TABLE $t DROP COLUMN data") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array(Column.create("id", IntegerType))) + } + } + + test("drop nested column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point struct) $defaultUsing") + sql(s"ALTER TABLE $t DROP COLUMN point.z") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", + StructType(Seq(StructField("x", DoubleType), StructField("y", DoubleType)))))) + } + } + + test("drop nested column in map key") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point map, bigint>) " + + s"$defaultUsing") + sql(s"ALTER TABLE $t DROP COLUMN point.key.y") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", MapType(StructType(Seq(StructField("x", DoubleType))), LongType)))) + } + } + + test("drop nested column in map value") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point map>) " + + s"$defaultUsing") + sql(s"ALTER TABLE $t DROP COLUMN point.value.y") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", MapType(StringType, StructType(Seq(StructField("x", DoubleType))))))) + } + } + + test("drop nested column in array element") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, points array>) $defaultUsing") + sql(s"ALTER TABLE $t DROP COLUMN points.element.y") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("points", ArrayType(StructType(Seq(StructField("x", DoubleType))))))) + } + } + + test("drop column must exist if required") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + + val sqlText = s"ALTER TABLE $t DROP COLUMN does_not_exist" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "_LEGACY_ERROR_TEMP_1331", + parameters = Map( + "fieldName" -> "does_not_exist", + "table" -> t, + "schema" -> + """root + | |-- id: integer (nullable = true) + |""".stripMargin), + context = ExpectedContext(fragment = sqlText, start = 0, stop = 57) + ) + } + } + + test("nested drop column must exist if required") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + + val sqlText = s"ALTER TABLE $t DROP COLUMN point.does_not_exist" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "_LEGACY_ERROR_TEMP_1331", + parameters = Map( + "fieldName" -> "point.does_not_exist", + "table" -> t, + "schema" -> + """root + | |-- id: integer (nullable = true) + |""".stripMargin), + context = ExpectedContext(fragment = sqlText, start = 0, stop = 63) + ) + + // with if exists it should pass + sql(s"ALTER TABLE $t DROP COLUMN IF EXISTS point.does_not_exist") + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array(Column.create("id", IntegerType))) + } + } + + test("drop mixed existing/non-existing columns using IF EXISTS") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, name string, points array>) " + + s"$defaultUsing") + // with if exists it should pass + sql(s"ALTER TABLE $t DROP COLUMNS IF EXISTS " + + s"names, name, points.element.z, id, points.element.x") + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array(Column.create("points", + ArrayType(StructType(Seq(StructField("y", DoubleType))))))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala new file mode 100644 index 0000000000000..a15d141822611 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/v2/AlterTableRenameColumnSuite.scala @@ -0,0 +1,190 @@ +/* + * 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.command.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.connector.catalog.{Column, Identifier, Table} +import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.CatalogHelper +import org.apache.spark.sql.execution.command +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType} + +/** + * The class contains tests for the `ALTER TABLE .. RENAME COLUMN` command + * to check V2 table catalogs. + */ +class AlterTableRenameColumnSuite + extends command.AlterTableRenameColumnSuiteBase with CommandSuiteBase { + + private def getTableMetadata(tableIndent: TableIdentifier): Table = { + val nameParts = tableIndent.nameParts + val v2Catalog = spark.sessionState.catalogManager.catalog(nameParts.head).asTableCatalog + val namespace = nameParts.drop(1).init.toArray + v2Catalog.loadTable(Identifier.of(namespace, nameParts.last)) + } + + test("table does not exist") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") + checkError( + exception = intercept[AnalysisException] { + sql("ALTER TABLE does_not_exist RENAME COLUMN col1 TO col3") + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`does_not_exist`"), + context = ExpectedContext(fragment = "does_not_exist", start = 12, stop = 25) + ) + } + } + + test("rename column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (col1 int, col2 string, a int, b int) $defaultUsing") + sql(s"ALTER TABLE $t RENAME COLUMN col1 to col3") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("col3", IntegerType), + Column.create("col2", StringType), + Column.create("a", IntegerType), + Column.create("b", IntegerType))) + } + } + + test("rename nested column") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point struct) $defaultUsing") + // don't write as: s"ALTER TABLE $t RENAME COLUMN point.z to point.w", + // otherwise, it will throw an exception + sql(s"ALTER TABLE $t RENAME COLUMN point.z to w") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", + StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("w", DoubleType)) + ) + )) + ) + } + } + + test("rename nested column in map key") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point map, bigint>) " + + s"$defaultUsing") + sql(s"ALTER TABLE $t RENAME COLUMN point.key.y to z") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", + MapType( + StructType(Seq(StructField("x", DoubleType), StructField("z", DoubleType))), + LongType + ) + )) + ) + } + } + + test("rename nested column in map value") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, point map>) " + + s"$defaultUsing") + sql(s"ALTER TABLE $t RENAME COLUMN point.value.y to z") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("point", + MapType( + StringType, + StructType(Seq(StructField("x", DoubleType), StructField("z", DoubleType))) + ) + )) + ) + } + } + + test("rename nested column in array element") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int, points array>) $defaultUsing") + sql(s"ALTER TABLE $t RENAME COLUMN points.element.y to z") + + val table = getTableMetadata(TableIdentifier("tbl", Some("ns"), Some(catalog))) + assert(table.name === t) + assert(table.columns() === Array( + Column.create("id", IntegerType), + Column.create("points", + ArrayType(StructType(Seq(StructField("x", DoubleType), StructField("z", DoubleType)))))) + ) + } + } + + test("rename column must exist if required") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + + val sqlText = s"ALTER TABLE $t RENAME COLUMN does_not_exist to x" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "_LEGACY_ERROR_TEMP_1331", + parameters = Map( + "fieldName" -> "does_not_exist", + "table" -> t, + "schema" -> + """root + | |-- id: integer (nullable = true) + |""".stripMargin), + context = ExpectedContext(fragment = sqlText, start = 0, stop = 64) + ) + } + } + + test("nested rename column must exist if required") { + withNamespaceAndTable("ns", "tbl") { t => + sql(s"CREATE TABLE $t (id int) $defaultUsing") + + val sqlText = s"ALTER TABLE $t RENAME COLUMN point.does_not_exist TO x" + checkError( + exception = intercept[AnalysisException] { + sql(sqlText) + }, + errorClass = "_LEGACY_ERROR_TEMP_1331", + parameters = Map( + "fieldName" -> "point.does_not_exist", + "table" -> t, + "schema" -> + """root + | |-- id: integer (nullable = true) + |""".stripMargin), + context = ExpectedContext(fragment = sqlText, start = 0, stop = 70) + ) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropColumnSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropColumnSuite.scala new file mode 100644 index 0000000000000..8a7dd5862925d --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableDropColumnSuite.scala @@ -0,0 +1,26 @@ +/* + * 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.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for the `ALTER TABLE ... DROP (COLUMN | COLUMNS)` command to check + * V1 Hive external table catalog. + */ +class AlterTableDropColumnSuite extends v1.AlterTableDropColumnSuiteBase with CommandSuiteBase diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenameColumnSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenameColumnSuite.scala new file mode 100644 index 0000000000000..f52142906b0e1 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/command/AlterTableRenameColumnSuite.scala @@ -0,0 +1,26 @@ +/* + * 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.hive.execution.command + +import org.apache.spark.sql.execution.command.v1 + +/** + * The class contains tests for the `ALTER TABLE ... RENAME COLUMN` command to check + * V1 Hive external table catalog. + */ +class AlterTableRenameColumnSuite extends v1.AlterTableRenameColumnSuiteBase with CommandSuiteBase From f8052ddd42086a624c43048c4b9c826bc625040c Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 11 Jul 2024 08:12:27 -0700 Subject: [PATCH 33/64] [SPARK-46738][PYTHON] Reenable a group of doctests ### What changes were proposed in this pull request? the `cast` issue has been resolved in https://github.com/apache/spark/pull/47249 , then we can reenable a group of doctests ### Why are the changes needed? test coverage ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #47302 from zhengruifeng/enable_more_doctest. Authored-by: Ruifeng Zheng Signed-off-by: Dongjoon Hyun --- python/pyspark/sql/functions/builtin.py | 30 ++++++++++--------------- 1 file changed, 12 insertions(+), 18 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index c24b9e4378a61..3184ce78ac7e2 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -20237,8 +20237,6 @@ def nvl2(col1: "ColumnOrName", col2: "ColumnOrName", col3: "ColumnOrName") -> Co return _invoke_function_over_columns("nvl2", col1, col2, col3) -# TODO(SPARK-46738) Re-enable testing that includes the 'Cast' operation after -# fixing the display difference between Regular Spark and Spark Connect on `Cast`. @_try_remote_functions def aes_encrypt( input: "ColumnOrName", @@ -20331,7 +20329,7 @@ def aes_encrypt( ... ) >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, df.mode, df.padding), ... df.key, df.mode, df.padding - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------------------------------+ |CAST(aes_decrypt(aes_encrypt(input, key, mode, padding, , ), key, mode, padding, ) AS STRING)| +---------------------------------------------------------------------------------------------+ @@ -20347,7 +20345,7 @@ def aes_encrypt( ... ) >>> df.select(sf.aes_decrypt(sf.aes_encrypt(df.input, df.key, df.mode), ... df.key, df.mode - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------------------------------+ |CAST(aes_decrypt(aes_encrypt(input, key, mode, DEFAULT, , ), key, mode, DEFAULT, ) AS STRING)| +---------------------------------------------------------------------------------------------+ @@ -20363,7 +20361,7 @@ def aes_encrypt( ... ) >>> df.select(sf.aes_decrypt( ... sf.unbase64(sf.base64(sf.aes_encrypt(df.input, df.key))), df.key - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +-------------------------------------------------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(base64(aes_encrypt(input, key, GCM, DEFAULT, , ))), key, GCM, DEFAULT, ) AS STRING)| +-------------------------------------------------------------------------------------------------------------+ @@ -20377,8 +20375,6 @@ def aes_encrypt( return _invoke_function_over_columns("aes_encrypt", input, key, _mode, _padding, _iv, _aad) -# TODO(SPARK-46738) Re-enable testing that includes the 'Cast' operation after -# fixing the display difference between Regular Spark and Spark Connect on `Cast`. @_try_remote_functions def aes_decrypt( input: "ColumnOrName", @@ -20432,7 +20428,7 @@ def aes_decrypt( ... ) >>> df.select(sf.aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +---------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| +---------------------------------------------------------------------+ @@ -20449,7 +20445,7 @@ def aes_decrypt( ... ) >>> df.select(sf.aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode, df.padding - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, padding, ) AS STRING)| +------------------------------------------------------------------+ @@ -20466,7 +20462,7 @@ def aes_decrypt( ... ) >>> df.select(sf.aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +------------------------------------------------------------------+ |CAST(aes_decrypt(unbase64(input), key, mode, DEFAULT, ) AS STRING)| +------------------------------------------------------------------+ @@ -20483,7 +20479,7 @@ def aes_decrypt( ... ) >>> df.select(sf.aes_decrypt( ... sf.unhex(df.input), df.key - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +--------------------------------------------------------------+ |CAST(aes_decrypt(unhex(input), key, GCM, DEFAULT, ) AS STRING)| +--------------------------------------------------------------+ @@ -20496,8 +20492,6 @@ def aes_decrypt( return _invoke_function_over_columns("aes_decrypt", input, key, _mode, _padding, _aad) -# TODO(SPARK-46738) Re-enable testing that includes the 'Cast' operation after -# fixing the display difference between Regular Spark and Spark Connect on `Cast`. @_try_remote_functions def try_aes_decrypt( input: "ColumnOrName", @@ -20553,7 +20547,7 @@ def try_aes_decrypt( ... ) >>> df.select(sf.try_aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +-------------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| +-------------------------------------------------------------------------+ @@ -20571,7 +20565,7 @@ def try_aes_decrypt( ... ) >>> df.select(sf.try_aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode, df.padding, df.aad - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +-------------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, aad) AS STRING)| +-------------------------------------------------------------------------+ @@ -20588,7 +20582,7 @@ def try_aes_decrypt( ... ) >>> df.select(sf.try_aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode, df.padding - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +----------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, padding, ) AS STRING)| +----------------------------------------------------------------------+ @@ -20605,7 +20599,7 @@ def try_aes_decrypt( ... ) >>> df.select(sf.try_aes_decrypt( ... sf.unbase64(df.input), df.key, df.mode - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +----------------------------------------------------------------------+ |CAST(try_aes_decrypt(unbase64(input), key, mode, DEFAULT, ) AS STRING)| +----------------------------------------------------------------------+ @@ -20622,7 +20616,7 @@ def try_aes_decrypt( ... ) >>> df.select(sf.try_aes_decrypt( ... sf.unhex(df.input), df.key - ... ).cast("STRING")).show(truncate=False) # doctest: +SKIP + ... ).cast("STRING")).show(truncate=False) +------------------------------------------------------------------+ |CAST(try_aes_decrypt(unhex(input), key, GCM, DEFAULT, ) AS STRING)| +------------------------------------------------------------------+ From 297a9d2ac77373157473950f607728b6f4c1c542 Mon Sep 17 00:00:00 2001 From: Wei Guo Date: Thu, 11 Jul 2024 08:16:37 -0700 Subject: [PATCH 34/64] [MINOR][SQL][TESTS] Remove a duplicate test case in `CSVExprUtilsSuite` ### What changes were proposed in this pull request? This PR aims to remove a duplicate test case in `CSVExprUtilsSuite`. ### Why are the changes needed? Clean duplicate code. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47298 from wayneguow/csv_suite. Authored-by: Wei Guo Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala | 2 -- 1 file changed, 2 deletions(-) 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 d4b68500e0789..7d9015e566a8c 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 @@ -93,8 +93,6 @@ class CSVExprUtilsSuite extends SparkFunSuite { // tab in the middle of some other letters ("""ba\tr""", Some("ba\tr"), None), // null character, expressed in Unicode literal syntax - ("\u0000", Some("\u0000"), None), - // and specified directly ("\u0000", Some("\u0000"), None) ) From 4501285a49e4c0429c9cf2c105f044e1c8a93d21 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Thu, 11 Jul 2024 10:44:51 -0700 Subject: [PATCH 35/64] [SPARK-48775][SQL][STS] Replace SQLContext with SparkSession in STS ### What changes were proposed in this pull request? Remove the exposed `SQLContext` which was added in SPARK-46575. And migrate STS internal used `SQLContext` to `SparkSession`. ### Why are the changes needed? `SQLContext` is not recommended since Spark 2.0, the suggested replacement is `SparkSession`. We should avoid exposing the deprecated class to Developer API in new versions. ### Does this PR introduce _any_ user-facing change? No. It touched the Developer API added in SPARK-46575, but is not released yet. ### How was this patch tested? Pass GHA, and `dev/mima` (not breaking changes involved) ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47176 from pan3793/SPARK-48775. Lead-authored-by: Cheng Pan Co-authored-by: Cheng Pan Signed-off-by: Dongjoon Hyun --- .../hive/thriftserver/HiveThriftServer2.scala | 27 +++++---- .../SparkExecuteStatementOperation.scala | 26 ++++----- .../SparkGetCatalogsOperation.scala | 8 +-- .../SparkGetColumnsOperation.scala | 10 ++-- .../SparkGetFunctionsOperation.scala | 10 ++-- .../SparkGetSchemasOperation.scala | 12 ++-- .../SparkGetTableTypesOperation.scala | 8 +-- .../SparkGetTablesOperation.scala | 10 ++-- .../SparkGetTypeInfoOperation.scala | 8 +-- .../hive/thriftserver/SparkOperation.scala | 14 ++--- .../hive/thriftserver/SparkSQLCLIDriver.scala | 18 +++--- .../thriftserver/SparkSQLCLIService.scala | 8 +-- .../hive/thriftserver/SparkSQLDriver.scala | 10 ++-- .../sql/hive/thriftserver/SparkSQLEnv.scala | 11 ++-- .../thriftserver/SparkSQLSessionManager.scala | 35 ++++++------ .../server/SparkSQLOperationManager.scala | 56 +++++++++---------- .../thriftserver/SharedThriftServer.scala | 15 ++--- .../SparkExecuteStatementOperationSuite.scala | 12 ++-- 18 files changed, 150 insertions(+), 148 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 7bbab24966580..a1ed6e61e8bed 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -30,7 +30,7 @@ import org.apache.spark.SparkContext import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.UI.UI_ENABLED -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.ui._ @@ -49,30 +49,32 @@ object HiveThriftServer2 extends Logging { /** * :: DeveloperApi :: - * Starts a new thrift server with the given context. + * Starts a new thrift server with the given SparkSession. * - * @param sqlContext SQLContext to use for the server + * @param sparkSession SparkSession to use for the server * @param exitOnError Whether to exit the JVM if HiveThriftServer2 fails to initialize. When true, * the call logs the error and exits the JVM with exit code -1. When false, the * call throws an exception instead. */ @Since("4.0.0") @DeveloperApi - def startWithContext(sqlContext: SQLContext, exitOnError: Boolean): HiveThriftServer2 = { + def startWithSparkSession( + sparkSession: SparkSession, + exitOnError: Boolean): HiveThriftServer2 = { systemExitOnError.set(exitOnError) val executionHive = HiveUtils.newClientForExecution( - sqlContext.sparkContext.conf, - sqlContext.sessionState.newHadoopConf()) + sparkSession.sparkContext.conf, + sparkSession.sessionState.newHadoopConf()) // Cleanup the scratch dir before starting ServerUtils.cleanUpScratchDir(executionHive.conf) - val server = new HiveThriftServer2(sqlContext) + val server = new HiveThriftServer2(sparkSession) server.init(executionHive.conf) server.start() logInfo("HiveThriftServer2 started") - createListenerAndUI(server, sqlContext.sparkContext) + createListenerAndUI(server, sparkSession.sparkContext) server } @@ -82,10 +84,11 @@ object HiveThriftServer2 extends Logging { * * @param sqlContext SQLContext to use for the server */ + @deprecated("Use startWithSparkSession instead", since = "4.0.0") @Since("2.0.0") @DeveloperApi def startWithContext(sqlContext: SQLContext): HiveThriftServer2 = { - startWithContext(sqlContext, exitOnError = true) + startWithSparkSession(sqlContext.sparkSession, exitOnError = true) } private def createListenerAndUI(server: HiveThriftServer2, sc: SparkContext): Unit = { @@ -122,7 +125,7 @@ object HiveThriftServer2 extends Logging { } try { - startWithContext(SparkSQLEnv.sqlContext) + startWithContext(SparkSQLEnv.sparkSession.sqlContext) // If application was killed before HiveThriftServer2 start successfully then SparkSubmit // process can not exit, so check whether if SparkContext was stopped. if (SparkSQLEnv.sparkContext.stopped.get()) { @@ -142,7 +145,7 @@ object HiveThriftServer2 extends Logging { } } -private[hive] class HiveThriftServer2(sqlContext: SQLContext) +private[hive] class HiveThriftServer2(sparkSession: SparkSession) extends HiveServer2 with ReflectedCompositeService { // state is tracked internally so that the server only attempts to shut down if it successfully @@ -150,7 +153,7 @@ private[hive] class HiveThriftServer2(sqlContext: SQLContext) private val started = new AtomicBoolean(false) override def init(hiveConf: HiveConf): Unit = { - val sparkSqlCliService = new SparkSQLCLIService(this, sqlContext) + val sparkSqlCliService = new SparkSQLCLIService(this, sparkSession) setSuperField(this, "cliService", sparkSqlCliService) addService(sparkSqlCliService) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 51a5e88aa633e..2cd67cdd03bd9 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -32,7 +32,7 @@ import org.apache.hive.service.rpc.thrift.{TCLIServiceConstants, TColumnDesc, TP import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.{DataFrame, Row, SQLContext} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.util.CharVarcharUtils import org.apache.spark.sql.catalyst.util.DateTimeConstants.MILLIS_PER_SECOND import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution} @@ -40,7 +40,7 @@ import org.apache.spark.sql.types._ import org.apache.spark.util.{Utils => SparkUtils} private[hive] class SparkExecuteStatementOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession, statement: String, confOverlay: JMap[String, String], @@ -50,8 +50,6 @@ private[hive] class SparkExecuteStatementOperation( with SparkOperation with Logging { - val session = sqlContext.sparkSession - // If a timeout value `queryTimeout` is specified by users and it is smaller than // a global timeout value, we use the user-specified value. // This code follows the Hive timeout behaviour (See #29933 for details). @@ -90,10 +88,10 @@ private[hive] class SparkExecuteStatementOperation( def getNextRowSet(order: FetchOrientation, maxRowsL: Long): TRowSet = withLocalProperties { try { - sqlContext.sparkContext.setJobGroup(statementId, redactedStatement, forceCancel) + session.sparkContext.setJobGroup(statementId, redactedStatement, forceCancel) getNextRowSetInternal(order, maxRowsL) } finally { - sqlContext.sparkContext.clearJobGroup() + session.sparkContext.clearJobGroup() } } @@ -224,7 +222,7 @@ private[hive] class SparkExecuteStatementOperation( } } // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) // Always set the session state classloader to `executionHiveClassLoader` even for sync mode @@ -232,12 +230,12 @@ private[hive] class SparkExecuteStatementOperation( parentSession.getSessionState.getConf.setClassLoader(executionHiveClassLoader) } - sqlContext.sparkContext.setJobGroup(statementId, redactedStatement, forceCancel) - result = sqlContext.sql(statement) + session.sparkContext.setJobGroup(statementId, redactedStatement, forceCancel) + result = session.sql(statement) logDebug(result.queryExecution.toString()) HiveThriftServer2.eventManager.onStatementParsed(statementId, result.queryExecution.toString()) - iter = if (sqlContext.getConf(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { + iter = if (session.conf.get(SQLConf.THRIFTSERVER_INCREMENTAL_COLLECT.key).toBoolean) { new IterableFetchIterator[Row](new Iterable[Row] { override def iterator: Iterator[Row] = result.toLocalIterator().asScala }) @@ -254,7 +252,7 @@ private[hive] class SparkExecuteStatementOperation( // task interrupted, it may have start some spark job, so we need to cancel again to // make sure job was cancelled when background thread was interrupted if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) + session.sparkContext.cancelJobGroup(statementId) } val currentState = getStatus().getState() if (currentState.isTerminal) { @@ -271,7 +269,7 @@ private[hive] class SparkExecuteStatementOperation( e match { case _: HiveSQLException => throw e case _ => throw HiveThriftServerErrors.runningQueryError( - e, sqlContext.sparkSession.sessionState.conf.errorMessageFormat) + e, session.sessionState.conf.errorMessageFormat) } } } finally { @@ -281,7 +279,7 @@ private[hive] class SparkExecuteStatementOperation( HiveThriftServer2.eventManager.onStatementFinish(statementId) } } - sqlContext.sparkContext.clearJobGroup() + session.sparkContext.clearJobGroup() } } @@ -318,7 +316,7 @@ private[hive] class SparkExecuteStatementOperation( } // RDDs will be cleaned automatically upon garbage collection. if (statementId != null) { - sqlContext.sparkContext.cancelJobGroup(statementId) + session.sparkContext.cancelJobGroup(statementId) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala index fd99a5b246d9b..e4bb91d466ff4 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetCatalogsOperation.scala @@ -24,16 +24,16 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Spark's own GetCatalogsOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager */ private[hive] class SparkGetCatalogsOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession) extends GetCatalogsOperation(parentSession) with SparkOperation @@ -44,7 +44,7 @@ private[hive] class SparkGetCatalogsOperation( logInfo(log"Listing catalogs with ${MDC(STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) HiveThriftServer2.eventManager.onStatementStart( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala index 507dfc2ec50eb..1004ca8cf2712 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -29,7 +29,7 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.types._ @@ -37,7 +37,7 @@ import org.apache.spark.sql.types._ /** * Spark's own SparkGetColumnsOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager * @param catalogName catalog name. NULL if not applicable. * @param schemaName database name, NULL or a concrete database name @@ -45,7 +45,7 @@ import org.apache.spark.sql.types._ * @param columnName column name */ private[hive] class SparkGetColumnsOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession, catalogName: String, schemaName: String, @@ -55,7 +55,7 @@ private[hive] class SparkGetColumnsOperation( with SparkOperation with Logging { - val catalog: SessionCatalog = sqlContext.sessionState.catalog + val catalog: SessionCatalog = session.sessionState.catalog override def runInternal(): Unit = { // Do not change cmdStr. It's used for Hive auditing and authorization. @@ -72,7 +72,7 @@ private[hive] class SparkGetColumnsOperation( setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) HiveThriftServer2.eventManager.onStatementStart( diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala index b060bf3d4ec8d..515e64f5f529c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetFunctionsOperation.scala @@ -28,19 +28,19 @@ import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATA import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, LogKeys, MDC} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Spark's own GetFunctionsOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager * @param catalogName catalog name. null if not applicable * @param schemaName database name, null or a concrete database name * @param functionName function name pattern */ private[hive] class SparkGetFunctionsOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession, catalogName: String, schemaName: String, @@ -58,10 +58,10 @@ private[hive] class SparkGetFunctionsOperation( logInfo(logMDC + log" with ${MDC(LogKeys.STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - val catalog = sqlContext.sessionState.catalog + val catalog = session.sessionState.catalog // get databases for schema pattern val schemaPattern = convertSchemaPattern(schemaName) val matchingDbs = catalog.listDatabases(schemaPattern) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala index db1cf201b2e92..0e2c35b5ef550 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -27,18 +27,18 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Spark's own GetSchemasOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager * @param catalogName catalog name. null if not applicable. * @param schemaName database name, null or a concrete database name */ private[hive] class SparkGetSchemasOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession, catalogName: String, schemaName: String) @@ -59,7 +59,7 @@ private[hive] class SparkGetSchemasOperation( setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) if (isAuthV2Enabled) { @@ -75,11 +75,11 @@ private[hive] class SparkGetSchemasOperation( try { val schemaPattern = convertSchemaPattern(schemaName) - sqlContext.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => + session.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => rowSet.addRow(Array[AnyRef](dbName, DEFAULT_HIVE_CATALOG)) } - val globalTempViewDb = sqlContext.sessionState.catalog.globalTempDatabase + val globalTempViewDb = session.sessionState.catalog.globalTempDatabase val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) if (schemaName == null || schemaName.isEmpty || databasePattern.matcher(globalTempViewDb).matches()) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala index a0c6cd1dcd92f..9709739a64a42 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTableTypesOperation.scala @@ -26,17 +26,17 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTableType /** * Spark's own GetTableTypesOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager */ private[hive] class SparkGetTableTypesOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession) extends GetTableTypesOperation(parentSession) with SparkOperation @@ -48,7 +48,7 @@ private[hive] class SparkGetTableTypesOperation( logInfo(log"Listing table types with ${MDC(STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) if (isAuthV2Enabled) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala index 9d90878050678..e1dd6e8dd95bc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -29,13 +29,13 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ /** * Spark's own GetTablesOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager * @param catalogName catalog name. null if not applicable * @param schemaName database name, null or a concrete database name @@ -43,7 +43,7 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ * @param tableTypes list of allowed table types, e.g. "TABLE", "VIEW" */ private[hive] class SparkGetTablesOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession, catalogName: String, schemaName: String, @@ -68,10 +68,10 @@ private[hive] class SparkGetTablesOperation( log"with ${MDC(STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) - val catalog = sqlContext.sessionState.catalog + val catalog = session.sessionState.catalog val schemaPattern = convertSchemaPattern(schemaName) val tablePattern = convertIdentifierPattern(tableName, true) val matchingDbs = catalog.listDatabases(schemaPattern) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala index 9ae62ed2fed74..456ec44678c5c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTypeInfoOperation.scala @@ -28,16 +28,16 @@ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession /** * Spark's own GetTypeInfoOperation * - * @param sqlContext SQLContext to use + * @param session SparkSession to use * @param parentSession a HiveSession from SessionManager */ private[hive] class SparkGetTypeInfoOperation( - val sqlContext: SQLContext, + val session: SparkSession, parentSession: HiveSession) extends GetTypeInfoOperation(parentSession) with SparkOperation @@ -49,7 +49,7 @@ private[hive] class SparkGetTypeInfoOperation( logInfo(log"Listing type info with ${MDC(STATEMENT_ID, statementId)}") setState(OperationState.RUNNING) // Always use the latest class loader provided by executionHive's state. - val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + val executionHiveClassLoader = session.sharedState.jarClassLoader Thread.currentThread().setContextClassLoader(executionHiveClassLoader) if (isAuthV2Enabled) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala index 11e4817fe2a4c..b56888f49c1bf 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkOperation.scala @@ -23,7 +23,7 @@ import org.apache.hive.service.cli.operation.Operation import org.apache.spark.SparkContext import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.{HIVE_OPERATION_TYPE, STATEMENT_ID} -import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.CurrentUserContext.CURRENT_USER import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.catalyst.catalog.CatalogTableType.{EXTERNAL, MANAGED, VIEW} @@ -35,7 +35,7 @@ import org.apache.spark.util.Utils */ private[hive] trait SparkOperation extends Operation with Logging { - protected def sqlContext: SQLContext + protected def session: SparkSession protected var statementId = getHandle().getHandleIdentifier().getPublicId().toString() @@ -62,17 +62,17 @@ private[hive] trait SparkOperation extends Operation with Logging { // - set appropriate SparkSession // - set scheduler pool for the operation def withLocalProperties[T](f: => T): T = { - val originalProps = Utils.cloneProperties(sqlContext.sparkContext.getLocalProperties) + val originalProps = Utils.cloneProperties(session.sparkContext.getLocalProperties) val originalSession = SparkSession.getActiveSession try { // Set active SparkSession - SparkSession.setActiveSession(sqlContext.sparkSession) + SparkSession.setActiveSession(session) // Set scheduler pool - sqlContext.sparkSession.conf.getOption(SQLConf.THRIFTSERVER_POOL.key) match { + session.conf.getOption(SQLConf.THRIFTSERVER_POOL.key) match { case Some(pool) => - sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) + session.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) case None => } CURRENT_USER.set(getParentSession.getUserName) @@ -81,7 +81,7 @@ private[hive] trait SparkOperation extends Operation with Logging { } finally { CURRENT_USER.remove() // reset local properties, will also reset SPARK_SCHEDULER_POOL - sqlContext.sparkContext.setLocalProperties(originalProps) + session.sparkContext.setLocalProperties(originalProps) originalSession match { case Some(session) => SparkSession.setActiveSession(session) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index e64e1c283e27c..faab14bb9e365 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -167,7 +167,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { // Spark's SessionResourceLoader to obtain these jars. val auxJars = HiveConf.getVar(conf, HiveConf.getConfVars("hive.aux.jars.path")) if (StringUtils.isNotBlank(auxJars)) { - val resourceLoader = SparkSQLEnv.sqlContext.sessionState.resourceLoader + val resourceLoader = SparkSQLEnv.sparkSession.sessionState.resourceLoader StringUtils.split(auxJars, ",").foreach(resourceLoader.addJar(_)) } @@ -176,7 +176,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { // sharedState.jarClassLoader which contain jar path passed by --jars in main thread. // We set CliSessionState's conf class loader to sharedState.jarClassLoader. // Thus we can load all jars passed by --jars and AddJarsCommand. - sessionState.getConf.setClassLoader(SparkSQLEnv.sqlContext.sharedState.jarClassLoader) + sessionState.getConf.setClassLoader(SparkSQLEnv.sparkSession.sharedState.jarClassLoader) // TODO work around for set the log output to console, because the HiveContext // will set the output into an invalid buffer. @@ -193,11 +193,11 @@ private[hive] object SparkSQLCLIDriver extends Logging { // [[SharedState.loadHiveConfFile]] based on the user specified or default values of // spark.sql.warehouse.dir and hive.metastore.warehouse.dir. for ((k, v) <- newHiveConf if k != "hive.metastore.warehouse.dir") { - SparkSQLEnv.sqlContext.setConf(k, v) + SparkSQLEnv.sparkSession.conf.set(k, v) } if (sessionState.database != null) { - SparkSQLEnv.sqlContext.sql(s"USE ${sessionState.database}") + SparkSQLEnv.sparkSession.sql(s"USE ${sessionState.database}") } // Execute -i init files (always in silent mode) @@ -261,9 +261,9 @@ private[hive] object SparkSQLCLIDriver extends Logging { var prefix = "" def currentDB = { - if (!SparkSQLEnv.sqlContext.sparkSession.sessionState.conf + if (!SparkSQLEnv.sparkSession.sessionState.conf .getConf(LEGACY_EMPTY_CURRENT_DB_IN_CLI)) { - s" (${SparkSQLEnv.sqlContext.sparkSession.catalog.currentDatabase})" + s" (${SparkSQLEnv.sparkSession.catalog.currentDatabase})" } else { ReflectionUtils.invokeStatic(classOf[CliDriver], "getFormattedDb", classOf[HiveConf] -> conf, classOf[CliSessionState] -> sessionState) @@ -409,7 +409,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { override def setHiveVariables(hiveVariables: java.util.Map[String, String]): Unit = { hiveVariables.asScala.foreach(kv => - SparkSQLEnv.sqlContext.sparkSession.sessionState.conf.setConfString(kv._1, kv._2)) + SparkSQLEnv.sparkSession.sessionState.conf.setConfString(kv._1, kv._2)) } def printMasterAndAppId(): Unit = { @@ -464,7 +464,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { ret = rc.getResponseCode if (ret != 0) { - val format = SparkSQLEnv.sqlContext.sparkSession.sessionState.conf.errorMessageFormat + val format = SparkSQLEnv.sparkSession.sessionState.conf.errorMessageFormat val e = rc.getException val msg = e match { case st: SparkThrowable with Throwable => SparkThrowableHelper.getMessage(st, format) @@ -483,7 +483,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val res = new JArrayList[String]() if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_CLI_PRINT_HEADER) || - SparkSQLEnv.sqlContext.sparkSession.sessionState.conf.cliPrintHeader) { + SparkSQLEnv.sparkSession.sessionState.conf.cliPrintHeader) { // Print the column names. Option(driver.getSchema.getFieldSchemas).foreach { fields => out.println(fields.asScala.map(_.getName).mkString("\t")) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 46537f75f1a11..092b6b444fdb8 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -35,19 +35,19 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 import org.apache.spark.internal.SparkLogger -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util.SQLKeywordUtils import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLContext) +private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sparkSession: SparkSession) extends CLIService(hiveServer) with ReflectedCompositeService { override def init(hiveConf: HiveConf): Unit = { setSuperField(this, "hiveConf", hiveConf) - val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, sqlContext) + val sparkSqlSessionManager = new SparkSQLSessionManager(hiveServer, sparkSession) setSuperField(this, "sessionManager", sparkSqlSessionManager) addService(sparkSqlSessionManager) var sparkServiceUGI: UserGroupInformation = null @@ -103,7 +103,7 @@ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLC getInfoType match { case GetInfoType.CLI_SERVER_NAME => new GetInfoValue("Spark SQL") case GetInfoType.CLI_DBMS_NAME => new GetInfoValue("Spark SQL") - case GetInfoType.CLI_DBMS_VER => new GetInfoValue(sqlContext.sparkContext.version) + case GetInfoType.CLI_DBMS_VER => new GetInfoValue(sparkSession.version) case GetInfoType.CLI_ODBC_KEYWORDS => new GetInfoValue(SQLKeywordUtils.keywords.mkString(",")) case _ => super.getInfo(sessionHandle, getInfoType) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index a9c5d3e250797..650a5df340215 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -29,14 +29,14 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.SparkThrowable import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.COMMAND -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.CommandResult import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution} -private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) +private[hive] class SparkSQLDriver(val sparkSession: SparkSession = SparkSQLEnv.sparkSession) extends Driver with Logging { @@ -62,11 +62,11 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont override def run(command: String): CommandProcessorResponse = { try { - val substitutorCommand = SQLConf.withExistingConf(context.sparkSession.sessionState.conf) { + val substitutorCommand = SQLConf.withExistingConf(sparkSession.sessionState.conf) { new VariableSubstitution().substitute(command) } - context.sparkContext.setJobDescription(substitutorCommand) - val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) + sparkSession.sparkContext.setJobDescription(substitutorCommand) + val execution = sparkSession.sessionState.executePlan(sparkSession.sql(command).logicalPlan) // The SQL command has been executed above via `executePlan`, therefore we don't need to // wrap it again with a new execution ID when getting Hive result. execution.logical match { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 88a5c87eab5d9..f1a49745fa33c 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -22,7 +22,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hive.HiveExternalCatalog import org.apache.spark.sql.hive.HiveUtils._ import org.apache.spark.sql.internal.SQLConf @@ -33,11 +33,11 @@ import org.apache.spark.util.Utils private[hive] object SparkSQLEnv extends Logging { logDebug("Initializing SparkSQLEnv") - var sqlContext: SQLContext = _ + var sparkSession: SparkSession = _ var sparkContext: SparkContext = _ def init(): Unit = { - if (sqlContext == null) { + if (sparkSession == null) { val sparkConf = new SparkConf(loadDefaults = true) // If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of // the default appName [SparkSQLCLIDriver] in cli or beeline. @@ -61,9 +61,8 @@ private[hive] object SparkSQLEnv extends Logging { if (!shouldUseInMemoryCatalog) { builder.enableHiveSupport() } - val sparkSession = builder.getOrCreate() + sparkSession = builder.getOrCreate() sparkContext = sparkSession.sparkContext - sqlContext = sparkSession.sqlContext // SPARK-29604: force initialization of the session state with the Spark class loader, // instead of having it happen during the initialization of the Hive client (which may use a @@ -87,7 +86,7 @@ private[hive] object SparkSQLEnv extends Logging { if (SparkSQLEnv.sparkContext != null) { sparkContext.stop(exitCode) sparkContext = null - sqlContext = null + sparkSession = null } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 7acc485b01e57..ce3ece75c0aeb 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -26,13 +26,13 @@ import org.apache.hive.service.rpc.thrift.TProtocolVersion import org.apache.hive.service.server.HiveServer2 import org.apache.spark.internal.Logging -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.thriftserver.server.SparkSQLOperationManager import org.apache.spark.sql.internal.SQLConf -private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: SQLContext) +private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sparkSession: SparkSession) extends SessionManager(hiveServer) with ReflectedCompositeService with Logging { @@ -55,22 +55,22 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: super.openSession(protocol, username, passwd, ipAddress, sessionConf, withImpersonation, delegationToken) try { - val session = super.getSession(sessionHandle) + val hiveSession = super.getSession(sessionHandle) HiveThriftServer2.eventManager.onSessionCreated( - session.getIpAddress, sessionHandle.getSessionId.toString, session.getUsername) - val ctx = if (sqlContext.sparkSession.sessionState.conf.hiveThriftServerSingleSession) { - sqlContext + hiveSession.getIpAddress, sessionHandle.getSessionId.toString, hiveSession.getUsername) + val session = if (sparkSession.sessionState.conf.hiveThriftServerSingleSession) { + sparkSession } else { - sqlContext.newSession() + sparkSession.newSession() } - ctx.setConf(SQLConf.DATETIME_JAVA8API_ENABLED, true) - val hiveSessionState = session.getSessionState - setConfMap(ctx, hiveSessionState.getOverriddenConfigurations) - setConfMap(ctx, hiveSessionState.getHiveVariables) + session.sessionState.conf.setConf(SQLConf.DATETIME_JAVA8API_ENABLED, true) + val hiveSessionState = hiveSession.getSessionState + setConfMap(session, hiveSessionState.getOverriddenConfigurations) + setConfMap(session, hiveSessionState.getHiveVariables) if (sessionConf != null && sessionConf.containsKey("use:database")) { - ctx.sql(s"use ${sessionConf.get("use:database")}") + session.sql(s"use ${sessionConf.get("use:database")}") } - sparkSqlOperationManager.sessionToContexts.put(sessionHandle, ctx) + sparkSqlOperationManager.sessionToContexts.put(sessionHandle, session) sessionHandle } catch { case NonFatal(e) => @@ -86,17 +86,18 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: override def closeSession(sessionHandle: SessionHandle): Unit = { HiveThriftServer2.eventManager.onSessionClosed(sessionHandle.getSessionId.toString) - val ctx = sparkSqlOperationManager.sessionToContexts.getOrDefault(sessionHandle, sqlContext) - ctx.sparkSession.sessionState.catalog.getTempViewNames().foreach(ctx.uncacheTable) + val session = sparkSqlOperationManager.sessionToContexts + .getOrDefault(sessionHandle, sparkSession) + session.sessionState.catalog.getTempViewNames().foreach(session.catalog.uncacheTable) super.closeSession(sessionHandle) sparkSqlOperationManager.sessionToContexts.remove(sessionHandle) } - def setConfMap(conf: SQLContext, confMap: java.util.Map[String, String]): Unit = { + def setConfMap(sparkSession: SparkSession, confMap: java.util.Map[String, String]): Unit = { val iterator = confMap.entrySet().iterator() while (iterator.hasNext) { val kv = iterator.next() - conf.setConf(kv.getKey, kv.getValue) + sparkSession.conf.set(kv.getKey, kv.getValue) } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index ba42eefed2a22..8e12165dd6f14 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -25,7 +25,7 @@ import org.apache.hive.service.cli.operation._ import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver._ @@ -38,7 +38,7 @@ private[thriftserver] class SparkSQLOperationManager() val handleToOperation = ReflectionUtils .getSuperField[JMap[OperationHandle, Operation]](this, "handleToOperation") - val sessionToContexts = new ConcurrentHashMap[SessionHandle, SQLContext]() + val sessionToContexts = new ConcurrentHashMap[SessionHandle, SparkSession]() override def newExecuteStatementOperation( parentSession: HiveSession, @@ -46,13 +46,13 @@ private[thriftserver] class SparkSQLOperationManager() confOverlay: JMap[String, String], async: Boolean, queryTimeout: Long): ExecuteStatementOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + - s" initialized or had already closed.") - val conf = sqlContext.sessionState.conf + val sparkSession = sessionToContexts.get(parentSession.getSessionHandle) + require(sparkSession != null, s"Session handle: ${parentSession.getSessionHandle} " + + s"has not been initialized or had already closed.") + val conf = sparkSession.sessionState.conf val runInBackground = async && conf.getConf(HiveUtils.HIVE_THRIFT_SERVER_ASYNC) val operation = new SparkExecuteStatementOperation( - sqlContext, parentSession, statement, confOverlay, runInBackground, queryTimeout) + sparkSession, parentSession, statement, confOverlay, runInBackground, queryTimeout) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created Operation for $statement with session=$parentSession, " + s"runInBackground=$runInBackground") @@ -61,10 +61,10 @@ private[thriftserver] class SparkSQLOperationManager() override def newGetCatalogsOperation( parentSession: HiveSession): GetCatalogsOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetCatalogsOperation(sqlContext, parentSession) + val operation = new SparkGetCatalogsOperation(session, parentSession) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetCatalogsOperation with session=$parentSession.") operation @@ -74,10 +74,10 @@ private[thriftserver] class SparkSQLOperationManager() parentSession: HiveSession, catalogName: String, schemaName: String): GetSchemasOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetSchemasOperation(sqlContext, parentSession, catalogName, schemaName) + val operation = new SparkGetSchemasOperation(session, parentSession, catalogName, schemaName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetSchemasOperation with session=$parentSession.") operation @@ -89,10 +89,10 @@ private[thriftserver] class SparkSQLOperationManager() schemaName: String, tableName: String, tableTypes: JList[String]): MetadataOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTablesOperation(sqlContext, parentSession, + val operation = new SparkGetTablesOperation(session, parentSession, catalogName, schemaName, tableName, tableTypes) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTablesOperation with session=$parentSession.") @@ -105,10 +105,10 @@ private[thriftserver] class SparkSQLOperationManager() schemaName: String, tableName: String, columnName: String): GetColumnsOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetColumnsOperation(sqlContext, parentSession, + val operation = new SparkGetColumnsOperation(session, parentSession, catalogName, schemaName, tableName, columnName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetColumnsOperation with session=$parentSession.") @@ -117,10 +117,10 @@ private[thriftserver] class SparkSQLOperationManager() override def newGetTableTypesOperation( parentSession: HiveSession): GetTableTypesOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTableTypesOperation(sqlContext, parentSession) + val operation = new SparkGetTableTypesOperation(session, parentSession) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTableTypesOperation with session=$parentSession.") operation @@ -131,10 +131,10 @@ private[thriftserver] class SparkSQLOperationManager() catalogName: String, schemaName: String, functionName: String): GetFunctionsOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetFunctionsOperation(sqlContext, parentSession, + val operation = new SparkGetFunctionsOperation(session, parentSession, catalogName, schemaName, functionName) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetFunctionsOperation with session=$parentSession.") @@ -143,10 +143,10 @@ private[thriftserver] class SparkSQLOperationManager() override def newGetTypeInfoOperation( parentSession: HiveSession): GetTypeInfoOperation = synchronized { - val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) - require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + val session = sessionToContexts.get(parentSession.getSessionHandle) + require(session != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + " initialized or had already closed.") - val operation = new SparkGetTypeInfoOperation(sqlContext, parentSession) + val operation = new SparkGetTypeInfoOperation(session, parentSession) handleToOperation.put(operation.getHandle, operation) logDebug(s"Created GetTypeInfoOperation with session=$parentSession.") operation diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala index edd247e48f530..6f0fedcb85368 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SharedThriftServer.scala @@ -128,21 +128,22 @@ trait SharedThriftServer extends SharedSparkSession { private def startThriftServer(attempt: Int): Unit = { logInfo(s"Trying to start HiveThriftServer2: mode=$mode, attempt=$attempt") - val sqlContext = spark.newSession().sqlContext + val sparkSession = spark.newSession() // Set the HIVE_SERVER2_THRIFT_PORT and HIVE_SERVER2_THRIFT_HTTP_PORT to 0, so it could // randomly pick any free port to use. // It's much more robust than set a random port generated by ourselves ahead - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") - sqlContext.setConf(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) - sqlContext.setConf("hive.exec.scratchdir", tempScratchDir.getAbsolutePath) - sqlContext.setConf(ConfVars.HIVE_START_CLEANUP_SCRATCHDIR.varname, "true") + val sessionConf = sparkSession.sessionState.conf + sessionConf.setConfString(ConfVars.HIVE_SERVER2_THRIFT_PORT.varname, "0") + sessionConf.setConfString(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT.varname, "0") + sessionConf.setConfString(ConfVars.HIVE_SERVER2_TRANSPORT_MODE.varname, mode.toString) + sessionConf.setConfString("hive.exec.scratchdir", tempScratchDir.getAbsolutePath) + sessionConf.setConfString(ConfVars.HIVE_START_CLEANUP_SCRATCHDIR.varname, "true") try { // Set exitOnError to false to avoid exiting the JVM process and tearing down the SparkContext // instance in case of any exceptions here. Otherwise, the following retries are doomed to // fail on a stopped context. - hiveServer2 = HiveThriftServer2.startWithContext(sqlContext, exitOnError = false) + hiveServer2 = HiveThriftServer2.startWithSparkSession(sparkSession, exitOnError = false) hiveServer2.getServices.asScala.foreach { case t: ThriftCLIService => serverPort = t.getPortNumber diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala index d085f596397f7..83f1824c26d2b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -30,7 +30,7 @@ import org.mockito.Mockito.{doReturn, mock, spy, when, RETURNS_DEEP_STUBS} import org.mockito.invocation.InvocationOnMock import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.hive.thriftserver.ui.HiveThriftServer2EventManager import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.{IntegerType, NullType, StringType, StructField, StructType} @@ -78,7 +78,7 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark HiveThriftServer2.eventManager = mock(classOf[HiveThriftServer2EventManager]) - val spySqlContext = spy[SQLContext](sqlContext) + val spySparkSession = spy[SparkSession](spark) // When cancel() is called on the operation, cleanup causes an exception to be thrown inside // of execute(). This should not cause the state to become ERROR. The exception here will be @@ -90,9 +90,9 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark throw new RuntimeException("Operation was cancelled by test cleanup.") }) val statement = "stmt" - doReturn(dataFrame, Nil: _*).when(spySqlContext).sql(statement) + doReturn(dataFrame, Nil: _*).when(spySparkSession).sql(statement) - val executeStatementOperation = new MySparkExecuteStatementOperation(spySqlContext, + val executeStatementOperation = new MySparkExecuteStatementOperation(spySparkSession, hiveSession, statement, signal, finalState) val run = new Thread() { @@ -110,12 +110,12 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite with SharedSpark } private class MySparkExecuteStatementOperation( - sqlContext: SQLContext, + session: SparkSession, hiveSession: HiveSession, statement: String, signal: Semaphore, finalState: OperationState) - extends SparkExecuteStatementOperation(sqlContext, hiveSession, statement, + extends SparkExecuteStatementOperation(session, hiveSession, statement, new util.HashMap, false, 0) { override def cleanup(): Unit = { From 71cf25ec7b0a234fe5570e952a76afb8eb1d1704 Mon Sep 17 00:00:00 2001 From: Amanda Liu Date: Thu, 11 Jul 2024 14:12:32 -0700 Subject: [PATCH 36/64] [SPARK-48623][CORE] Structured logging migrations [Part 3] ### What changes were proposed in this pull request? This PR makes additional Scala logging migrations to comply with the scala style changes in https://github.com/apache/spark/pull/46947 ### Why are the changes needed? This makes development and PR review of the structured logging migration easier. ### Does this PR introduce any user-facing change? No ### How was this patch tested? Tested by ensuring dev/scalastyle checks pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #47275 from asl3/formatstructuredlogmigrations. Lead-authored-by: Amanda Liu Co-authored-by: Gengliang Wang Signed-off-by: Gengliang Wang --- .../org/apache/spark/internal/LogKey.scala | 7 ++++++ .../org/apache/spark/util/MavenUtils.scala | 5 ++-- .../sql/connect/service/SessionHolder.scala | 25 +++++++++++-------- .../SparkConnectListenerBusListener.scala | 10 ++++---- .../org/apache/spark/SecurityManager.scala | 13 +++++----- .../spark/api/python/PythonRunner.scala | 24 +++++++++++------- .../apache/spark/scheduler/DAGScheduler.scala | 12 +++++---- .../spark/util/ShutdownHookManager.scala | 2 +- .../spark/util/collection/Spillable.scala | 9 ++++--- .../mllib/clustering/StreamingKMeans.scala | 4 +-- .../mllib/optimization/GradientDescent.scala | 4 +-- .../spark/mllib/optimization/LBFGS.scala | 6 ++--- .../sql/hive/client/HiveClientImpl.scala | 6 ++--- .../dstream/DStreamCheckpointData.scala | 3 ++- .../streaming/dstream/FileInputDStream.scala | 3 ++- .../streaming/dstream/RawInputDStream.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 4 +-- 17 files changed, 81 insertions(+), 58 deletions(-) diff --git a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala index 59e467402aa8d..51ef112a677d4 100644 --- a/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala +++ b/common/utils/src/main/scala/org/apache/spark/internal/LogKey.scala @@ -101,6 +101,7 @@ private[spark] object LogKeys { case object BLOCK_TYPE extends LogKey case object BOOT extends LogKey case object BOOTSTRAP_TIME extends LogKey + case object BOOT_TIME extends LogKey case object BROADCAST extends LogKey case object BROADCAST_ID extends LogKey case object BROADCAST_OUTPUT_STATUS_SIZE extends LogKey @@ -110,6 +111,7 @@ private[spark] object LogKeys { case object BYTE_SIZE extends LogKey case object CACHED_TABLE_PARTITION_METADATA_SIZE extends LogKey case object CACHE_AUTO_REMOVED_SIZE extends LogKey + case object CACHE_SIZE extends LogKey case object CACHE_UNTIL_HIGHEST_CONSUMED_SIZE extends LogKey case object CACHE_UNTIL_LAST_PRODUCED_SIZE extends LogKey case object CALL_SITE_LONG_FORM extends LogKey @@ -282,6 +284,7 @@ private[spark] object LogKeys { case object FINAL_CONTEXT extends LogKey case object FINAL_OUTPUT_PATH extends LogKey case object FINAL_PATH extends LogKey + case object FINISH_TIME extends LogKey case object FINISH_TRIGGER_DURATION extends LogKey case object FREE_MEMORY_SIZE extends LogKey case object FROM_OFFSET extends LogKey @@ -320,10 +323,12 @@ private[spark] object LogKeys { case object INITIAL_CAPACITY extends LogKey case object INITIAL_HEARTBEAT_INTERVAL extends LogKey case object INIT_MODE extends LogKey + case object INIT_TIME extends LogKey case object INPUT extends LogKey case object INPUT_SPLIT extends LogKey case object INTEGRAL extends LogKey case object INTERVAL extends LogKey + case object INVALID_PARAMS extends LogKey case object ISOLATION_LEVEL extends LogKey case object ISSUE_DATE extends LogKey case object IS_NETWORK_REQUEST_DONE extends LogKey @@ -369,6 +374,7 @@ private[spark] object LogKeys { case object LOG_LEVEL extends LogKey case object LOG_OFFSET extends LogKey case object LOG_TYPE extends LogKey + case object LOSSES extends LogKey case object LOWER_BOUND extends LogKey case object MALFORMATTED_STRING extends LogKey case object MAP_ID extends LogKey @@ -566,6 +572,7 @@ private[spark] object LogKeys { case object OS_NAME extends LogKey case object OS_VERSION extends LogKey case object OUTPUT extends LogKey + case object OUTPUT_BUFFER extends LogKey case object OVERHEAD_MEMORY_SIZE extends LogKey case object PAGE_SIZE extends LogKey case object PARENT_STAGES extends LogKey diff --git a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala index 546981c8b5435..42a1d1612aeeb 100644 --- a/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala +++ b/common/utils/src/main/scala/org/apache/spark/util/MavenUtils.scala @@ -650,8 +650,9 @@ private[spark] object MavenUtils extends Logging { val invalidParams = groupedParams.keys.filterNot(validParams.contains).toSeq if (invalidParams.nonEmpty) { logWarning( - s"Invalid parameters `${invalidParams.sorted.mkString(",")}` found " + - s"in Ivy URI query `$uriQuery`.") + log"Invalid parameters `${MDC(LogKeys.INVALID_PARAMS, + invalidParams.sorted.mkString(","))}` " + + log"found in Ivy URI query `${MDC(LogKeys.URI, uriQuery)}`.") } (transitive, exclusionList, repos) diff --git a/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 681f7e29630ff..fbae94afc43df 100644 --- a/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -33,8 +33,7 @@ import com.google.common.cache.{Cache, CacheBuilder} import org.apache.spark.{SparkEnv, SparkException, SparkSQLException} import org.apache.spark.api.python.PythonFunction.PythonAccumulator import org.apache.spark.connect.proto -import org.apache.spark.internal.{Logging, MDC} -import org.apache.spark.internal.LogKeys._ +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -63,9 +62,11 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio private lazy val planCache: Option[Cache[proto.Relation, LogicalPlan]] = { if (SparkEnv.get.conf.get(Connect.CONNECT_SESSION_PLAN_CACHE_SIZE) <= 0) { logWarning( - s"Session plan cache is disabled due to non-positive cache size." + - s" Current value of '${Connect.CONNECT_SESSION_PLAN_CACHE_SIZE.key}' is" + - s" ${SparkEnv.get.conf.get(Connect.CONNECT_SESSION_PLAN_CACHE_SIZE)}.") + log"Session plan cache is disabled due to non-positive cache size." + + log" Current value of " + + log"'${MDC(LogKeys.CONFIG, Connect.CONNECT_SESSION_PLAN_CACHE_SIZE.key)}' is ${MDC( + LogKeys.CACHE_SIZE, + SparkEnv.get.conf.get(Connect.CONNECT_SESSION_PLAN_CACHE_SIZE))}") None } else { Some( @@ -248,15 +249,17 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio private[connect] def updateAccessTime(): Unit = { lastAccessTimeMs = System.currentTimeMillis() logInfo( - log"Session ${MDC(SESSION_KEY, key)} accessed, " + - log"time ${MDC(LAST_ACCESS_TIME, lastAccessTimeMs)} ms.") + log"Session with userId: ${MDC(LogKeys.USER_ID, userId)} and " + + log"sessionId: ${MDC(LogKeys.SESSION_ID, sessionId)} accessed," + + log"time ${MDC(LogKeys.LAST_ACCESS_TIME, lastAccessTimeMs)} ms.") } private[connect] def setCustomInactiveTimeoutMs(newInactiveTimeoutMs: Option[Long]): Unit = { customInactiveTimeoutMs = newInactiveTimeoutMs logInfo( - log"Session ${MDC(SESSION_KEY, key)} " + - log"inactive timeout set to ${MDC(TIMEOUT, customInactiveTimeoutMs)} ms.") + log"Session with userId: ${MDC(LogKeys.USER_ID, userId)} and " + + log"sessionId: ${MDC(LogKeys.SESSION_ID, sessionId)} inactive timeout set to " + + log"${MDC(LogKeys.TIMEOUT, customInactiveTimeoutMs)} ms") } /** @@ -282,8 +285,8 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio throw new IllegalStateException(s"Session $key is already closed.") } logInfo( - log"Closing session with userId: ${MDC(USER_ID, userId)} and " + - log"sessionId: ${MDC(SESSION_ID, sessionId)}") + log"Closing session with userId: ${MDC(LogKeys.USER_ID, userId)} and " + + log"sessionId: ${MDC(LogKeys.SESSION_ID, sessionId)}") closedTimeMs = Some(System.currentTimeMillis()) if (Utils.isTesting && eventManager.status == SessionStatus.Pending) { diff --git a/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala b/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala index 56d0d920e95b4..c6baad72ee181 100644 --- a/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala +++ b/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectListenerBusListener.scala @@ -28,7 +28,7 @@ import org.apache.spark.connect.proto.ExecutePlanResponse import org.apache.spark.connect.proto.StreamingQueryEventType import org.apache.spark.connect.proto.StreamingQueryListenerEvent import org.apache.spark.connect.proto.StreamingQueryListenerEventsResult -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.sql.streaming.StreamingQueryListener import org.apache.spark.util.ArrayImplicits._ @@ -131,10 +131,10 @@ private[sql] class SparkConnectListenerBusListener( .build()) } catch { case NonFatal(e) => - logError( - s"[SessionId: ${sessionHolder.sessionId}][UserId: ${sessionHolder.userId}] " + - s"Removing SparkConnectListenerBusListener and terminating the long-running thread " + - s"because of exception: $e") + logError(log"[SessionId: ${MDC(LogKeys.SESSION_ID, sessionHolder.sessionId)}]" + + log"[UserId: ${MDC(LogKeys.USER_ID, sessionHolder.userId)}] " + + log"Removing SparkConnectListenerBusListener and terminating the long-running thread " + + log"because of exception: ${MDC(LogKeys.EXCEPTION, e)}") // This likely means that the client is not responsive even with retry, we should // remove this listener and cleanup resources. serverSideListenerHolder.cleanUp() diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 5a40afc76728e..c951876e62034 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -122,7 +122,7 @@ private[spark] class SecurityManager( */ def setViewAcls(defaultUsers: Set[String], allowedUsers: Seq[String]): Unit = { viewAcls = adminAcls ++ defaultUsers ++ allowedUsers - logInfo("Changing view acls to: " + viewAcls.mkString(",")) + logInfo(log"Changing view acls to: ${MDC(LogKeys.VIEW_ACLS, viewAcls.mkString(","))}") } def setViewAcls(defaultUser: String, allowedUsers: Seq[String]): Unit = { @@ -135,7 +135,7 @@ private[spark] class SecurityManager( */ def setViewAclsGroups(allowedUserGroups: Seq[String]): Unit = { viewAclsGroups = adminAclsGroups ++ allowedUserGroups - logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) + logInfo(log"Changing view acls groups to: ${MDC(LogKeys.VIEW_ACLS, viewAcls.mkString(","))}") } /** @@ -163,7 +163,7 @@ private[spark] class SecurityManager( */ def setModifyAcls(defaultUsers: Set[String], allowedUsers: Seq[String]): Unit = { modifyAcls = adminAcls ++ defaultUsers ++ allowedUsers - logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) + logInfo(log"Changing modify acls to: ${MDC(LogKeys.MODIFY_ACLS, modifyAcls.mkString(","))}") } /** @@ -172,7 +172,8 @@ private[spark] class SecurityManager( */ def setModifyAclsGroups(allowedUserGroups: Seq[String]): Unit = { modifyAclsGroups = adminAclsGroups ++ allowedUserGroups - logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) + logInfo(log"Changing modify acls groups to: ${MDC(LogKeys.MODIFY_ACLS, + modifyAcls.mkString(","))}") } /** @@ -200,7 +201,7 @@ private[spark] class SecurityManager( */ def setAdminAcls(adminUsers: Seq[String]): Unit = { adminAcls = adminUsers.toSet - logInfo("Changing admin acls to: " + adminAcls.mkString(",")) + logInfo(log"Changing admin acls to: ${MDC(LogKeys.ADMIN_ACLS, adminAcls.mkString(","))}") } /** @@ -209,7 +210,7 @@ private[spark] class SecurityManager( */ def setAdminAclsGroups(adminUserGroups: Seq[String]): Unit = { adminAclsGroups = adminUserGroups.toSet - logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) + logInfo(log"Changing admin acls groups to: ${MDC(LogKeys.ADMIN_ACLS, adminAcls.mkString(","))}") } def setAcls(aclSetting: Boolean): Unit = { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index b2571ffddc577..6a67587fbd80c 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -31,7 +31,7 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.api.python.PythonFunction.PythonAccumulator -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.TASK_NAME import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES, Python} import org.apache.spark.internal.config.Python._ @@ -131,9 +131,11 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val daemonModule = conf.get(PYTHON_DAEMON_MODULE).map { value => logInfo( - s"Python daemon module in PySpark is set to [$value] in '${PYTHON_DAEMON_MODULE.key}', " + - "using this to start the daemon up. Note that this configuration only has an effect when " + - s"'${PYTHON_USE_DAEMON.key}' is enabled and the platform is not Windows.") + log"Python daemon module in PySpark is set to " + + log"[${MDC(LogKeys.VALUE, value)}] in '${MDC(LogKeys.CONFIG, + PYTHON_DAEMON_MODULE.key)}', using this to start the daemon up. Note that this " + + log"configuration only has an effect when '${MDC(LogKeys.CONFIG2, + PYTHON_USE_DAEMON.key)}' is enabled and the platform is not Windows.") value }.getOrElse("pyspark.daemon") @@ -141,9 +143,11 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( private val workerModule = conf.get(PYTHON_WORKER_MODULE).map { value => logInfo( - s"Python worker module in PySpark is set to [$value] in '${PYTHON_WORKER_MODULE.key}', " + - "using this to start the worker up. Note that this configuration only has an effect when " + - s"'${PYTHON_USE_DAEMON.key}' is disabled or the platform is Windows.") + log"Python worker module in PySpark is set to ${MDC(LogKeys.VALUE, value)} " + + log"in ${MDC(LogKeys.CONFIG, PYTHON_WORKER_MODULE.key)}, " + + log"using this to start the worker up. Note that this configuration only has " + + log"an effect when ${MDC(LogKeys.CONFIG2, PYTHON_USE_DAEMON.key)} " + + log"is disabled or the platform is Windows.") value }.getOrElse("pyspark.worker") @@ -509,8 +513,10 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val init = initTime - bootTime val finish = finishTime - initTime val total = finishTime - startTime - logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot, - init, finish)) + logInfo(log"Times: total = ${MDC(LogKeys.TOTAL_TIME, total)}, " + + log"boot = ${MDC(LogKeys.BOOT_TIME, boot)}, " + + log"init = ${MDC(LogKeys.INIT_TIME, init)}, " + + log"finish = ${MDC(LogKeys.FINISH_TIME, finish)}") val memoryBytesSpilled = stream.readLong() val diskBytesSpilled = stream.readLong() context.taskMetrics().incMemoryBytesSpilled(memoryBytesSpilled) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f50e8bd25fec8..78ae5bd39d230 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -36,7 +36,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.errors.SparkCoreErrors import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} -import org.apache.spark.internal.{config, Logging, MDC} +import org.apache.spark.internal.{config, Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.internal.config.{LEGACY_ABORT_STAGE_AFTER_KILL_TASKS, RDD_CACHE_VISIBILITY_TRACKING_ENABLED} import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY @@ -998,11 +998,13 @@ private[spark] class DAGScheduler( ThreadUtils.awaitReady(waiter.completionFuture, Duration.Inf) waiter.completionFuture.value.get match { case scala.util.Success(_) => - logInfo("Job %d finished: %s, took %f s".format - (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) + logInfo(log"Job ${MDC(LogKeys.JOB_ID, waiter.jobId)} finished: " + + log"${MDC(LogKeys.CALL_SITE_SHORT_FORM, callSite.shortForm)}, took " + + log"${MDC(LogKeys.TIME, (System.nanoTime - start) / 1e6)} ms") case scala.util.Failure(exception) => - logInfo("Job %d failed: %s, took %f s".format - (waiter.jobId, callSite.shortForm, (System.nanoTime - start) / 1e9)) + logInfo(log"Job ${MDC(LogKeys.JOB_ID, waiter.jobId)} failed: " + + log"${MDC(LogKeys.CALL_SITE_SHORT_FORM, callSite.shortForm)}, took " + + log"${MDC(LogKeys.TIME, (System.nanoTime - start) / 1e6)} ms") // SPARK-8644: Include user stack trace in exceptions coming from DAGScheduler. val callerStackTrace = Thread.currentThread().getStackTrace.tail exception.setStackTrace(exception.getStackTrace ++ callerStackTrace) diff --git a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala index 02556528ce38b..ad9c3d3f10cfe 100644 --- a/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala +++ b/core/src/main/scala/org/apache/spark/util/ShutdownHookManager.scala @@ -110,7 +110,7 @@ private[spark] object ShutdownHookManager extends Logging { } } if (retval) { - logInfo("path = " + file + ", already present as root for deletion.") + logInfo(log"path = ${MDC(LogKeys.FILE_NAME, file)}, already present as root for deletion.") } retval } diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index fe488f9cf0daf..c3d648dccea73 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -18,7 +18,7 @@ package org.apache.spark.util.collection import org.apache.spark.SparkEnv -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config._ import org.apache.spark.memory.{MemoryConsumer, MemoryMode, TaskMemoryManager} @@ -143,8 +143,9 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) */ @inline private def logSpillage(size: Long): Unit = { val threadId = Thread.currentThread().getId - logInfo("Thread %d spilling in-memory map of %s to disk (%d time%s so far)" - .format(threadId, org.apache.spark.util.Utils.bytesToString(size), - _spillCount, if (_spillCount > 1) "s" else "")) + logInfo(log"Thread ${MDC(LogKeys.THREAD_ID, threadId)} " + + log"spilling in-memory map of ${MDC(LogKeys.BYTE_SIZE, + org.apache.spark.util.Utils.bytesToString(size))} to disk " + + log"(${MDC(LogKeys.SPILL_TIMES, _spillCount)} times so far)") } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index 85a7350078101..641b4fa4048a6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaSparkContext._ -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys.{CLUSTER_CENTROIDS, CLUSTER_LABEL, CLUSTER_WEIGHT, LARGEST_CLUSTER_INDEX, SMALLEST_CLUSTER_INDEX} import org.apache.spark.mllib.linalg.{BLAS, Vector, Vectors} import org.apache.spark.rdd.RDD @@ -222,7 +222,7 @@ class StreamingKMeans @Since("1.2.0") ( throw new IllegalArgumentException("Invalid time unit for decay: " + timeUnit) } this.decayFactor = math.exp(math.log(0.5) / halfLife) - logInfo("Setting decay factor to: %g ".format (this.decayFactor)) + logInfo(log"Setting decay factor to: ${MDC(LogKeys.VALUE, this.decayFactor)}") this.timeUnit = timeUnit this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index a288d13e57f7b..8da2b0e27fb87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -299,8 +299,8 @@ object GradientDescent extends Logging { i += 1 } - logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format( - stochasticLossHistory.takeRight(10).mkString(", "))) + logInfo(log"GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses " + + log"${MDC(LogKeys.LOSSES, stochasticLossHistory.takeRight(10).mkString(", "))}") (weights, stochasticLossHistory.toArray) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala index 4fc297560c088..28c997f5301c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/LBFGS.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import breeze.linalg.{DenseVector => BDV} import breeze.optimize.{CachedDiffFunction, DiffFunction, LBFGS => BreezeLBFGS} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy import org.apache.spark.rdd.RDD @@ -217,8 +217,8 @@ object LBFGS extends Logging { val lossHistoryArray = lossHistory.result() - logInfo("LBFGS.runLBFGS finished. Last 10 losses %s".format( - lossHistoryArray.takeRight(10).mkString(", "))) + logInfo(log"LBFGS.runLBFGS finished. Last 10 losses ${MDC(LogKeys.LOSSES, + lossHistoryArray.takeRight(10).mkString(", "))}") (weights, lossHistoryArray) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala index 11e077e891bd7..735814c9ae084 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala @@ -46,7 +46,7 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil.SOURCE_SPARK -import org.apache.spark.internal.{Logging, MDC} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.metrics.source.HiveCatalogMetrics import org.apache.spark.sql.catalyst.TableIdentifier @@ -906,11 +906,11 @@ private[hive] class HiveClientImpl( } catch { case e: Exception => logError( - s""" + log""" |====================== |HIVE FAILURE OUTPUT |====================== - |${outputBuffer.toString} + |${MDC(LogKeys.OUTPUT_BUFFER, outputBuffer.toString)} |====================== |END HIVE FAILURE OUTPUT |====================== diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 6fd6597d4f14c..128a5fded49a9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -117,7 +117,8 @@ class DStreamCheckpointData[T: ClassTag](dstream: DStream[T]) // Create RDDs from the checkpoint data currentCheckpointFiles.foreach { case(time, file) => - logInfo("Restoring checkpointed RDD for time " + time + " from file '" + file + "'") + logInfo(log"Restoring checkpointed RDD for time ${MDC(LogKeys.TIME, time)} from file " + + log"'${MDC(LogKeys.FILE_NAME, file)}'") dstream.generatedRDDs += ((time, dstream.context.sparkContext.checkpointFile[T](file))) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index b39cd43ce1a66..b067c505da0dd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -147,7 +147,8 @@ class FileInputDStream[K, V, F <: NewInputFormat[K, V]]( override def compute(validTime: Time): Option[RDD[(K, V)]] = { // Find new files val newFiles = findNewFiles(validTime.milliseconds) - logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) + logInfo(log"New files at time ${MDC(LogKeys.BATCH_TIMESTAMP, validTime)}:\n" + + log"${MDC(LogKeys.FILE_NAME, newFiles.mkString("\n"))}") batchTimeToSelectedFiles.synchronized { batchTimeToSelectedFiles += ((validTime, newFiles)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index efda751c1e43e..de7882032122a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -87,7 +87,7 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) val dataBuffer = ByteBuffer.allocate(length) readFully(channel, dataBuffer) dataBuffer.flip() - logInfo("Read a block with " + length + " bytes") + logInfo(log"Read a block with ${MDC(LogKeys.BYTE_SIZE, length)} bytes") queue.put(dataBuffer) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 91d15e7956a65..1dde435a913c8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -221,8 +221,8 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val checkpointTime = ssc.initialCheckpoint.checkpointTime val restartTime = new Time(timer.getRestartTime(graph.zeroTime.milliseconds)) val downTimes = checkpointTime.until(restartTime, batchDuration) - logInfo("Batches during down time (" + downTimes.size + " batches): " - + downTimes.mkString(", ")) + logInfo(log"Batches during down time (${MDC(LogKeys.NUM_BATCHES, downTimes.size)} batches): " + + log"${MDC(LogKeys.BATCH_TIMES, downTimes.mkString(","))}") // Batches that were unprocessed before failure val pendingTimes = ssc.initialCheckpoint.pendingTimes.sorted(Time.ordering) From 47f85be055d34475c8155eb095b6e87176961a9f Mon Sep 17 00:00:00 2001 From: Yuchen Liu Date: Fri, 12 Jul 2024 07:36:17 +0900 Subject: [PATCH 37/64] [SPARK-48850][DOCS][SS][SQL] Add documentation for new options added to State Data Source ### What changes were proposed in this pull request? In https://github.com/apache/spark/pull/46944 and https://github.com/apache/spark/pull/47188, we introduced some new options to the State Data Source. This PR aims to explain these new features in the documentation. ### Why are the changes needed? It is necessary to reflect the latest change in the documentation website. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The API Doc website can be rendered correctly. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47274 from eason-yuchen-liu/snapshot-doc. Authored-by: Yuchen Liu Signed-off-by: Jungtaek Lim --- .../structured-streaming-state-data-source.md | 115 +++++++++++++++++- 1 file changed, 112 insertions(+), 3 deletions(-) diff --git a/docs/structured-streaming-state-data-source.md b/docs/structured-streaming-state-data-source.md index 9866991306690..1fac39566c1da 100644 --- a/docs/structured-streaming-state-data-source.md +++ b/docs/structured-streaming-state-data-source.md @@ -42,7 +42,7 @@ Users can read an instance of state store, which is matched to a single stateful Note that there could be an exception, e.g. stream-stream join, which leverages multiple state store instances internally. The data source abstracts the internal representation away from users and provides a user-friendly approach to read the state. See the section for stream-stream join for more details. -### Creating a State store for Batch Queries (all defaults) +### Creating a state store for batch queries (all defaults)
@@ -144,9 +144,40 @@ The following configurations are optional:
(none) Represents the target side to read from. This option is used when users want to read the state from stream-stream join.
snapshotStartBatchIdnumeric valueIf specified, force to read the snapshot at this batch ID, then changelogs will be replayed until 'batchId' or its default. Note that snapshot batch ID starts with 0 and equals to snapshot version ID minus 1. This option must be used together with 'snapshotPartitionId'.
snapshotPartitionIdnumeric valueIf specified, only this specific partition will be read. Note that partition ID starts with 0. This option must be used together with 'snapshotStartBatchId'.
readChangeFeedbooleanfalseIf set to true, will read the change of state over microbatches. The output table schema will also differ. Details can be found in section "Reading state changes over microbatches". Option 'changeStartBatchId' must be specified with this option. Option 'batchId', 'joinSide', 'snapshotStartBatchId' and 'snapshotPartitionId' cannot be used together with this option.
changeStartBatchIdnumeric valueRepresents the first batch to read in the read change feed mode. This option requires 'readChangeFeed' to be set to true.
changeEndBatchIdnumeric valuelatest commited batchIdRepresents the last batch to read in the read change feed mode. This option requires 'readChangeFeed' to be set to true.
-### Reading state for Stream-stream join + +### Reading state for stream-stream join Structured Streaming implements the stream-stream join feature via leveraging multiple instances of state store internally. These instances logically compose buffers to store the input rows for left and right. @@ -154,6 +185,85 @@ These instances logically compose buffers to store the input rows for left and r Since it is more obvious to users to reason about, the data source provides the option 'joinSide' to read the buffered input for specific side of the join. To enable the functionality to read the internal state store instance directly, we also allow specifying the option 'storeName', with restriction that 'storeName' and 'joinSide' cannot be specified together. +### Reading state changes over microbatches + +If we want to understand the change of state store over microbatches instead of the whole state store at a particular microbatch, 'readChangeFeed' is the option to use. +For example, this is the code to read the change of state from batch 2 to the latest committed batch. + +
+ +
+{% highlight python %} + +df = spark \ +.read \ +.format("statestore") \ +.option("readChangeFeed", true) \ +.option("changeStartBatchId", 2) \ +.load("") + +{% endhighlight %} +
+ +
+{% highlight scala %} + +val df = spark +.read +.format("statestore") +.option("readChangeFeed", true) +.option("changeStartBatchId", 2) +.load("") + +{% endhighlight %} +
+ +
+{% highlight java %} + +Dataset df = spark +.read() +.format("statestore") +.option("readChangeFeed", true) +.option("changeStartBatchId", 2) +.load(""); + +{% endhighlight %} +
+ +
+ +The output schema will also be different from the normal output. + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
ColumnTypeNote
batch_idlong
change_typestringThere are two possible values: 'update' and 'delete'. Update represents either inserting a non-existing key-value pair or updating an existing key with new value. The 'value' field will be null for delete records.
keystruct (depends on the type for state key)
valuestruct (depends on the type for state value)
partition_idint
+ ## State metadata source Before querying the state from existing checkpoint via state data source, users would like to understand the information for the checkpoint, especially about state operator. This includes which operators and state store instances are available in the checkpoint, available range of batch IDs, etc. @@ -179,7 +289,6 @@ df = spark \
{% highlight scala %} - val df = spark .read .format("state-metadata") From 52958d9c34d9e7b1bbb23958e29258c6bb1a9322 Mon Sep 17 00:00:00 2001 From: Wei Liu Date: Fri, 12 Jul 2024 08:40:08 +0900 Subject: [PATCH 38/64] [SPARK-48717][FOLLOWUP][PYTHON][SS] Catch Cancelled Job Group wrapped by Py4JJavaError in StreamExecution ### What changes were proposed in this pull request? The previous commit https://github.com/apache/spark/commit/1581264a5d77e1731a987020a448ddc3f3ea8f27 doesn't capture the situation when a job group is cancelled. This patches that situation. ### Why are the changes needed? Bug fix, without this change, calling query.stop() would sometimes (when there is a python foreachBatch function, and this error is thrown) cause query appears as failed. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Added unit test. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47307 from WweiL/SPARK-48717-job-cancel. Authored-by: Wei Liu Signed-off-by: Hyukjin Kwon --- .../execution/streaming/StreamExecution.scala | 65 ++++++++++--------- .../spark/sql/streaming/StreamSuite.scala | 27 ++++++++ 2 files changed, 62 insertions(+), 30 deletions(-) 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 81f7acdb755bc..178451baf265e 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 @@ -692,40 +692,45 @@ object StreamExecution { s"((.|\\r\\n|\\r|\\n)*)(${IO_EXCEPTION_NAMES.mkString("|")})").r @scala.annotation.tailrec - def isInterruptionException(e: Throwable, sc: SparkContext): Boolean = e match { - // InterruptedIOException - thrown when an I/O operation is interrupted - // ClosedByInterruptException - thrown when an I/O operation upon a channel is interrupted - case _: InterruptedException | _: InterruptedIOException | _: ClosedByInterruptException => - true - // The cause of the following exceptions may be one of the above exceptions: - // - // UncheckedIOException - thrown by codes that cannot throw a checked IOException, such as - // BiFunction.apply - // ExecutionException - thrown by codes running in a thread pool and these codes throw an - // exception - // UncheckedExecutionException - thrown by codes that cannot throw a checked - // ExecutionException, such as BiFunction.apply - case e2 @ (_: UncheckedIOException | _: ExecutionException | _: UncheckedExecutionException) - if e2.getCause != null => - isInterruptionException(e2.getCause, sc) - case fe: ForeachBatchUserFuncException => isInterruptionException(fe.getCause, sc) - case se: SparkException => + def isInterruptionException(e: Throwable, sc: SparkContext): Boolean = { + def isCancelledJobGroup(errorMsg: String): Boolean = { val jobGroup = sc.getLocalProperty("spark.jobGroup.id") if (jobGroup == null) return false - val errorMsg = se.getMessage - if (errorMsg.contains("cancelled") && errorMsg.contains(jobGroup) && se.getCause == null) { + errorMsg.contains("cancelled") && errorMsg.contains(jobGroup) + } + + e match { + // InterruptedIOException - thrown when an I/O operation is interrupted + // ClosedByInterruptException - thrown when an I/O operation upon a channel is interrupted + case _: InterruptedException | _: InterruptedIOException | _: ClosedByInterruptException => true - } else if (se.getCause != null) { - isInterruptionException(se.getCause, sc) - } else { + // The cause of the following exceptions may be one of the above exceptions: + // + // UncheckedIOException - thrown by codes that cannot throw a checked IOException, such as + // BiFunction.apply + // ExecutionException - thrown by codes running in a thread pool and these codes throw an + // exception + // UncheckedExecutionException - thrown by codes that cannot throw a checked + // ExecutionException, such as BiFunction.apply + case e2 @ (_: UncheckedIOException | _: ExecutionException | _: UncheckedExecutionException) + if e2.getCause != null => + isInterruptionException(e2.getCause, sc) + case fe: ForeachBatchUserFuncException => isInterruptionException(fe.getCause, sc) + case se: SparkException => + if (se.getCause == null) { + isCancelledJobGroup(se.getMessage) + } else { + isInterruptionException(se.getCause, sc) + } + // py4j.Py4JException - with pinned thread mode on, the exception can be interrupted by Py4J + // access, for example, in `DataFrameWriter.foreachBatch`. See also + // SPARK-39218. + case e: py4j.Py4JException => + PROXY_ERROR.findFirstIn(e.getMessage).isDefined || (e.getMessage + .contains("org.apache.spark.SparkException") && isCancelledJobGroup(e.getMessage)) + case _ => false - } - // py4j.Py4JException - with pinned thread mode on, the exception can be interrupted by Py4J - // access, for example, in `DataFrameWriter.foreachBatch`. See also - // SPARK-39218. - case e: py4j.Py4JException => PROXY_ERROR.findFirstIn(e.getMessage).isDefined - case _ => - false + } } /** Whether the path contains special chars that will be escaped when converting to a `URI`. */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index b4f29fa9f01ac..9edd1acaddbc1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -1355,6 +1355,33 @@ class StreamSuite extends StreamTest { // scalastyle:on line.size.limit val febError2 = ForeachBatchUserFuncException(e2) assert(StreamExecution.isInterruptionException(febError2, spark.sparkContext)) + + // scalastyle:off line.size.limit + val e3 = new py4j.Py4JException( + """ + py4j.protocol.Py4JJavaError: An error occurred while calling o6032.save. + |: org.apache.spark.SparkException: [SPARK_JOB_CANCELLED] Job 89 cancelled part of cancelled job group a5bc7e26-f199-463e-95ad-4ec960a62d79 SQLSTATE: XXKDA + |at org.apache.spark.scheduler.DAGScheduler.handleJobCancellation(DAGScheduler.scala:3747) + |at org.apache.spark.scheduler.DAGScheduler.$anonfun$handleJobGroupCancelled$4(DAGScheduler.scala:1592) + |at scala.runtime.java8.JFunction1$mcVI$sp.apply(JFunction1$mcVI$sp.java:23) + |at scala.collection.mutable.HashSet.foreach(HashSet.scala:79) + |at org.apache.spark.scheduler.DAGScheduler.handleJobGroupCancelled(DAGScheduler.scala:1591) + |at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.doOnReceive(DAGScheduler.scala:4102) + |at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:4058) + |at org.apache.spark.scheduler.DAGSchedulerEventProcessLoop.onReceive(DAGScheduler.scala:4046) + |at org.apache.spark.util.EventLoop$$anon$1.run(EventLoop.scala:54) + |at org.apache.spark.scheduler.DAGScheduler.$anonfun$runJob$1(DAGScheduler.scala:1348) + |""".stripMargin) + // scalastyle:on line.size.limit + val febError3 = ForeachBatchUserFuncException(e3) + val prevJobId = spark.sparkContext.getLocalProperty("spark.jobGroup.id") + try { + spark.sparkContext.setLocalProperty( + "spark.jobGroup.id", "a5bc7e26-f199-463e-95ad-4ec960a62d79") + assert(StreamExecution.isInterruptionException(febError3, spark.sparkContext)) + } finally { + spark.sparkContext.setLocalProperty("spark.jobGroup.id", prevJobId) + } } } From 06bebb884961f8b164e218af0ab2e8cb7517a66a Mon Sep 17 00:00:00 2001 From: Biruk Tesfaye Date: Fri, 12 Jul 2024 08:55:51 +0900 Subject: [PATCH 39/64] [SPARK-48852][CONNECT] Fix string trim function in connect ### What changes were proposed in this pull request? Changed the order of arguments passed in the connect client's trim function call to match [`sql/core/src/main/scala/org/apache/spark/sql/functions.scala`](https://github.com/apache/spark/blob/f2dd0b3338a6937bbfbea6cd5fffb2bf9992a1f3/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L4322) ### Why are the changes needed? This change fixes a correctness bug in spark connect where a query to trim characters `s` from a column will be replaced by a substring of `s`. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Updated golden files for [`/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala`](https://github.com/apache/spark/blob/f2dd0b3338a6937bbfbea6cd5fffb2bf9992a1f3/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/PlanGenerationTestSuite.scala#L1815) and added an additional test to verify correctness. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47277 from biruktesf-db/fix-trim-connect. Authored-by: Biruk Tesfaye Signed-off-by: Hyukjin Kwon --- .../function_ltrim_with_pattern.explain | 2 +- .../function_rtrim_with_pattern.explain | 2 +- .../function_trim_with_pattern.explain | 2 +- .../queries/function_ltrim_with_pattern.json | 8 ++++---- .../function_ltrim_with_pattern.proto.bin | Bin 183 -> 183 bytes .../queries/function_rtrim_with_pattern.json | 8 ++++---- .../function_rtrim_with_pattern.proto.bin | Bin 183 -> 183 bytes .../queries/function_trim_with_pattern.json | 8 ++++---- .../function_trim_with_pattern.proto.bin | Bin 182 -> 182 bytes .../scala/org/apache/spark/sql/functions.scala | 6 +++--- .../apache/spark/sql/ClientE2ETestSuite.scala | 8 ++++++++ 11 files changed, 26 insertions(+), 18 deletions(-) diff --git a/connect/common/src/test/resources/query-tests/explain-results/function_ltrim_with_pattern.explain b/connect/common/src/test/resources/query-tests/explain-results/function_ltrim_with_pattern.explain index 68c20c2b8660c..13fcc399cd996 100644 --- a/connect/common/src/test/resources/query-tests/explain-results/function_ltrim_with_pattern.explain +++ b/connect/common/src/test/resources/query-tests/explain-results/function_ltrim_with_pattern.explain @@ -1,2 +1,2 @@ -Project [ltrim(xxx, Some(g#0)) AS TRIM(LEADING g FROM xxx)#0] +Project [ltrim(g#0, Some(xxx)) AS TRIM(LEADING xxx FROM g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connect/common/src/test/resources/query-tests/explain-results/function_rtrim_with_pattern.explain b/connect/common/src/test/resources/query-tests/explain-results/function_rtrim_with_pattern.explain index ea262f52de416..9366a62e82fbc 100644 --- a/connect/common/src/test/resources/query-tests/explain-results/function_rtrim_with_pattern.explain +++ b/connect/common/src/test/resources/query-tests/explain-results/function_rtrim_with_pattern.explain @@ -1,2 +1,2 @@ -Project [rtrim(yyy, Some(g#0)) AS TRIM(TRAILING g FROM yyy)#0] +Project [rtrim(g#0, Some(yyy)) AS TRIM(TRAILING yyy FROM g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connect/common/src/test/resources/query-tests/explain-results/function_trim_with_pattern.explain b/connect/common/src/test/resources/query-tests/explain-results/function_trim_with_pattern.explain index 90a5607114fe7..2969602494311 100644 --- a/connect/common/src/test/resources/query-tests/explain-results/function_trim_with_pattern.explain +++ b/connect/common/src/test/resources/query-tests/explain-results/function_trim_with_pattern.explain @@ -1,2 +1,2 @@ -Project [trim(---, Some(g#0)) AS TRIM(BOTH g FROM ---)#0] +Project [trim(g#0, Some(---)) AS TRIM(BOTH --- FROM g)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json b/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json index 3c4825792dc3c..01dc81bdae7bc 100644 --- a/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json +++ b/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json @@ -15,13 +15,13 @@ "unresolvedFunction": { "functionName": "ltrim", "arguments": [{ - "unresolvedAttribute": { - "unparsedIdentifier": "g" - } - }, { "literal": { "string": "xxx" } + }, { + "unresolvedAttribute": { + "unparsedIdentifier": "g" + } }] } }] diff --git a/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin b/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin index 13455d7091e9f5c969b60ae7843498f4d0dae6d6..0cd62226c9716a638e99894e1bf426e1adc5f812 100644 GIT binary patch delta 22 dcmdnaxSesr3VwDj)-2|ViV7iCA!aVdbO1>F1wjA+ delta 22 dcmdnaxSesr3Vv20W-i8bA$Bg-Ear-e3IIT#1wjA+ diff --git a/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json b/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json index d4c3c0ca68eb2..0ac2401f9eacf 100644 --- a/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json +++ b/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json @@ -15,13 +15,13 @@ "unresolvedFunction": { "functionName": "rtrim", "arguments": [{ - "unresolvedAttribute": { - "unparsedIdentifier": "g" - } - }, { "literal": { "string": "yyy" } + }, { + "unresolvedAttribute": { + "unparsedIdentifier": "g" + } }] } }] diff --git a/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin b/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin index 37f4782f46161bb5199895c6c796d4a41968f438..1332f5b330000a3ba6184cdf6fdab2668864d855 100644 GIT binary patch delta 22 dcmdnaxSesr3VwDj)-2}A%1R+tA!aVdbO1>g1w;S< delta 22 dcmdnaxSesr3Vv20W-i8bA$Bg-Eau9}N&rBm1w;S< diff --git a/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json b/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json index 82b1616ef38ed..fc3281c921531 100644 --- a/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json +++ b/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json @@ -15,13 +15,13 @@ "unresolvedFunction": { "functionName": "trim", "arguments": [{ - "unresolvedAttribute": { - "unparsedIdentifier": "g" - } - }, { "literal": { "string": "---" } + }, { + "unresolvedAttribute": { + "unparsedIdentifier": "g" + } }] } }] diff --git a/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin b/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin index 6a86e87c9850b94b240ac9b55e5168c8a98e00fa..2136b55656212e31b809e7cade085e79303bd585 100644 GIT binary patch delta 22 dcmdnSxQ%hba(;F$)+}aSU0oqoA!aVdbO1ou1YH0C delta 22 dcmdnSxQ%hba(-4JW-i8bA$Bg-EM{F@T>w6!1YH0C diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index eae239a25589c..92e7bc9da5904 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -4002,7 +4002,7 @@ object functions { * @group string_funcs * @since 3.4.0 */ - def ltrim(e: Column, trimString: String): Column = Column.fn("ltrim", e, lit(trimString)) + def ltrim(e: Column, trimString: String): Column = Column.fn("ltrim", lit(trimString), e) /** * Calculates the byte length for the specified string column. @@ -4183,7 +4183,7 @@ object functions { * @group string_funcs * @since 3.4.0 */ - def rtrim(e: Column, trimString: String): Column = Column.fn("rtrim", e, lit(trimString)) + def rtrim(e: Column, trimString: String): Column = Column.fn("rtrim", lit(trimString), e) /** * Returns the soundex code for the specified expression. @@ -4360,7 +4360,7 @@ object functions { * @group string_funcs * @since 3.4.0 */ - def trim(e: Column, trimString: String): Column = Column.fn("trim", e, lit(trimString)) + def trim(e: Column, trimString: String): Column = Column.fn("trim", lit(trimString), e) /** * Converts a string column to upper case. diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala index 95410ab6a7e7d..a66f790e08740 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala @@ -1555,6 +1555,14 @@ class ClientE2ETestSuite val metrics = SparkThreadUtils.awaitResult(future, 2.seconds) assert(metrics === Map("min(id)" -> 0, "avg(id)" -> 49, "max(id)" -> 98)) } + + test("SPARK-48852: trim function on a string column returns correct results") { + val session: SparkSession = spark + import session.implicits._ + val df = Seq(" a ", "b ", " c").toDF("col") + val result = df.select(trim(col("col"), " ").as("trimmed_col")).collect() + assert(result sameElements Array(Row("a"), Row("b"), Row("c"))) + } } private[sql] case class ClassData(a: String, b: Int) From c1a8966e3091b050b060016ce8605b3020ebc368 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Fri, 12 Jul 2024 09:42:07 +0900 Subject: [PATCH 40/64] [SPARK-48872][PYTHON] Reduce the overhead of _capture_call_site ### What changes were proposed in this pull request? Reduces the overhead of `inspect.stack` in `_capture_call_site` by inlining `inspect.stack` with using `generator` instead of `list`. Also, specify `context=0` for `inspect.getframeinfo` to avoid unnecessary field retrievals. ### Why are the changes needed? The `_capture_call_site` has inevitable overhead when `Column` operations happen a lot, but it can be reduced. E.g., ```py from functools import reduce def alias_schema(): return df.select(reduce(lambda x, y: x.alias(f"col_a_{y}"), range(20), F.col("a"))).schema profile(alias_schema) ``` Screenshot 2024-07-11 at 15 24 31 The function calls and duration are: - before ``` 18013240 function calls (18012760 primitive calls) in 2.327 seconds ncalls tottime percall cumtime percall filename:lineno(function) ... 200 0.001 0.000 2.231 0.011 /.../python/pyspark/errors/utils.py:164(_capture_call_site) ``` - after ``` 1421240 function calls (1420760 primitive calls) in 0.265 seconds ncalls tottime percall cumtime percall filename:lineno(function) ... 200 0.001 0.000 0.182 0.001 /.../python/pyspark/errors/utils.py:165(_capture_call_site) ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? The existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47308 from ueshin/issues/SPARK-48872/inspect_stack. Authored-by: Takuya Ueshin Signed-off-by: Hyukjin Kwon --- python/pyspark/errors/utils.py | 23 ++++++++++++++++------- 1 file changed, 16 insertions(+), 7 deletions(-) diff --git a/python/pyspark/errors/utils.py b/python/pyspark/errors/utils.py index 703978e9573cd..a9889da2b6103 100644 --- a/python/pyspark/errors/utils.py +++ b/python/pyspark/errors/utils.py @@ -14,16 +14,17 @@ # See the License for the specific language governing permissions and # limitations under the License. # - import re import functools import inspect +import itertools import os import threading from typing import ( Any, Callable, Dict, + Iterator, List, Match, TypeVar, @@ -179,11 +180,19 @@ def _capture_call_site(spark_session: "SparkSession", depth: int) -> str: """ # Filtering out PySpark code and keeping user code only pyspark_root = os.path.dirname(pyspark.__file__) - stack = [ - frame_info for frame_info in inspect.stack() if pyspark_root not in frame_info.filename - ] - selected_frames = stack[:depth] + def inspect_stack() -> Iterator[inspect.FrameInfo]: + frame = inspect.currentframe() + while frame: + frameinfo = (frame,) + inspect.getframeinfo(frame, context=0) + yield inspect.FrameInfo(*frameinfo) + frame = frame.f_back + + stack = ( + frame_info for frame_info in inspect_stack() if pyspark_root not in frame_info.filename + ) + + selected_frames: Iterator[inspect.FrameInfo] = itertools.islice(stack, depth) # We try import here since IPython is not a required dependency try: @@ -196,11 +205,11 @@ def _capture_call_site(spark_session: "SparkSession", depth: int) -> str: # Filtering out IPython related frames ipy_root = os.path.dirname(IPython.__file__) ipykernel_root = os.path.dirname(ipykernel.__file__) - selected_frames = [ + selected_frames = ( frame for frame in selected_frames if (ipy_root not in frame.filename) and (ipykernel_root not in frame.filename) - ] + ) except ImportError: ipython = None From ef71600e137886b9996ddd22b57a02e0c3955a22 Mon Sep 17 00:00:00 2001 From: Andy Lam Date: Fri, 12 Jul 2024 10:08:04 +0800 Subject: [PATCH 41/64] [SPARK-46743][SQL][FOLLOW UP] Count bug after ScalarSubqery is folded if it has an empty relation ### What changes were proposed in this pull request? In this PR https://github.com/apache/spark/pull/45125, we handled the case where an Aggregate is folded into a Project, causing a count bug. We missed cases where: 1. The entire ScalarSubquery's plan is regarded as empty relation, and is folded completely. 2. There are operations above the Aggregate in the subquery (such as filter and project). ### Why are the changes needed? This PR fixes that by adding the case handling in ConstantFolding and OptimizeSubqueries. ### Does this PR introduce _any_ user-facing change? Yes. There was a correctness error which happens when the scalar subquery is count-bug-susceptible, and empty, and thus folded by `ConstantFolding`. ### How was this patch tested? Added SQL query tests in `scalar-subquery-count-bug.sql`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47290 from andylam-db/decorr-bugs. Authored-by: Andy Lam Signed-off-by: Wenchen Fan --- .../querytest/GeneratedSubquerySuite.scala | 17 +- .../sql/catalyst/optimizer/Optimizer.scala | 22 ++- .../sql/catalyst/optimizer/expressions.scala | 6 + .../scalar-subquery-count-bug.sql.out | 160 ++++++++++++++++++ .../scalar-subquery-count-bug.sql | 67 ++++++++ .../scalar-subquery-count-bug.sql.out | 116 +++++++++++++ 6 files changed, 367 insertions(+), 21 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala index 8b27e9cb0e0a3..015898b70b7c5 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/querytest/GeneratedSubquerySuite.scala @@ -445,20 +445,5 @@ object GeneratedSubquerySuite { // Limit number of generated queries per test so that tests will not take too long. private val NUM_QUERIES_PER_TEST = 1000 - // scalastyle:off line.size.limit - private val KNOWN_QUERIES_WITH_DIFFERENT_RESULTS = Seq( - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(null_table.a) AS aggFunctionAlias FROM null_table WHERE null_table.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;", - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(null_table.a) AS aggFunctionAlias FROM null_table INNER JOIN join_table ON null_table.a = join_table.a WHERE null_table.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;", - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(null_table.a) AS aggFunctionAlias FROM null_table LEFT OUTER JOIN join_table ON null_table.a = join_table.a WHERE null_table.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;", - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(null_table.a) AS aggFunctionAlias FROM null_table RIGHT OUTER JOIN join_table ON null_table.a = join_table.a WHERE null_table.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;", - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(innerSubqueryAlias.a) AS aggFunctionAlias FROM (SELECT null_table.a, null_table.b FROM null_table INTERSECT SELECT join_table.a, join_table.b FROM join_table) AS innerSubqueryAlias WHERE innerSubqueryAlias.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;", - // SPARK-46743 - "SELECT outer_table.a, outer_table.b, (SELECT COUNT(innerSubqueryAlias.a) AS aggFunctionAlias FROM (SELECT null_table.a, null_table.b FROM null_table EXCEPT SELECT join_table.a, join_table.b FROM join_table) AS innerSubqueryAlias WHERE innerSubqueryAlias.a = outer_table.a) AS subqueryAlias FROM outer_table ORDER BY a DESC NULLS FIRST, b DESC NULLS FIRST, subqueryAlias DESC NULLS FIRST;" - ) - // scalastyle:on line.size.limit + private val KNOWN_QUERIES_WITH_DIFFERENT_RESULTS = Seq() } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 95923a1419513..647812ff80e78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -332,7 +333,9 @@ abstract class Optimizer(catalogManager: CatalogManager) // Do not optimize DPP subquery, as it was created from optimized plan and we should not // optimize it again, to save optimization time and avoid breaking broadcast/subquery reuse. case d: DynamicPruningSubquery => d - case s @ ScalarSubquery(a @ Aggregate(group, _, child), _, _, _, _, mayHaveCountBug) + case s @ ScalarSubquery( + PhysicalOperation(projections, predicates, a @ Aggregate(group, _, child)), + _, _, _, _, mayHaveCountBug) if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && mayHaveCountBug.nonEmpty && mayHaveCountBug.get => // This is a subquery with an aggregate that may suffer from a COUNT bug. @@ -357,19 +360,28 @@ abstract class Optimizer(catalogManager: CatalogManager) val needProject = projectOverAggregateChild.output.zip(optimizedInput.output).exists { case (oldAttr, newAttr) => oldAttr.exprId != newAttr.exprId } - if (needProject) { + val optimizedAgg = if (needProject) { val updatedProjectList = projectOverAggregateChild.output.zip(optimizedInput.output).map { case (oldAttr, newAttr) => Alias(newAttr, newAttr.name)(exprId = oldAttr.exprId) } - s.withNewPlan(a.withNewChildren(Seq(Project(updatedProjectList, optimizedInput)))) + a.withNewChildren(Seq(Project(updatedProjectList, optimizedInput))) } else { // Remove the top-level project if it is trivial. We do it to minimize plan changes. optimizedInput match { case Project(projectList, input) if projectList.forall(_.isInstanceOf[Attribute]) => - s.withNewPlan(a.withNewChildren(Seq(input))) - case _ => s.withNewPlan(a.withNewChildren(Seq(optimizedInput))) + a.withNewChildren(Seq(input)) + case _ => a.withNewChildren(Seq(optimizedInput)) } } + val newPlan = Project(projections, + if (predicates.nonEmpty) Filter(predicates.reduce(And), optimizedAgg) else optimizedAgg + ) + val needTopLevelProject = newPlan.output.zip(optimizedAgg.output).exists { + case (oldAttr, newAttr) => oldAttr.exprId != newAttr.exprId + } + s.withNewPlan( + if (needTopLevelProject) newPlan else newPlan.child + ) case s: SubqueryExpression => val Subquery(newPlan, _) = Optimizer.this.execute(Subquery.fromExpression(s)) // At this point we have an optimized subquery plan that we are going to attach 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 1750a0e275732..d0ee9f2d110d5 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 @@ -89,6 +89,12 @@ object ConstantFolding extends Rule[LogicalPlan] { e } + // Don't replace ScalarSubquery if its plan is an aggregate that may suffer from a COUNT bug. + case s @ ScalarSubquery(_, _, _, _, _, mayHaveCountBug) + if conf.getConf(SQLConf.DECORRELATE_SUBQUERY_PREVENT_CONSTANT_FOLDING_FOR_COUNT_BUG) && + mayHaveCountBug.nonEmpty && mayHaveCountBug.get => + s + // Replace ScalarSubquery with null if its maxRows is 0 case s: ScalarSubquery if s.plan.maxRows.contains(0) => Literal(null, s.dataType) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out index ebe071c5261fd..a668e96e76295 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out @@ -233,6 +233,166 @@ Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] +- LocalRelation [col1#x, col2#x] +-- !query +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + ) +FROM + l +-- !query analysis +Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] +: +- Aggregate [count(a#x) AS result#xL] +: +- Filter (a#x = outer(a#x)) +: +- Join Inner, ((c#x = a#x) AND isnotnull(c#x)) +: :- SubqueryAlias null_view +: : +- View (`null_view`, [a#x, b#x]) +: : +- Project [cast(CAST(NULL AS INT)#x as int) AS a#x, cast(CAST(NULL AS INT)#x as int) AS b#x] +: : +- Project [cast(null as int) AS CAST(NULL AS INT)#x, cast(null as int) AS CAST(NULL AS INT)#x] +: : +- OneRowRelation +: +- SubqueryAlias r +: +- View (`r`, [c#x, d#x]) +: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias l + +- View (`l`, [a#x, b#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT +( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query analysis +Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] +: +- Project [result#xL] +: +- Filter (count(1)#xL > cast(-1 as bigint)) +: +- Aggregate [count(a#x) AS result#xL, count(1) AS count(1)#xL] +: +- Filter (a#x = outer(a#x)) +: +- Join Inner, ((c#x = a#x) AND isnotnull(c#x)) +: :- SubqueryAlias null_view +: : +- View (`null_view`, [a#x, b#x]) +: : +- Project [cast(CAST(NULL AS INT)#x as int) AS a#x, cast(CAST(NULL AS INT)#x as int) AS b#x] +: : +- Project [cast(null as int) AS CAST(NULL AS INT)#x, cast(null as int) AS CAST(NULL AS INT)#x] +: : +- OneRowRelation +: +- SubqueryAlias r +: +- View (`r`, [c#x, d#x]) +: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias l + +- View (`l`, [a#x, b#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT + ( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + ) +FROM + l +-- !query analysis +Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] +: +- Aggregate [count(a#x) AS result#xL] +: +- Filter (a#x = outer(a#x)) +: +- SubqueryAlias f +: +- Intersect false +: :- Project [a#x, cast(b#x as decimal(11,1)) AS b#x] +: : +- Project [a#x, b#x] +: : +- SubqueryAlias null_view +: : +- View (`null_view`, [a#x, b#x]) +: : +- Project [cast(CAST(NULL AS INT)#x as int) AS a#x, cast(CAST(NULL AS INT)#x as int) AS b#x] +: : +- Project [cast(null as int) AS CAST(NULL AS INT)#x, cast(null as int) AS CAST(NULL AS INT)#x] +: : +- OneRowRelation +: +- Project [c#x, cast(d#x as decimal(11,1)) AS d#x] +: +- Project [c#x, d#x] +: +- Filter isnotnull(c#x) +: +- SubqueryAlias r +: +- View (`r`, [c#x, d#x]) +: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias l + +- View (`l`, [a#x, b#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x] + +- LocalRelation [col1#x, col2#x] + + +-- !query +SELECT +( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query analysis +Project [scalar-subquery#x [a#x] AS scalarsubquery(a)#xL] +: +- Project [result#xL] +: +- Filter (count(1)#xL > cast(-1 as bigint)) +: +- Aggregate [count(a#x) AS result#xL, count(1) AS count(1)#xL] +: +- Filter (a#x = outer(a#x)) +: +- SubqueryAlias f +: +- Intersect false +: :- Project [a#x, cast(b#x as decimal(11,1)) AS b#x] +: : +- Project [a#x, b#x] +: : +- SubqueryAlias null_view +: : +- View (`null_view`, [a#x, b#x]) +: : +- Project [cast(CAST(NULL AS INT)#x as int) AS a#x, cast(CAST(NULL AS INT)#x as int) AS b#x] +: : +- Project [cast(null as int) AS CAST(NULL AS INT)#x, cast(null as int) AS CAST(NULL AS INT)#x] +: : +- OneRowRelation +: +- Project [c#x, cast(d#x as decimal(11,1)) AS d#x] +: +- Project [c#x, d#x] +: +- Filter isnotnull(c#x) +: +- SubqueryAlias r +: +- View (`r`, [c#x, d#x]) +: +- Project [cast(col1#x as int) AS c#x, cast(col2#x as decimal(2,1)) AS d#x] +: +- LocalRelation [col1#x, col2#x] ++- SubqueryAlias l + +- View (`l`, [a#x, b#x]) + +- Project [cast(col1#x as int) AS a#x, cast(col2#x as decimal(2,1)) AS b#x] + +- LocalRelation [col1#x, col2#x] + + -- !query set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true -- !query analysis diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql index 8b37256a37aa4..b2e4bea480da1 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/scalar-subquery/scalar-subquery-count-bug.sql @@ -82,6 +82,73 @@ SELECT FROM l; +-- Count bug over a subquery with an empty relation after optimization. +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + ) +FROM + l; + +-- Same as above but with a filter (HAVING) above the aggregate +SELECT +( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l; + +-- Same as above but with intersect +SELECT + ( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + ) +FROM + l; + +SELECT +( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l; + set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true; diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out index 1f65fa8403121..aa295dfaca24c 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/scalar-subquery/scalar-subquery-count-bug.sql.out @@ -205,6 +205,122 @@ struct 0 +-- !query +SELECT + ( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT +( + SELECT + COUNT(null_view.a) AS result + FROM + null_view + INNER JOIN r + ON r.c = null_view.a + AND r.c IS NOT NULL + WHERE + null_view.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT + ( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + ) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + +-- !query +SELECT +( + SELECT + COUNT(f.a) AS result + FROM + ( + SELECT a, b FROM null_view + INTERSECT + SELECT c, d FROM r WHERE c IS NOT NULL + ) AS f + WHERE + f.a = l.a + HAVING COUNT(*) > -1 +) +FROM + l +-- !query schema +struct +-- !query output +0 +0 +0 +0 +0 +0 +0 +0 + + -- !query set spark.sql.optimizer.decorrelateSubqueryLegacyIncorrectCountHandling.enabled = true -- !query schema From 73eab92d30bf78d39c80dd5af03b052fe9fc5211 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 12 Jul 2024 10:16:11 +0800 Subject: [PATCH 42/64] [SPARK-48841][SQL] Include `collationName` to `sql()` of `Collate` ### What changes were proposed in this pull request? In the PR, I propose to fix the `sql()` method of the `Collate` expression, and append the `collationName` clause. ### Why are the changes needed? To distinguish column names when the `collationName` argument is used by `collate`. Before the changes, columns might conflict like the example below, and that could confuse users: ``` sql("CREATE TEMP VIEW tbl as (SELECT collate('A', 'UTF8_BINARY'), collate('A', 'UTF8_LCASE'))") ``` - Before: ``` [COLUMN_ALREADY_EXISTS] The column `collate(a)` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 org.apache.spark.sql.AnalysisException: [COLUMN_ALREADY_EXISTS] The column `collate(a)` already exists. Choose another name or rename the existing column. SQLSTATE: 42711 at org.apache.spark.sql.errors.QueryCompilationErrors$.columnAlreadyExistsError(QueryCompilationErrors.scala:2595) at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:115) at org.apache.spark.sql.util.SchemaUtils$.checkColumnNameDuplication(SchemaUtils.scala:97) ``` - After: ``` describe extended tbl; +-----------------------+-------------------------+-------+ |col_name |data_type |comment| +-----------------------+-------------------------+-------+ |collate(A, UTF8_BINARY)|string |NULL | |collate(A, UTF8_LCASE) |string collate UTF8_LCASE|NULL | +-----------------------+-------------------------+-------+ ``` ### Does this PR introduce _any_ user-facing change? Should not. ### How was this patch tested? Update existed UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47265 from panbingkun/SPARK-48841. Authored-by: panbingkun Signed-off-by: Wenchen Fan --- .../explain-results/function_collate.explain | 2 +- .../expressions/collationExpressions.scala | 4 + .../sql-functions/sql-expression-schema.md | 2 +- .../analyzer-results/collations.sql.out | 60 ++-- .../sql-tests/results/collations.sql.out | 50 +-- .../sql/CollationExpressionWalkerSuite.scala | 4 +- .../spark/sql/CollationSQLRegexpSuite.scala | 292 ++++++++++++++---- .../org/apache/spark/sql/CollationSuite.scala | 4 +- 8 files changed, 296 insertions(+), 122 deletions(-) diff --git a/connect/common/src/test/resources/query-tests/explain-results/function_collate.explain b/connect/common/src/test/resources/query-tests/explain-results/function_collate.explain index c736abf67b11a..e4e6aedc34de9 100644 --- a/connect/common/src/test/resources/query-tests/explain-results/function_collate.explain +++ b/connect/common/src/test/resources/query-tests/explain-results/function_collate.explain @@ -1,2 +1,2 @@ -Project [collate(g#0, UNICODE) AS collate(g)#0] +Project [collate(g#0, UNICODE) AS collate(g, UNICODE)#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala index c528b523c5e7f..c7fbb39ea2859 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collationExpressions.scala @@ -92,6 +92,10 @@ case class Collate(child: Expression, collationName: String) override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = defineCodeGen(ctx, ev, (in) => in) + + override def sql: String = s"$prettyName(${child.sql}, $collationName)" + + override def toString: String = s"$prettyName($child, $collationName)" } // scalastyle:off line.contains.tab diff --git a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md index cf218becdf1d4..bf4622accf411 100644 --- a/sql/core/src/test/resources/sql-functions/sql-expression-schema.md +++ b/sql/core/src/test/resources/sql-functions/sql-expression-schema.md @@ -81,7 +81,7 @@ | org.apache.spark.sql.catalyst.expressions.Chr | char | SELECT char(65) | struct | | org.apache.spark.sql.catalyst.expressions.Chr | chr | SELECT chr(65) | struct | | org.apache.spark.sql.catalyst.expressions.Coalesce | coalesce | SELECT coalesce(NULL, 1, NULL) | struct | -| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UTF8_LCASE) | struct | +| org.apache.spark.sql.catalyst.expressions.CollateExpressionBuilder | collate | SELECT COLLATION('Spark SQL' collate UTF8_LCASE) | struct | | org.apache.spark.sql.catalyst.expressions.Collation | collation | SELECT collation('Spark SQL') | struct | | org.apache.spark.sql.catalyst.expressions.Concat | concat | SELECT concat('Spark', 'SQL') | struct | | org.apache.spark.sql.catalyst.expressions.ConcatWs | concat_ws | SELECT concat_ws(' ', 'Spark', 'SQL') | struct | diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out index e6409806bad7a..6f9cb3b759710 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/collations.sql.out @@ -159,9 +159,9 @@ DropTable false, false select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Except false -:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] +- LocalRelation [col1#x] @@ -169,9 +169,9 @@ Except false select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Except All true -:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] +- LocalRelation [col1#x] @@ -180,9 +180,9 @@ select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ( -- !query analysis Distinct +- Union false, false - :- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] + :- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] : +- LocalRelation [col1#x] - +- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] + +- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] +- LocalRelation [col1#x] @@ -190,9 +190,9 @@ Distinct select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Union false, false -:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] +- LocalRelation [col1#x] @@ -200,9 +200,9 @@ Union false, false select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query analysis Intersect false -:- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] +:- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] : +- LocalRelation [col1#x] -+- Project [collate(col1#x, utf8_lcase) AS collate(col1)#x] ++- Project [collate(col1#x, utf8_lcase) AS collate(col1, utf8_lcase)#x] +- LocalRelation [col1#x] @@ -254,138 +254,138 @@ DropTable false, false -- !query select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase) -- !query analysis -Project [array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase)) AS array_contains(array(collate(aaa)), collate(AAA))#x] +Project [array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase)) AS array_contains(array(collate(aaa, utf8_lcase)), collate(AAA, utf8_lcase))#x] +- OneRowRelation -- !query select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase) -- !query analysis -Project [array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase)) AS array_position(array(collate(aaa), collate(bbb)), collate(BBB))#xL] +Project [array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase)) AS array_position(array(collate(aaa, utf8_lcase), collate(bbb, utf8_lcase)), collate(BBB, utf8_lcase))#xL] +- OneRowRelation -- !query select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase) -- !query analysis -Project [nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)) AS nullif(collate(aaa), collate(AAA))#x] +Project [nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)) AS nullif(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))#x] +- OneRowRelation -- !query select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase) -- !query analysis -Project [least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase)) AS least(collate(aaa), collate(AAA), collate(a))#x] +Project [least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase)) AS least(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase), collate(a, utf8_lcase))#x] +- OneRowRelation -- !query select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS arrays_overlap(array(collate(aaa)), array(collate(AAA)))#x] +Project [arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS arrays_overlap(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x] +- OneRowRelation -- !query select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)) -- !query analysis -Project [array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))) AS array_distinct(array(collate(aaa), collate(AAA)))#x] +Project [array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase))) AS array_distinct(array(collate(aaa, utf8_lcase), collate(AAA, utf8_lcase)))#x] +- OneRowRelation -- !query select array_union(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_union(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_union(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x] +- OneRowRelation -- !query select array_intersect(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_intersect(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_intersect(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x] +- OneRowRelation -- !query select array_except(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query analysis -Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_except(array(collate(aaa)), array(collate(AAA)))#x] +Project [array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase))) AS array_except(array(collate(aaa, utf8_lcase)), array(collate(AAA, utf8_lcase)))#x] +- OneRowRelation -- !query select 'a' collate unicode < 'A' -- !query analysis -Project [(collate(a, unicode) < cast(A as string collate UNICODE)) AS (collate(a) < A)#x] +Project [(collate(a, unicode) < cast(A as string collate UNICODE)) AS (collate(a, unicode) < A)#x] +- OneRowRelation -- !query select 'a' collate unicode_ci = 'A' -- !query analysis -Project [(collate(a, unicode_ci) = cast(A as string collate UNICODE_CI)) AS (collate(a) = A)#x] +Project [(collate(a, unicode_ci) = cast(A as string collate UNICODE_CI)) AS (collate(a, unicode_ci) = A)#x] +- OneRowRelation -- !query select 'a' collate unicode_ai = 'å' -- !query analysis -Project [(collate(a, unicode_ai) = cast(å as string collate UNICODE_AI)) AS (collate(a) = å)#x] +Project [(collate(a, unicode_ai) = cast(å as string collate UNICODE_AI)) AS (collate(a, unicode_ai) = å)#x] +- OneRowRelation -- !query select 'a' collate unicode_ci_ai = 'Å' -- !query analysis -Project [(collate(a, unicode_ci_ai) = cast(Å as string collate UNICODE_CI_AI)) AS (collate(a) = Å)#x] +Project [(collate(a, unicode_ci_ai) = cast(Å as string collate UNICODE_CI_AI)) AS (collate(a, unicode_ci_ai) = Å)#x] +- OneRowRelation -- !query select 'a' collate en < 'A' -- !query analysis -Project [(collate(a, en) < cast(A as string collate en)) AS (collate(a) < A)#x] +Project [(collate(a, en) < cast(A as string collate en)) AS (collate(a, en) < A)#x] +- OneRowRelation -- !query select 'a' collate en_ci = 'A' -- !query analysis -Project [(collate(a, en_ci) = cast(A as string collate en_CI)) AS (collate(a) = A)#x] +Project [(collate(a, en_ci) = cast(A as string collate en_CI)) AS (collate(a, en_ci) = A)#x] +- OneRowRelation -- !query select 'a' collate en_ai = 'å' -- !query analysis -Project [(collate(a, en_ai) = cast(å as string collate en_AI)) AS (collate(a) = å)#x] +Project [(collate(a, en_ai) = cast(å as string collate en_AI)) AS (collate(a, en_ai) = å)#x] +- OneRowRelation -- !query select 'a' collate en_ci_ai = 'Å' -- !query analysis -Project [(collate(a, en_ci_ai) = cast(Å as string collate en_CI_AI)) AS (collate(a) = Å)#x] +Project [(collate(a, en_ci_ai) = cast(Å as string collate en_CI_AI)) AS (collate(a, en_ci_ai) = Å)#x] +- OneRowRelation -- !query select 'Kypper' collate sv < 'Köpfe' -- !query analysis -Project [(collate(Kypper, sv) < cast(Köpfe as string collate sv)) AS (collate(Kypper) < Köpfe)#x] +Project [(collate(Kypper, sv) < cast(Köpfe as string collate sv)) AS (collate(Kypper, sv) < Köpfe)#x] +- OneRowRelation -- !query select 'Kypper' collate de > 'Köpfe' -- !query analysis -Project [(collate(Kypper, de) > cast(Köpfe as string collate de)) AS (collate(Kypper) > Köpfe)#x] +Project [(collate(Kypper, de) > cast(Köpfe as string collate de)) AS (collate(Kypper, de) > Köpfe)#x] +- OneRowRelation -- !query select 'I' collate tr_ci = 'ı' -- !query analysis -Project [(collate(I, tr_ci) = cast(ı as string collate tr_CI)) AS (collate(I) = ı)#x] +Project [(collate(I, tr_ci) = cast(ı as string collate tr_CI)) AS (collate(I, tr_ci) = ı)#x] +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/results/collations.sql.out b/sql/core/src/test/resources/sql-tests/results/collations.sql.out index 89e6665df9d04..37fb6c4e114ef 100644 --- a/sql/core/src/test/resources/sql-tests/results/collations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/collations.sql.out @@ -170,7 +170,7 @@ struct<> -- !query select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output zzz @@ -178,7 +178,7 @@ zzz -- !query select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') except all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb @@ -189,7 +189,7 @@ zzz -- !query select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb @@ -199,7 +199,7 @@ zzz -- !query select col1 collate utf8_lcase from values ('aaa'), ('AAA'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') union all select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output AAA BBB @@ -214,7 +214,7 @@ zzz -- !query select col1 collate utf8_lcase from values ('aaa'), ('bbb'), ('BBB'), ('zzz'), ('ZZZ') intersect select col1 collate utf8_lcase from values ('aaa'), ('bbb') -- !query schema -struct +struct -- !query output aaa bbb @@ -272,7 +272,7 @@ struct<> -- !query select array_contains(ARRAY('aaa' collate utf8_lcase),'AAA' collate utf8_lcase) -- !query schema -struct +struct -- !query output true @@ -280,7 +280,7 @@ true -- !query select array_position(ARRAY('aaa' collate utf8_lcase, 'bbb' collate utf8_lcase),'BBB' collate utf8_lcase) -- !query schema -struct +struct -- !query output 2 @@ -288,7 +288,7 @@ struct -- !query select nullif('aaa' COLLATE utf8_lcase, 'AAA' COLLATE utf8_lcase) -- !query schema -struct +struct -- !query output NULL @@ -296,7 +296,7 @@ NULL -- !query select least('aaa' COLLATE utf8_lcase, 'AAA' collate utf8_lcase, 'a' collate utf8_lcase) -- !query schema -struct +struct -- !query output a @@ -304,7 +304,7 @@ a -- !query select arrays_overlap(array('aaa' collate utf8_lcase), array('AAA' collate utf8_lcase)) -- !query schema -struct +struct -- !query output true @@ -312,7 +312,7 @@ true -- !query select array_distinct(array('aaa' collate utf8_lcase, 'AAA' collate utf8_lcase)) -- !query schema -struct> +struct> -- !query output ["aaa"] @@ -320,7 +320,7 @@ struct> +struct> -- !query output ["aaa"] @@ -328,7 +328,7 @@ struct> +struct> -- !query output ["aaa"] @@ -336,7 +336,7 @@ struct> +struct> -- !query output [] @@ -344,7 +344,7 @@ struct +struct<(collate(a, unicode) < A):boolean> -- !query output true @@ -352,7 +352,7 @@ true -- !query select 'a' collate unicode_ci = 'A' -- !query schema -struct<(collate(a) = A):boolean> +struct<(collate(a, unicode_ci) = A):boolean> -- !query output true @@ -360,7 +360,7 @@ true -- !query select 'a' collate unicode_ai = 'å' -- !query schema -struct<(collate(a) = å):boolean> +struct<(collate(a, unicode_ai) = å):boolean> -- !query output true @@ -368,7 +368,7 @@ true -- !query select 'a' collate unicode_ci_ai = 'Å' -- !query schema -struct<(collate(a) = Å):boolean> +struct<(collate(a, unicode_ci_ai) = Å):boolean> -- !query output true @@ -376,7 +376,7 @@ true -- !query select 'a' collate en < 'A' -- !query schema -struct<(collate(a) < A):boolean> +struct<(collate(a, en) < A):boolean> -- !query output true @@ -384,7 +384,7 @@ true -- !query select 'a' collate en_ci = 'A' -- !query schema -struct<(collate(a) = A):boolean> +struct<(collate(a, en_ci) = A):boolean> -- !query output true @@ -392,7 +392,7 @@ true -- !query select 'a' collate en_ai = 'å' -- !query schema -struct<(collate(a) = å):boolean> +struct<(collate(a, en_ai) = å):boolean> -- !query output true @@ -400,7 +400,7 @@ true -- !query select 'a' collate en_ci_ai = 'Å' -- !query schema -struct<(collate(a) = Å):boolean> +struct<(collate(a, en_ci_ai) = Å):boolean> -- !query output true @@ -408,7 +408,7 @@ true -- !query select 'Kypper' collate sv < 'Köpfe' -- !query schema -struct<(collate(Kypper) < Köpfe):boolean> +struct<(collate(Kypper, sv) < Köpfe):boolean> -- !query output true @@ -416,7 +416,7 @@ true -- !query select 'Kypper' collate de > 'Köpfe' -- !query schema -struct<(collate(Kypper) > Köpfe):boolean> +struct<(collate(Kypper, de) > Köpfe):boolean> -- !query output true @@ -424,6 +424,6 @@ true -- !query select 'I' collate tr_ci = 'ı' -- !query schema -struct<(collate(I) = ı):boolean> +struct<(collate(I, tr_ci) = ı):boolean> -- !query output true 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 a639367e8ca53..e4f4bb6e85575 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 @@ -568,8 +568,8 @@ class CollationExpressionWalkerSuite extends SparkFunSuite with SharedSparkSessi case st if utf8BinaryResultChecked != null && utf8BinaryLcaseResultChecked != null && hasStringType(st) => // scalastyle:off caselocale - assert(utf8BinaryResultChecked.getRows(1, 0).map(_.map(_.toLowerCase)) === - utf8BinaryLcaseResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))) + assert(utf8BinaryResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1) === + utf8BinaryLcaseResultChecked.getRows(1, 0).map(_.map(_.toLowerCase))(1)) // scalastyle:on caselocale case _ => assert(utf8BinaryResultChecked.getRows(1, 0)(1) === diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala index 885ed37098680..8ff7bed60bbce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationSQLRegexpSuite.scala @@ -50,10 +50,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT like(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"collate(ABC, UNICODE_CI) LIKE %b%\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABC, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"like(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 47) + ) }) } @@ -132,10 +144,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT ilike(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"ilike(collate(ABC, UNICODE_CI), %b%)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABC, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"ilike(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 48) + ) }) } @@ -160,10 +184,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT collate('${t.s}', '${t.c}') LIKE ALL ('${t.p.mkString("','")}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"likeall(collate(Foo, UNICODE_CI))\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(Foo, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"LIKE ALL ('${t.p.mkString("','")}')", + start = 36, + stop = 59) + ) }) } @@ -188,10 +224,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT collate('${t.s}', '${t.c}') NOT LIKE ALL ('${t.p.mkString("','")}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"notlikeall(collate(Foo, UNICODE_CI))\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(Foo, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"NOT LIKE ALL ('${t.p.mkString("','")}')", + start = 36, + stop = 63) + ) }) } @@ -216,10 +264,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT collate('${t.s}', '${t.c}') LIKE ANY ('${t.p.mkString("','")}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"likeany(collate(Foo, UNICODE_CI))\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(Foo, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"LIKE ANY ('${t.p.mkString("','")}')", + start = 36, + stop = 59) + ) }) } @@ -244,10 +304,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT collate('${t.s}', '${t.c}') NOT LIKE ANY ('${t.p.mkString("','")}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"notlikeany(collate(Foo, UNICODE_CI))\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(Foo, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"NOT LIKE ANY ('${t.p.mkString("','")}')", + start = 36, + stop = 63) + ) }) } @@ -272,10 +344,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT rlike(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"RLIKE(collate(ABC, UNICODE_CI), .b.)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABC, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"rlike(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 48) + ) }) } @@ -300,10 +384,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT split(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"split(collate(ABC, UNICODE_CI), [b], -1)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABC, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"split(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 48) + ) }) } @@ -329,10 +425,13 @@ class CollationSQLRegexpSuite }) // Collation mismatch val (c1, c2) = ("UTF8_BINARY", "UTF8_LCASE") - val collationMismatch = intercept[AnalysisException] { - sql(s"SELECT regexp_replace(collate('ABCDE','$c1'), '.c.', collate('FFF','$c2'))") - } - assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") + checkError( + exception = intercept[AnalysisException] { + sql(s"SELECT regexp_replace(collate('ABCDE','$c1'), '.c.', collate('FFF','$c2'))") + }, + errorClass = "COLLATION_MISMATCH.EXPLICIT", + parameters = Map("explicitTypes" -> "`string`.`string collate UTF8_LCASE`") + ) // Unsupported collations case class RegExpReplaceTestFail(l: String, r: String, c: String) val failCases = Seq( @@ -341,10 +440,22 @@ class CollationSQLRegexpSuite failCases.foreach(t => { val query = s"SELECT regexp_replace(collate('${t.l}', '${t.c}'), '${t.r}', 'FFF')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_replace(collate(ABCDE, UNICODE_CI), .c., FFF, 1)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_replace(collate('${t.l}', '${t.c}'), '${t.r}', 'FFF')", + start = 7, + stop = 66) + ) }) } @@ -371,10 +482,22 @@ class CollationSQLRegexpSuite failCases.foreach(t => { val query = s"SELECT regexp_extract(collate('${t.l}', '${t.c}'), '${t.r}', 0)" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_extract(collate(ABCDE, UNICODE_CI), .c., 0)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_extract(collate('${t.l}', '${t.c}'), '${t.r}', 0)", + start = 7, + stop = 62) + ) }) } @@ -401,10 +524,22 @@ class CollationSQLRegexpSuite failCases.foreach(t => { val query = s"SELECT regexp_extract_all(collate('${t.l}', '${t.c}'), '${t.r}', 0)" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_extract_all(collate(ABCDE, UNICODE_CI), .c., 0)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_extract_all(collate('${t.l}', '${t.c}'), '${t.r}', 0)", + start = 7, + stop = 66) + ) }) } @@ -429,10 +564,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT regexp_count(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_count(collate(ABCDE, UNICODE_CI), .c.)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_count(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 57) + ) }) } @@ -457,10 +604,22 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT regexp_substr(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_substr(collate(ABCDE, UNICODE_CI), .c.)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_substr(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 58) + ) }) } @@ -485,12 +644,23 @@ class CollationSQLRegexpSuite ) failCases.foreach(t => { val query = s"SELECT regexp_instr(collate('${t.l}', '${t.c}'), '${t.r}')" - val unsupportedCollation = intercept[AnalysisException] { - sql(query) - } - assert(unsupportedCollation.getErrorClass === "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE") + checkError( + exception = intercept[AnalysisException] { + sql(query) + }, + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"regexp_instr(collate(ABCDE, UNICODE_CI), .c., 0)\"", + "paramIndex" -> "first", + "inputSql" -> "\"collate(ABCDE, UNICODE_CI)\"", + "inputType" -> "\"STRING COLLATE UNICODE_CI\"", + "requiredType" -> "\"STRING\""), + context = ExpectedContext( + fragment = s"regexp_instr(collate('${t.l}', '${t.c}'), '${t.r}')", + start = 7, + stop = 57) + ) }) } - } // scalastyle:on nonascii 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 f662b86eaf815..af3169932bfc5 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 @@ -135,7 +135,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", sqlState = "42K09", parameters = Map( - "sqlExpr" -> "\"collate(1)\"", + "sqlExpr" -> "\"collate(1, UTF8_BINARY)\"", "paramIndex" -> "first", "inputSql" -> "\"1\"", "inputType" -> "\"INT\"", @@ -1026,7 +1026,7 @@ class CollationSuite extends DatasourceV2SQLBase with AdaptiveSparkPlanHelper { exception = intercept[AnalysisException](sql(query)), errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", parameters = Map( - "sqlExpr" -> "\"map(collate(aaa), 1, collate(AAA), 2)[AaA]\"", + "sqlExpr" -> "\"map(collate(aaa, utf8_lcase), 1, collate(AAA, utf8_lcase), 2)[AaA]\"", "paramIndex" -> "second", "inputSql" -> "\"AaA\"", "inputType" -> toSQLType(StringType), From 2bee9a611b8e97d0d069d3711c53ee3877c6970d Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 12 Jul 2024 11:30:46 +0900 Subject: [PATCH 43/64] [SPARK-46654][DOCS][FOLLOW-UP] Remove obsolete TODO item ### What changes were proposed in this pull request? Remove obsolete TODO item ### Why are the changes needed? the `Example 2` test had been already enabled ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #47312 from zhengruifeng/simple_folloup. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 3184ce78ac7e2..1ca522313f246 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -16509,8 +16509,6 @@ def schema_of_csv(csv: Union[Column, str], options: Optional[Dict[str, str]] = N return _invoke_function("schema_of_csv", col, _options_to_str(options)) -# TODO(SPARK-46654) Re-enable the `Example 2` test after fixing the display -# difference between Regular Spark and Spark Connect on `df.show`. @_try_remote_functions def to_csv(col: "ColumnOrName", options: Optional[Dict[str, str]] = None) -> Column: """ From cbe6846c477bc8b6d94385ddd0097c4e97b05d41 Mon Sep 17 00:00:00 2001 From: Jiaheng Tang Date: Fri, 12 Jul 2024 11:10:38 +0800 Subject: [PATCH 44/64] [SPARK-48760][SQL] Fix CatalogV2Util.applyClusterByChanges ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/47156/ introduced a bug in `CatalogV2Util.applyClusterByChanges` that it will remove the existing `ClusterByTransform` first, regardless of whether there is a `ClusterBy` table change. This means any table change will remove the clustering columns from the table. This PR fixes the bug by removing the `ClusterByTransform` only when there is a `ClusterBy` table change. ### Why are the changes needed? ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Amend existing test to catch this bug. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47288 from zedtang/fix-apply-cluster-by-changes. Authored-by: Jiaheng Tang Signed-off-by: Wenchen Fan --- .../sql/connector/catalog/CatalogV2Util.scala | 19 +++++++++++-------- .../command/DescribeTableSuiteBase.scala | 3 ++- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala index c5888d72c2b23..645ed9e6bb0c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogV2Util.scala @@ -178,16 +178,19 @@ private[sql] object CatalogV2Util { schema: StructType, changes: Seq[TableChange]): Array[Transform] = { - val newPartitioning = partitioning.filterNot(_.isInstanceOf[ClusterByTransform]).toBuffer - changes.foreach { - case clusterBy: ClusterBy => - newPartitioning += ClusterBySpec.extractClusterByTransform( + var newPartitioning = partitioning + // If there is a clusterBy change (only the first one), we overwrite the existing + // clustering columns. + val clusterByOpt = changes.collectFirst { case c: ClusterBy => c } + clusterByOpt.foreach { clusterBy => + newPartitioning = partitioning.map { + case _: ClusterByTransform => ClusterBySpec.extractClusterByTransform( schema, ClusterBySpec(clusterBy.clusteringColumns.toIndexedSeq), conf.resolver) - - case _ => - // ignore other changes + case other => other + } } - newPartitioning.toArray + + newPartitioning } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala index 2588aa4313fa7..02e8a5e689998 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DescribeTableSuiteBase.scala @@ -181,8 +181,9 @@ trait DescribeTableSuiteBase extends QueryTest with DDLCommandTestUtils { test("describe a clustered table") { withNamespaceAndTable("ns", "tbl") { tbl => - sql(s"CREATE TABLE $tbl (col1 STRING COMMENT 'this is comment', col2 struct) " + + sql(s"CREATE TABLE $tbl (col1 STRING, col2 struct) " + s"$defaultUsing CLUSTER BY (col1, col2.x)") + sql(s"ALTER TABLE $tbl ALTER COLUMN col1 COMMENT 'this is comment';") val descriptionDf = sql(s"DESC $tbl") assert(descriptionDf.schema.map(field => (field.name, field.dataType)) === Seq( ("col_name", StringType), From d274c2cdcbeb42f7bf165001d20c9b9b0be0771b Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 12 Jul 2024 11:41:28 +0800 Subject: [PATCH 45/64] [SPARK-48874][SQL][DOCKER][BUILD][TESTS] Upgrade `MySQL` docker image version to `9.0.0` ### What changes were proposed in this pull request? The pr aims to upgrade `MySQ`L docker image version from `8.4.0` to `9.0.0`. ### Why are the changes needed? After https://issues.apache.org/jira/browse/SPARK-48795, we have upgraded the `mysql jdbc driver` version to `9.0.0` for testing, so I propose that the corresponding `mysql server docker image` should also be upgraded to `9.0.0` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47311 from panbingkun/mysql_image_9. Authored-by: panbingkun Signed-off-by: Kent Yao --- .../org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala | 2 +- .../org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala | 8 ++++---- .../apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala | 8 ++++---- .../apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala | 4 ++-- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala index 570a81ac3947f..97f076eccf941 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLDatabaseOnDocker.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.jdbc class MySQLDatabaseOnDocker extends DatabaseOnDocker { - override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:8.4.0") + override val imageName = sys.env.getOrElse("MYSQL_DOCKER_IMAGE_NAME", "mysql:9.0.0") override val env = Map( "MYSQL_ROOT_PASSWORD" -> "rootpass" ) diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala index e6cca2ac9cd0a..b0ba86a9f6c4f 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MySQLIntegrationSuite.scala @@ -31,9 +31,9 @@ import org.apache.spark.sql.types.ShortType import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., mysql:8.3.0): + * To run this test suite for a specific version (e.g., mysql:9.0.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:8.3.0 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:9.0.0 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly org.apache.spark.sql.jdbc.MySQLIntegrationSuite" * }}} @@ -355,9 +355,9 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { /** - * To run this test suite for a specific version (e.g., mysql:8.3.0): + * To run this test suite for a specific version (e.g., mysql:9.0.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:8.3.0 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:9.0.0 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *MySQLOverMariaConnectorIntegrationSuite" * }}} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala index d5478e664221d..c48505ff153fd 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLIntegrationSuite.scala @@ -27,9 +27,9 @@ import org.apache.spark.sql.types._ import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., mysql:8.3.0): + * To run this test suite for a specific version (e.g., mysql:9.0.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:8.3.0 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:9.0.0 * ./build/sbt -Pdocker-integration-tests "testOnly *v2*MySQLIntegrationSuite" * }}} */ @@ -160,9 +160,9 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationV2Suite with V2JDBCTest } /** - * To run this test suite for a specific version (e.g., mysql:8.3.0): + * To run this test suite for a specific version (e.g., mysql:9.0.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:8.3.0 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:9.0.0 * ./build/sbt -Pdocker-integration-tests * "docker-integration-tests/testOnly *MySQLOverMariaConnectorIntegrationSuite" * }}} diff --git a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala index 2b607fccd1710..b49f6901034b1 100644 --- a/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala +++ b/connector/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/v2/MySQLNamespaceSuite.scala @@ -28,9 +28,9 @@ import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.tags.DockerTest /** - * To run this test suite for a specific version (e.g., mysql:8.3.0): + * To run this test suite for a specific version (e.g., mysql:9.0.0): * {{{ - * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:8.3.0 + * ENABLE_DOCKER_INTEGRATION_TESTS=1 MYSQL_DOCKER_IMAGE_NAME=mysql:9.0.0 * ./build/sbt -Pdocker-integration-tests "testOnly *v2*MySQLNamespaceSuite" * }}} */ From b560e4ebbd244b4152678d888da23d1d657b1091 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 12 Jul 2024 13:18:02 +0900 Subject: [PATCH 46/64] Remove unused test jar (#47309) --- connect/server/src/test/resources/udf_noA.jar | Bin 5545 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 connect/server/src/test/resources/udf_noA.jar diff --git a/connect/server/src/test/resources/udf_noA.jar b/connect/server/src/test/resources/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw Date: Fri, 12 Jul 2024 13:19:44 +0900 Subject: [PATCH 47/64] Revert "Remove unused test jar (#47309)" This reverts commit b560e4ebbd244b4152678d888da23d1d657b1091. --- connect/server/src/test/resources/udf_noA.jar | Bin 0 -> 5545 bytes 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 connect/server/src/test/resources/udf_noA.jar diff --git a/connect/server/src/test/resources/udf_noA.jar b/connect/server/src/test/resources/udf_noA.jar new file mode 100644 index 0000000000000000000000000000000000000000..4d8c423ab6dfbffe0b0f065faa15de74e133e0eb GIT binary patch literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw Date: Fri, 12 Jul 2024 12:41:07 +0800 Subject: [PATCH 48/64] [SPARK-48842][DOCS] Document non-determinism of max_by and min_by ### What changes were proposed in this pull request? Document non-determinism of max_by and min_by ### Why are the changes needed? I have been confused by this non-determinism twice, it occurred like a correctness bug to me. So I think we need to document it ### Does this PR introduce _any_ user-facing change? doc change only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #47266 from zhengruifeng/py_doc_max_by. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- R/pkg/R/functions.R | 6 ++++++ .../main/scala/org/apache/spark/sql/functions.scala | 8 ++++++++ python/pyspark/sql/functions/builtin.py | 10 ++++++++++ .../catalyst/expressions/aggregate/MaxByAndMinBy.scala | 8 ++++++++ .../main/scala/org/apache/spark/sql/functions.scala | 6 ++++++ 5 files changed, 38 insertions(+) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index a7e337d3f9af2..b91124f96a6fa 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -1558,6 +1558,9 @@ setMethod("max", #' @details #' \code{max_by}: Returns the value associated with the maximum value of ord. #' +#' Note: The function is non-deterministic so the output order can be different +#' for those associated the same values of `x`. +#' #' @rdname column_aggregate_functions #' @aliases max_by max_by,Column-method #' @note max_by since 3.3.0 @@ -1633,6 +1636,9 @@ setMethod("min", #' @details #' \code{min_by}: Returns the value associated with the minimum value of ord. #' +#' Note: The function is non-deterministic so the output order can be different +#' for those associated the same values of `x`. +#' #' @rdname column_aggregate_functions #' @aliases min_by min_by,Column-method #' @note min_by since 3.3.0 diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala index 92e7bc9da5904..81f25b3d743f0 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/functions.scala @@ -884,6 +884,10 @@ object functions { /** * Aggregate function: returns the value associated with the maximum value of ord. * + * @note + * The function is non-deterministic so the output order can be different for those associated + * the same values of `e`. + * * @group agg_funcs * @since 3.4.0 */ @@ -932,6 +936,10 @@ object functions { /** * Aggregate function: returns the value associated with the minimum value of ord. * + * @note + * The function is non-deterministic so the output order can be different for those associated + * the same values of `e`. + * * @group agg_funcs * @since 3.4.0 */ diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 1ca522313f246..446ff2b1be93d 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -1271,6 +1271,11 @@ def max_by(col: "ColumnOrName", ord: "ColumnOrName") -> Column: .. versionchanged:: 3.4.0 Supports Spark Connect. + Notes + ----- + The function is non-deterministic so the output order can be different for those + associated the same values of `col`. + Parameters ---------- col : :class:`~pyspark.sql.Column` or str @@ -1352,6 +1357,11 @@ def min_by(col: "ColumnOrName", ord: "ColumnOrName") -> Column: .. versionchanged:: 3.4.0 Supports Spark Connect. + Notes + ----- + The function is non-deterministic so the output order can be different for those + associated the same values of `col`. + Parameters ---------- col : :class:`~pyspark.sql.Column` or str diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala index 56941c9de4510..b33142ed29cc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala @@ -99,6 +99,10 @@ abstract class MaxMinBy extends DeclarativeAggregate with BinaryLike[Expression] > SELECT _FUNC_(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y); b """, + note = """ + The function is non-deterministic so the output order can be different for + those associated the same values of `x`. + """, group = "agg_funcs", since = "3.0.0") case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { @@ -122,6 +126,10 @@ case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMin > SELECT _FUNC_(x, y) FROM VALUES ('a', 10), ('b', 50), ('c', 20) AS tab(x, y); a """, + note = """ + The function is non-deterministic so the output order can be different for + those associated the same values of `x`. + """, group = "agg_funcs", since = "3.0.0") case class MinBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 882918eb78c7f..5b4d27fc65d01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -902,6 +902,9 @@ object functions { /** * Aggregate function: returns the value associated with the maximum value of ord. * + * @note The function is non-deterministic so the output order can be different for + * those associated the same values of `e`. + * * @group agg_funcs * @since 3.3.0 */ @@ -952,6 +955,9 @@ object functions { /** * Aggregate function: returns the value associated with the minimum value of ord. * + * @note The function is non-deterministic so the output order can be different for + * those associated the same values of `e`. + * * @group agg_funcs * @since 3.3.0 */ From 9cbd5dd4e7477294f7d4289880c7ea0dd67b38d3 Mon Sep 17 00:00:00 2001 From: "zhipeng.mao" Date: Fri, 12 Jul 2024 14:51:47 +0800 Subject: [PATCH 49/64] =?UTF-8?q?[SPARK-48871]=20Fix=20INVALID=5FNON=5FDET?= =?UTF-8?q?ERMINISTIC=5FEXPRESSIONS=20validation=20in=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … CheckAnalysis ### What changes were proposed in this pull request? The PR added a trait that logical plans can extend to implement a method to decide whether there can be non-deterministic expressions for the operator, and check this method in checkAnalysis. ### Why are the changes needed? I encountered the `INVALID_NON_DETERMINISTIC_EXPRESSIONS` exception when attempting to use a non-deterministic udf in my query. The non-deterministic expression can be safely allowed for my custom LogicalPlan, but it is disabled in the checkAnalysis phase. The CheckAnalysis rule is too strict so that reasonable use cases of non-deterministic expressions are also disabled. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The test case `"SPARK-48871: AllowsNonDeterministicExpression allow lists non-deterministic expressions"` is added. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47304 from zhipengmao-db/zhipengmao-db/SPARK-48871-check-analysis. Lead-authored-by: zhipeng.mao Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/CheckAnalysis.scala | 12 ++++++++++ .../plans/logical/basicLogicalOperators.scala | 10 ++++++++ .../analysis/AnalysisErrorSuite.scala | 23 +++++++++++++++++++ 3 files changed, 45 insertions(+) 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 92c734af3a60e..2bc6785aa40c3 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 @@ -143,6 +143,17 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB errorClass, missingCol, orderedCandidates, a.origin) } + /** + * Checks whether the operator allows non-deterministic expressions. + */ + private def operatorAllowsNonDeterministicExpressions(plan: LogicalPlan): Boolean = { + plan match { + case p: SupportsNonDeterministicExpression => + p.allowNonDeterministicExpression + case _ => false + } + } + def checkAnalysis(plan: LogicalPlan): Unit = { // We should inline all CTE relations to restore the original plan shape, as the analysis check // may need to match certain plan shapes. For dangling CTE relations, they will still be kept @@ -718,6 +729,7 @@ trait CheckAnalysis extends PredicateHelper with LookupCatalog with QueryErrorsB "dataType" -> toSQLType(mapCol.dataType))) case o if o.expressions.exists(!_.deterministic) && + !operatorAllowsNonDeterministicExpressions(o) && !o.isInstanceOf[Project] && // non-deterministic expressions inside CollectMetrics have been // already validated inside checkMetric function diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 0135fcfb3cc8c..e784e6695dbd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -1998,6 +1998,16 @@ case class DeduplicateWithinWatermark(keys: Seq[Attribute], child: LogicalPlan) */ trait SupportsSubquery extends LogicalPlan +/** + * Trait that logical plans can extend to check whether it can allow non-deterministic + * expressions and pass the CheckAnalysis rule. + */ +trait SupportsNonDeterministicExpression extends LogicalPlan { + + /** Returns whether it allows non-deterministic expressions. */ + def allowNonDeterministicExpression: Boolean +} + /** * Collect arbitrary (named) metrics from a dataset. As soon as the query reaches a completion * point (batch query completes or streaming query epoch completes) an event is emitted on the diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala index 19eb3a418543d..6c6414dd190de 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisErrorSuite.scala @@ -90,6 +90,13 @@ case class TestFunctionWithTypeCheckFailure( case class UnresolvedTestPlan() extends UnresolvedLeafNode +case class SupportsNonDeterministicExpressionTestOperator( + actions: Seq[Expression], + allowNonDeterministicExpression: Boolean) + extends LeafNode with SupportsNonDeterministicExpression { + override def output: Seq[Attribute] = Seq() +} + class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { import TestRelations._ @@ -1364,4 +1371,20 @@ class AnalysisErrorSuite extends AnalysisTest with DataTypeErrorsBase { messageParameters = Map( "expr" -> "\"_w0\"", "exprType" -> "\"MAP\"")) + + test("SPARK-48871: SupportsNonDeterministicExpression allows non-deterministic expressions") { + val nonDeterministicExpressions = Seq(new Rand()) + val tolerantPlan = + SupportsNonDeterministicExpressionTestOperator( + nonDeterministicExpressions, allowNonDeterministicExpression = true) + assertAnalysisSuccess(tolerantPlan) + + val intolerantPlan = + SupportsNonDeterministicExpressionTestOperator( + nonDeterministicExpressions, allowNonDeterministicExpression = false) + assertAnalysisError( + intolerantPlan, + "INVALID_NON_DETERMINISTIC_EXPRESSIONS" :: Nil + ) + } } From 4c7edd2a20480a8521fcc88a966b22619143aebd Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 12 Jul 2024 15:22:34 +0800 Subject: [PATCH 50/64] [SPARK-48864][SQL][TESTS] Refactor `HiveQuerySuite` and fix bug ### What changes were proposed in this pull request? The pr aims to refactor `HiveQuerySuite` and `fix` bug, includes: - use `getWorkspaceFilePath` to enable `HiveQuerySuite` to run successfully in the IDE. - make the test `lookup hive UDF in another thread` `independence`, without relying on the previous UT `current_database with multiple sessions`. - enable two test: `non-boolean conditions in a CaseWhen are illegal` and `Dynamic partition folder layout`. ### Why are the changes needed? - Run successfully in the `IDE` Before: image After: image - Make UT `lookup hive UDF in another thread` `independence` when `only` running it, it actually failed with the following error: image **why ?** Because the previous UT `current_database with multiple sessions` changed `current database` and was not restored after it finished running. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Manually test - Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47293 from panbingkun/refactor_HiveQuerySuite. Authored-by: panbingkun Signed-off-by: yangjie01 --- .../hive/execution/HiveComparisonTest.scala | 5 +- .../sql/hive/execution/HiveQuerySuite.scala | 249 ++++++++++-------- 2 files changed, 135 insertions(+), 119 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index f0feccb4f494a..87e58bb8fa13a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -100,8 +100,9 @@ abstract class HiveComparisonTest extends SparkFunSuite with BeforeAndAfterAll { .map(name => new File(targetDir, s"$suiteName.$name")) /** The local directory with cached golden answer will be stored. */ - protected val answerCache = new File("src" + File.separator + "test" + - File.separator + "resources" + File.separator + "golden") + protected val answerCache = getWorkspaceFilePath( + "sql", "hive", "src", "test", "resources", "golden").toFile + if (!answerCache.exists) { answerCache.mkdir() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 5ccb7f0d1f84a..24d1e24b30c8b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.hive.execution import java.io.File -import java.net.URI import java.nio.file.Files import java.sql.Timestamp @@ -679,15 +678,23 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd assert(actual === expected) } - // TODO: adopt this test when Spark SQL has the functionality / framework to report errors. - // See https://github.com/apache/spark/pull/1055#issuecomment-45820167 for a discussion. - ignore("non-boolean conditions in a CaseWhen are illegal") { + test("non-boolean conditions in a CaseWhen are illegal") { checkError( exception = intercept[AnalysisException] { sql("SELECT (CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END) FROM src").collect() }, - errorClass = null, - parameters = Map.empty) + errorClass = "DATATYPE_MISMATCH.UNEXPECTED_INPUT_TYPE", + parameters = Map( + "sqlExpr" -> "\"CASE WHEN (key > 2) THEN 3 WHEN 1 THEN 2 ELSE 0 END\"", + "paramIndex" -> "second", + "inputSql" -> "\"1\"", + "inputType" -> "\"INT\"", + "requiredType" -> "\"BOOLEAN\""), + context = ExpectedContext( + fragment = "CASE WHEN key > 2 THEN 3 WHEN 1 THEN 2 ELSE 0 END", + start = 8, + stop = 56) + ) } createQueryTest("case sensitivity when query Hive table", @@ -804,18 +811,19 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } test("ADD JAR command") { - sql("CREATE TABLE alter1(a INT, b INT) USING HIVE") - checkError( - exception = intercept[AnalysisException] { - sql( - """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' - |WITH serdeproperties('s1'='9')""".stripMargin) - }, - errorClass = "_LEGACY_ERROR_TEMP_3065", - parameters = Map( - "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", - "msg" -> "at least one column must be specified for the table")) - sql("DROP TABLE alter1") + withTable("alter1") { + sql("CREATE TABLE alter1(a INT, b INT) USING HIVE") + checkError( + exception = intercept[AnalysisException] { + sql( + """ALTER TABLE alter1 SET SERDE 'org.apache.hadoop.hive.serde2.TestSerDe' + |WITH serdeproperties('s1'='9')""".stripMargin) + }, + errorClass = "_LEGACY_ERROR_TEMP_3065", + parameters = Map( + "clazz" -> "org.apache.hadoop.hive.ql.metadata.HiveException", + "msg" -> "at least one column must be specified for the table")) + } } test("ADD JAR command 2") { @@ -823,12 +831,13 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd val testJar = HiveTestJars.getHiveHcatalogCoreJar().toURI val testData = TestHive.getHiveFile("data/files/sample.json").toURI sql(s"ADD JAR $testJar") - sql( - """CREATE TABLE t1(a string, b string) - |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) - sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") - sql("select * from src join t1 on src.key = t1.a") - sql("DROP TABLE t1") + withTable("t1") { + sql( + """CREATE TABLE t1(a string, b string) + |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'""".stripMargin) + sql(s"""LOAD DATA LOCAL INPATH "$testData" INTO TABLE t1""") + sql("select * from src join t1 on src.key = t1.a") + } assert(sql("list jars"). filter(_.getString(0).contains(HiveTestJars.getHiveHcatalogCoreJar().getName)).count() > 0) assert(sql("list jar"). @@ -854,12 +863,13 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd val funcJar = TestHive.getHiveFile("TestUDTF.jar") val jarURL = funcJar.toURI.toURL sql(s"ADD JAR $jarURL") - sql( - """CREATE TEMPORARY FUNCTION udtf_count2 AS - |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' - """.stripMargin) - assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1) - sql("DROP TEMPORARY FUNCTION udtf_count2") + withUserDefinedFunction("udtf_count2" -> true) { + sql( + """CREATE TEMPORARY FUNCTION udtf_count2 AS + |'org.apache.spark.sql.hive.execution.GenericUDTFCount2' + """.stripMargin) + assert(sql("DESCRIBE FUNCTION udtf_count2").count() > 1) + } } test("ADD FILE command") { @@ -1167,43 +1177,48 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd |DROP TABLE IF EXISTS dynamic_part_table; """.stripMargin) - ignore("Dynamic partition folder layout") { - sql("DROP TABLE IF EXISTS dynamic_part_table") - sql("CREATE TABLE dynamic_part_table(intcol INT) PARTITIONED BY (partcol1 INT, partcol2 INT)") - sql("SET hive.exec.dynamic.partition.mode=nonstrict") + test("Dynamic partition folder layout") { + withTempDir { dir => + withTable("dynamic_part_table") { + sql("CREATE TABLE dynamic_part_table(intcol INT) USING HIVE " + + s"PARTITIONED BY (partcol1 INT, partcol2 INT) " + + s"LOCATION '${dir.getCanonicalPath}/dynamic_part_table'") + sql("SET hive.exec.dynamic.partition.mode=nonstrict") + + val data = Map( + Seq("1", "1") -> 1, + Seq("1", "NULL") -> 2, + Seq("NULL", "1") -> 3, + Seq("NULL", "NULL") -> 4) + + data.foreach { case (parts, value) => + sql( + s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) + |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 + """.stripMargin) + + val partFolder = Seq("partcol1", "partcol2") + .zip(parts) + .map { case (k, v) => + if (v == "NULL") { + s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}" + } else { + s"$k=$v" + } + } + .mkString("/") - val data = Map( - Seq("1", "1") -> 1, - Seq("1", "NULL") -> 2, - Seq("NULL", "1") -> 3, - Seq("NULL", "NULL") -> 4) + // Loads partition data to a temporary table to verify contents + val path = s"${dir.getCanonicalPath}/dynamic_part_table/$partFolder/part-00000*" - data.foreach { case (parts, value) => - sql( - s"""INSERT INTO TABLE dynamic_part_table PARTITION(partcol1, partcol2) - |SELECT $value, ${parts.mkString(", ")} FROM src WHERE key=150 - """.stripMargin) - - val partFolder = Seq("partcol1", "partcol2") - .zip(parts) - .map { case (k, v) => - if (v == "NULL") { - s"$k=${ConfVars.DEFAULTPARTITIONNAME.defaultStrVal}" - } else { - s"$k=$v" + withTable("dp_verify") { + sql("CREATE TABLE dp_verify(intcol INT) USING HIVE") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") + + assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) } } - .mkString("/") - - // Loads partition data to a temporary table to verify contents - val warehousePathFile = new URI(sparkSession.getWarehousePath()).getPath - val path = s"$warehousePathFile/dynamic_part_table/$partFolder/part-00000" - - sql("DROP TABLE IF EXISTS dp_verify") - sql("CREATE TABLE dp_verify(intcol INT)") - sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") - - assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) + } } } @@ -1334,69 +1349,69 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } test("current_database with multiple sessions") { - sql("create database a") - sql("use a") - val s2 = newSession() - s2.sql("create database b") - s2.sql("use b") + withCurrentCatalogAndNamespace { + sql("create database a") + sql("use a") + val s2 = newSession() + s2.sql("create database b") + s2.sql("use b") - assert(sql("select current_database()").first() === Row("a")) - assert(s2.sql("select current_database()").first() === Row("b")) + assert(sql("select current_database()").first() === Row("a")) + assert(s2.sql("select current_database()").first() === Row("b")) - try { - sql("create table test_a(key INT, value STRING)") - s2.sql("create table test_b(key INT, value STRING)") + try { + sql("create table test_a(key INT, value STRING)") + s2.sql("create table test_b(key INT, value STRING)") - sql("select * from test_a") - checkError( - exception = intercept[AnalysisException] { - sql("select * from test_b") - }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", - parameters = Map("relationName" -> "`test_b`"), - context = ExpectedContext( - fragment = "test_b", - start = 14, - stop = 19)) + sql("select * from test_a") + checkError( + exception = intercept[AnalysisException] { + sql("select * from test_b") + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`test_b`"), + context = ExpectedContext( + fragment = "test_b", + start = 14, + stop = 19)) - sql("select * from b.test_b") + sql("select * from b.test_b") - s2.sql("select * from test_b") - checkError( - exception = intercept[AnalysisException] { - s2.sql("select * from test_a") - }, - errorClass = "TABLE_OR_VIEW_NOT_FOUND", - parameters = Map("relationName" -> "`test_a`"), - context = ExpectedContext( - fragment = "test_a", - start = 14, - stop = 19)) - s2.sql("select * from a.test_a") - } finally { - sql("DROP TABLE IF EXISTS test_a") - s2.sql("DROP TABLE IF EXISTS test_b") + s2.sql("select * from test_b") + checkError( + exception = intercept[AnalysisException] { + s2.sql("select * from test_a") + }, + errorClass = "TABLE_OR_VIEW_NOT_FOUND", + parameters = Map("relationName" -> "`test_a`"), + context = ExpectedContext( + fragment = "test_a", + start = 14, + stop = 19)) + s2.sql("select * from a.test_a") + } finally { + sql("DROP TABLE IF EXISTS test_a") + s2.sql("DROP TABLE IF EXISTS test_b") + } } - } test("use database") { val currentDatabase = sql("select current_database()").first().getString(0) + withCurrentCatalogAndNamespace { + sql("CREATE DATABASE hive_test_db") + sql("USE hive_test_db") + assert("hive_test_db" == sql("select current_database()").first().getString(0)) + assert("hive_test_db" == sql("select current_schema()").first().getString(0)) - sql("CREATE DATABASE hive_test_db") - sql("USE hive_test_db") - assert("hive_test_db" == sql("select current_database()").first().getString(0)) - - assert("hive_test_db" == sql("select current_schema()").first().getString(0)) - - checkError( - exception = intercept[AnalysisException] { - sql("USE not_existing_db") - }, - errorClass = "SCHEMA_NOT_FOUND", - parameters = Map("schemaName" -> "`spark_catalog`.`not_existing_db`")) - - sql(s"USE $currentDatabase") + checkError( + exception = intercept[AnalysisException] { + sql("USE not_existing_db") + }, + errorClass = "SCHEMA_NOT_FOUND", + parameters = Map("schemaName" -> "`spark_catalog`.`not_existing_db`") + ) + } assert(currentDatabase == sql("select current_database()").first().getString(0)) } @@ -1409,7 +1424,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sqlState = None, parameters = Map( "routineName" -> "`not_a_udf`", - "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`a`]"), + "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), context = ExpectedContext( fragment = "not_a_udf()", start = 0, @@ -1426,7 +1441,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sqlState = None, parameters = Map( "routineName" -> "`not_a_udf`", - "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`a`]"), + "searchPath" -> "[`system`.`builtin`, `system`.`session`, `spark_catalog`.`default`]"), context = ExpectedContext( fragment = "not_a_udf()", start = 0, From 920df93c41edb76adbc9e0148c7fd2dc44a17b03 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 12 Jul 2024 17:11:41 +0900 Subject: [PATCH 51/64] [SPARK-48877][PYTHON][DOCS] Test the default column name of array functions ### What changes were proposed in this pull request? Test the default column name of array functions ### Why are the changes needed? for test coverage, sometime the default column name is a problem ### Does this PR introduce _any_ user-facing change? doc changes ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #47318 from zhengruifeng/py_avoid_alias_array_func. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/functions/builtin.py | 201 ++++++++++++------------ 1 file changed, 98 insertions(+), 103 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 446ff2b1be93d..0b464aa207103 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -13443,39 +13443,39 @@ def array( >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", "doctor"), ("Bob", "engineer")], ... ("name", "occupation")) - >>> df.select(sf.array('name', 'occupation').alias("arr")).show() - +---------------+ - | arr| - +---------------+ - |[Alice, doctor]| - |[Bob, engineer]| - +---------------+ + >>> df.select(sf.array('name', 'occupation')).show() + +-----------------------+ + |array(name, occupation)| + +-----------------------+ + | [Alice, doctor]| + | [Bob, engineer]| + +-----------------------+ Example 2: Usage of array function with Column objects. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", "doctor"), ("Bob", "engineer")], ... ("name", "occupation")) - >>> df.select(sf.array(df.name, df.occupation).alias("arr")).show() - +---------------+ - | arr| - +---------------+ - |[Alice, doctor]| - |[Bob, engineer]| - +---------------+ + >>> df.select(sf.array(df.name, df.occupation)).show() + +-----------------------+ + |array(name, occupation)| + +-----------------------+ + | [Alice, doctor]| + | [Bob, engineer]| + +-----------------------+ Example 3: Single argument as list of column names. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", "doctor"), ("Bob", "engineer")], ... ("name", "occupation")) - >>> df.select(sf.array(['name', 'occupation']).alias("arr")).show() - +---------------+ - | arr| - +---------------+ - |[Alice, doctor]| - |[Bob, engineer]| - +---------------+ + >>> df.select(sf.array(['name', 'occupation'])).show() + +-----------------------+ + |array(name, occupation)| + +-----------------------+ + | [Alice, doctor]| + | [Bob, engineer]| + +-----------------------+ Example 4: Usage of array function with columns of different types. @@ -13483,26 +13483,26 @@ def array( >>> df = spark.createDataFrame( ... [("Alice", 2, 22.2), ("Bob", 5, 36.1)], ... ("name", "age", "weight")) - >>> df.select(sf.array(['age', 'weight']).alias("arr")).show() - +-----------+ - | arr| - +-----------+ - |[2.0, 22.2]| - |[5.0, 36.1]| - +-----------+ + >>> df.select(sf.array(['age', 'weight'])).show() + +------------------+ + |array(age, weight)| + +------------------+ + | [2.0, 22.2]| + | [5.0, 36.1]| + +------------------+ Example 5: array function with a column containing null values. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("Alice", None), ("Bob", "engineer")], ... ("name", "occupation")) - >>> df.select(sf.array('name', 'occupation').alias("arr")).show() - +---------------+ - | arr| - +---------------+ - | [Alice, NULL]| - |[Bob, engineer]| - +---------------+ + >>> df.select(sf.array('name', 'occupation')).show() + +-----------------------+ + |array(name, occupation)| + +-----------------------+ + | [Alice, NULL]| + | [Bob, engineer]| + +-----------------------+ """ if len(cols) == 1 and isinstance(cols[0], (list, set)): cols = cols[0] # type: ignore[assignment] @@ -13540,13 +13540,13 @@ def array_contains(col: "ColumnOrName", value: Any) -> Column: >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(["a", "b", "c"],), ([],)], ['data']) - >>> df.select(sf.array_contains(df.data, "a").alias("contains_a")).show() - +----------+ - |contains_a| - +----------+ - | true| - | false| - +----------+ + >>> df.select(sf.array_contains(df.data, "a")).show() + +-----------------------+ + |array_contains(data, a)| + +-----------------------+ + | true| + | false| + +-----------------------+ Example 2: Usage of array_contains function with a column. @@ -13554,38 +13554,37 @@ def array_contains(col: "ColumnOrName", value: Any) -> Column: >>> df = spark.createDataFrame([(["a", "b", "c"], "c"), ... (["c", "d", "e"], "d"), ... (["e", "a", "c"], "b")], ["data", "item"]) - >>> df.select(sf.array_contains(df.data, sf.col("item")) - ... .alias("data_contains_item")).show() - +------------------+ - |data_contains_item| - +------------------+ - | true| - | true| - | false| - +------------------+ + >>> df.select(sf.array_contains(df.data, sf.col("item"))).show() + +--------------------------+ + |array_contains(data, item)| + +--------------------------+ + | true| + | true| + | false| + +--------------------------+ Example 3: Attempt to use array_contains function with a null array. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(None,), (["a", "b", "c"],)], ['data']) - >>> df.select(sf.array_contains(df.data, "a").alias("contains_a")).show() - +----------+ - |contains_a| - +----------+ - | NULL| - | true| - +----------+ + >>> df.select(sf.array_contains(df.data, "a")).show() + +-----------------------+ + |array_contains(data, a)| + +-----------------------+ + | NULL| + | true| + +-----------------------+ Example 4: Usage of array_contains with an array column containing null values. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(["a", None, "c"],)], ['data']) - >>> df.select(sf.array_contains(df.data, "a").alias("contains_a")).show() - +----------+ - |contains_a| - +----------+ - | true| - +----------+ + >>> df.select(sf.array_contains(df.data, "a")).show() + +-----------------------+ + |array_contains(data, a)| + +-----------------------+ + | true| + +-----------------------+ """ return _invoke_function_over_columns("array_contains", col, lit(value)) @@ -13620,49 +13619,49 @@ def arrays_overlap(a1: "ColumnOrName", a2: "ColumnOrName") -> Column: >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(["a", "b"], ["b", "c"]), (["a"], ["b", "c"])], ['x', 'y']) - >>> df.select(sf.arrays_overlap(df.x, df.y).alias("overlap")).show() - +-------+ - |overlap| - +-------+ - | true| - | false| - +-------+ + >>> df.select(sf.arrays_overlap(df.x, df.y)).show() + +--------------------+ + |arrays_overlap(x, y)| + +--------------------+ + | true| + | false| + +--------------------+ Example 2: Usage of arrays_overlap function with arrays containing null elements. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(["a", None], ["b", None]), (["a"], ["b", "c"])], ['x', 'y']) - >>> df.select(sf.arrays_overlap(df.x, df.y).alias("overlap")).show() - +-------+ - |overlap| - +-------+ - | NULL| - | false| - +-------+ + >>> df.select(sf.arrays_overlap(df.x, df.y)).show() + +--------------------+ + |arrays_overlap(x, y)| + +--------------------+ + | NULL| + | false| + +--------------------+ Example 3: Usage of arrays_overlap function with arrays that are null. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(None, ["b", "c"]), (["a"], None)], ['x', 'y']) - >>> df.select(sf.arrays_overlap(df.x, df.y).alias("overlap")).show() - +-------+ - |overlap| - +-------+ - | NULL| - | NULL| - +-------+ + >>> df.select(sf.arrays_overlap(df.x, df.y)).show() + +--------------------+ + |arrays_overlap(x, y)| + +--------------------+ + | NULL| + | NULL| + +--------------------+ Example 4: Usage of arrays_overlap on arrays with identical elements. >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([(["a", "b"], ["a", "b"]), (["a"], ["a"])], ['x', 'y']) - >>> df.select(sf.arrays_overlap(df.x, df.y).alias("overlap")).show() - +-------+ - |overlap| - +-------+ - | true| - | true| - +-------+ + >>> df.select(sf.arrays_overlap(df.x, df.y)).show() + +--------------------+ + |arrays_overlap(x, y)| + +--------------------+ + | true| + | true| + +--------------------+ """ return _invoke_function_over_columns("arrays_overlap", a1, a2) @@ -14669,23 +14668,19 @@ def array_insert(arr: "ColumnOrName", pos: Union["ColumnOrName", int], value: An Example 4: Inserting a NULL value >>> from pyspark.sql import functions as sf - >>> from pyspark.sql.types import StringType >>> df = spark.createDataFrame([(['a', 'b', 'c'],)], ['data']) - >>> df.select(sf.array_insert(df.data, 2, sf.lit(None).cast(StringType())) - ... .alias("result")).show() - +---------------+ - | result| - +---------------+ - |[a, NULL, b, c]| - +---------------+ + >>> df.select(sf.array_insert(df.data, 2, sf.lit(None))).show() + +---------------------------+ + |array_insert(data, 2, NULL)| + +---------------------------+ + | [a, NULL, b, c]| + +---------------------------+ Example 5: Inserting a value into a NULL array >>> from pyspark.sql import functions as sf >>> from pyspark.sql.types import ArrayType, IntegerType, StructType, StructField - >>> schema = StructType([ - ... StructField("data", ArrayType(IntegerType()), True) - ... ]) + >>> schema = StructType([StructField("data", ArrayType(IntegerType()), True)]) >>> df = spark.createDataFrame([(None,)], schema=schema) >>> df.select(sf.array_insert(df.data, 1, 5)).show() +------------------------+ From 2bec82c05895a1fb3e136cb922f595ae0a4242c6 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 12 Jul 2024 10:37:06 +0200 Subject: [PATCH 52/64] [MINOR][TESTS] Remove unused test jar (udf_noA.jar) ### What changes were proposed in this pull request? This jar was added in https://github.com/apache/spark/pull/42069 but moved in https://github.com/apache/spark/pull/43735. ### Why are the changes needed? To clean up a jar not used. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing tests should check ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47315 from HyukjinKwon/minor-cleanup-jar-2. Authored-by: Hyukjin Kwon Signed-off-by: Martin Grund --- connect/server/src/test/resources/udf_noA.jar | Bin 5545 -> 0 bytes 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 connect/server/src/test/resources/udf_noA.jar diff --git a/connect/server/src/test/resources/udf_noA.jar b/connect/server/src/test/resources/udf_noA.jar deleted file mode 100644 index 4d8c423ab6dfbffe0b0f065faa15de74e133e0eb..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 5545 zcma)A2{_c<_y5M&C&r&FjR_Uy6^8JZ-t5`e>|)H=%f4^f*9?hDBE}L*_I*!gL`Wo> zs3>HYC5D9mOdI;O9gpYvdfapGx%YFH&pBGE;fW+txquBs*VL7apNTTT*K%5pS~#wb^6$KepC;c5ZwmkInQ}UtXrRQ*0|w zfcr&PpI-mHLkHI~WNpCX&c*QDCoLx%21Act>!DNIVv>P?N=A@^!()#lWFR<}gdzqq zhs@(v5wrV^G1C_56a(@$?ckx_Zi(cqz2(v=6Kd?N0uhl65oT#HL{cz|?UJSrZ&YeV zy5;EFmhZ+IhAS!%+&kzw7_xjr&ZltS)kV>1An7s6hbjGTM>FlGO1kn(UDwVZl7^(b zr6PG0wPZ!U%N_s#vHcEBy02KFoKY6GR=)w2461M#If6=qpomkw zxkP8!gT40RX@*q=yMqmbMR!>Mx6~y}tA|g^V+Sft=*OMddOVz?3EBg2Tr<))B5y4X zsYHr%T1*`7XB&6$<_Z)JieIu2%T4b`#WmushoD%VFtWi5v zExABnU39>vH#zARrr$2}g{@|;Re-nqMVD8cABBQ1>I5H;IF8>E&LWJlHyERaJBDfA zZAEcWhj1-kY~ml>8_E?0JA954G$5bb4jvRMZ|rWBCIX>xoxL$P!$VXNY8x!xb5}&r z7|g&4zJH7J3$g}8WCP`79EtZMW-+bv%2fu~@nc%^l2 zgzK$3vf47G{?n@v#8uhCNG!vL zH27p$Xie#>1q^59aOzRZR525oD(Xr-$j)cH4lLPMDmJf}y(Qq?9UY2ioAuDI$+0(4 z2WsSy2%srH!wZZA04l3QEBz*6fsC=vQsK--sFIiQN9Xl>tpgIJcsj@fevo5AoYZ_) zwf?%LvCPm5fyIMDOMdIhyq+^SY#T9r^W6okhP#$7`;8D13QGdHvG@h-Yh?cef^RRc zS5?5oBU}Qu8!b^o`z3tipr6Ax_i7#+CiNL9eARh*0-U|w1N!{w5wdGJA=comp>^RD zLW{k^3K!UNmAWiY4g!*a3=KCnb)=9$7WfYGv{-rCm+&Pae%==LxQEg>UEMt|2F-%) zpu!405fbVo`oP*fJ|^6KV)$CFZ;gFi{{nK=Y$e4sXLM)bp`T& zOtclTJvD$xWEc7Sqm-Y35{jv-dQ2W2rFf3Gx$rbhdS6-7OU*| zeHKinc+2~>ve;~gl+!FWi-||oDA{idZTuVn(?fsspzTP z-rA&FsiISveKTsg`@SF^NMm+rtuVv}tXM*@)8#~G`Ibd$AI&)i>#`sDny|lywWb_zR6)jV%hBnru_;nmZxO?K zwvghUP{d+|zU3t9HQ+23*`gklBS>dAuOs*w@mN46AMHd!+;_jhN4GyB4*L~+7EX?i zRu=BRL7w+dsyjU3Tfwf~oz(AsP3i-gEy=qJcH_?rHmHeMuxp4(eNSs?n7EBNOip%& zS9@sh^*Z1Df{Ia&mORW0c9tTIgyorDlMkHKTD(oy)VldxuVn; z$VvDUs@0PJw_tzxfqotS6Nnb}=dB!fy1|b-2>j&^JG|i(eKQ;)Zxc(WDk{kL${BD( zo_R!54Nw@Q6MDD@7)zPYYVm~lBrNMk2cGB91n~3KWCe)Q0jylYwgp=kL5bpBxCrYOwq#O zh8fcaMaOVM&dy_N>ryDOEjsQ1HuQ)G$2>i#gTBafdMaXBReBy1p6h6z!T;)IEj4w? zK26edUI3yTN{Ltf#(%!haaZZxZ1&V&K8B9Fhq;_R%FRvQ!@9s!~S=i^p1(IF=5Tvrd5Be0_E?K+O z#K<=5k|umlHw*L-y&`>q-y=5o20s6`xcdc<0hPp{TvuB@UhXA?tBv@LjymOk(%P$) zgO9b`K-4LRubyZgTN+osoI>wMJIBoEeexERg-lgYgk!wWckYbA?S&U(u=jX#Xa0Nd z`-=+lTkB1{jnhSIi?@dL58gb2h_7Sb@U*ZmX{^<|eAVi8OA(sz?+F9v=Bw=>Gt15+ zlkl6himS2+6Y80+wUb6|-6iMz{=1}+a7$|DMSK;nTP7x}nb%+V+=~U4ks7^tI`7UN z#klePCfCx2yd{U$P&#;bKNBWPgg5L-4gDy-LdgX3-}CTC&7$KfN+D`GBWSVZc0%mU zpe9pJ1bVz*{xU|~fTua*zTeTg>9}(Wk>hM z-o$*kfqi-JF|L~p$nre@oOC>);02!hc13bUmM-x^zc5ZS) zPgU##Sl|9l`K;$c`O(sb#Jc=`u^f7`78wX-k8M{3OeN=Ix5-omy^?O&6Rx8{bP+Vw z1*iKgg|~7Kg^G$(W(nJt9}I&z3X@@Y?l?{?UruW><`Gd6=gYMu8{NBE%mCYR8c+4P z-DA#jbjE>lY0@PTj zBw$DMv`Po^Ubh8YK&=<YCA?B~6)Xr6=p36`-&aXZY9Odsz^PLxaf5`<~J>JyTYP{W9{HGfi?xWX%&r$a+-RU9qpdPvF7u9hizjkk7X-x z4YFFpW25HxjT#Cpu^DlLPUS;-D0D90Q;>6$zV|32*eNJKW>cVv*XE{m^&qE0D*m(T4W0oH0#Pm!sy&l+1wdD!6cFyc)59D>aH$$#??5%0@eZ}QG zH<3d2R_RqNt-DN^>6<(CPBTW-<8PRjsE4b-zTh{jJjRD_G0vQx972w=4u+dgSHue6 zLGnw`!)aNIy^vj50xwxd-=4pM#(*wGWX{OWgmfaGTzn}^=)gSTdK=-A)YRBMKW`NY zX*Q{3YC^sXDti?sg}iWNg+)>d9oHQiZe3qoVr4jOs**KXbw%TTc)aInNmKzW(JH`4>C=i*62*}=0T1CwnTA%2oHzSrO19Y*Tkw Date: Fri, 12 Jul 2024 17:52:48 +0900 Subject: [PATCH 53/64] [SPARK-48794][CONNECT] df.mergeInto support for Spark Connect (Scala and Python) ### What changes were proposed in this pull request? This PR introduces `df.mergeInto` support for Spark Connect Scala and Python clients. This work contains four components: 1. New Protobuf messages: command `MergeIntoTableCommand` and expression `MergeAction`. 2. Spark Connect planner change: translate proto messages into real `MergeIntoCommand`s. 3. Connect Scala client: `MetgeIntoWriter` that allows users to build merges. 4. Connect Python client: `MetgeIntoWriter` that allows users to build merges. Components 3 and 4 and independent to each other. They both depends on Component 1. ### Why are the changes needed? We need to increase the functionality of Spark Connect to be on par with Classic. ### Does this PR introduce _any_ user-facing change? Yes, new Dataframe APIs are introduced. ### How was this patch tested? Added new tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46960 from xupefei/merge-builder. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../protobuf/spark/connect/commands.proto | 24 + .../protobuf/spark/connect/expressions.proto | 29 ++ .../connect/planner/SparkConnectPlanner.scala | 60 ++- .../scala/org/apache/spark/sql/Dataset.scala | 33 ++ .../apache/spark/sql/MergeIntoWriter.scala | 411 ++++++++++++++++++ .../spark/sql/MergeIntoE2ETestSuite.scala | 187 ++++++++ .../CheckConnectJvmClientCompatibility.scala | 32 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/sql/connect/dataframe.py | 10 +- python/pyspark/sql/connect/merge.py | 253 +++++++++++ .../pyspark/sql/connect/proto/commands_pb2.py | 198 ++++----- .../sql/connect/proto/commands_pb2.pyi | 98 +++++ .../sql/connect/proto/expressions_pb2.py | 138 +++--- .../sql/connect/proto/expressions_pb2.pyi | 109 +++++ python/pyspark/sql/dataframe.py | 4 + python/pyspark/sql/merge.py | 1 - .../tests/connect/test_parity_dataframe.py | 4 - 17 files changed, 1407 insertions(+), 186 deletions(-) create mode 100644 connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala create mode 100644 connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/MergeIntoE2ETestSuite.scala create mode 100644 python/pyspark/sql/connect/merge.py diff --git a/connect/common/src/main/protobuf/spark/connect/commands.proto b/connect/common/src/main/protobuf/spark/connect/commands.proto index 0e0c55fa34f00..b861598ad39af 100644 --- a/connect/common/src/main/protobuf/spark/connect/commands.proto +++ b/connect/common/src/main/protobuf/spark/connect/commands.proto @@ -47,6 +47,7 @@ message Command { CreateResourceProfileCommand create_resource_profile_command = 13; CheckpointCommand checkpoint_command = 14; RemoveCachedRemoteRelationCommand remove_cached_remote_relation_command = 15; + MergeIntoTableCommand merge_into_table_command = 16; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // Commands they can add them here. During the planning the correct resolution is done. @@ -504,3 +505,26 @@ message CheckpointCommand { // (Required) Whether to checkpoint this dataframe immediately. bool eager = 3; } + +message MergeIntoTableCommand { + // (Required) The name of the target table. + string target_table_name = 1; + + // (Required) The relation of the source table. + Relation source_table_plan = 2; + + // (Required) The condition to match the source and target. + Expression merge_condition = 3; + + // (Optional) The actions to be taken when the condition is matched. + repeated Expression match_actions = 4; + + // (Optional) The actions to be taken when the condition is not matched. + repeated Expression not_matched_actions = 5; + + // (Optional) The actions to be taken when the condition is not matched by source. + repeated Expression not_matched_by_source_actions = 6; + + // (Required) Whether to enable schema evolution. + bool with_schema_evolution = 7; +} diff --git a/connect/common/src/main/protobuf/spark/connect/expressions.proto b/connect/common/src/main/protobuf/spark/connect/expressions.proto index 257634813e742..860e923576161 100644 --- a/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -50,6 +50,7 @@ message Expression { CommonInlineUserDefinedFunction common_inline_user_defined_function = 15; CallFunction call_function = 16; NamedArgumentExpression named_argument_expression = 17; + MergeAction merge_action = 19; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. @@ -416,3 +417,31 @@ message NamedArgumentExpression { // (Required) The value expression of the named argument. Expression value = 2; } + +message MergeAction { + // (Required) The action type of the merge action. + ActionType action_type = 1; + + // (Optional) The condition expression of the merge action. + optional Expression condition = 2; + + // (Optional) The assignments of the merge action. Required for ActionTypes INSERT and UPDATE. + repeated Assignment assignments = 3; + + enum ActionType { + ACTION_TYPE_INVALID = 0; + ACTION_TYPE_DELETE = 1; + ACTION_TYPE_INSERT = 2; + ACTION_TYPE_INSERT_STAR = 3; + ACTION_TYPE_UPDATE = 4; + ACTION_TYPE_UPDATE_STAR = 5; + } + + message Assignment { + // (Required) The key of the assignment. + Expression key = 1; + + // (Required) The value of the assignment. + Expression value = 2; + } +} diff --git a/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 93a01ea6c5740..449e923beae3a 100644 --- a/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -55,7 +55,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftAnti, LeftOuter, LeftSemi, RightOuter, UsingJoin} import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.{AppendColumns, CoGroup, CollectMetrics, CommandResult, Deduplicate, DeduplicateWithinWatermark, DeserializeToObject, Except, FlatMapGroupsWithState, Intersect, JoinWith, LocalRelation, LogicalGroupState, LogicalPlan, MapGroups, MapPartitions, Project, Sample, SerializeFromObject, Sort, SubqueryAlias, TypedFilter, Union, Unpivot, UnresolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{AppendColumns, Assignment, CoGroup, CollectMetrics, CommandResult, Deduplicate, DeduplicateWithinWatermark, DeleteAction, DeserializeToObject, Except, FlatMapGroupsWithState, InsertAction, InsertStarAction, Intersect, JoinWith, LocalRelation, LogicalGroupState, LogicalPlan, MapGroups, MapPartitions, MergeAction, Project, Sample, SerializeFromObject, Sort, SubqueryAlias, TypedFilter, Union, Unpivot, UnresolvedHint, UpdateAction, UpdateStarAction} import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.catalyst.trees.PySparkCurrentOrigin import org.apache.spark.sql.catalyst.types.DataTypeUtils @@ -1521,6 +1521,8 @@ class SparkConnectPlanner( transformCallFunction(exp.getCallFunction) case proto.Expression.ExprTypeCase.NAMED_ARGUMENT_EXPRESSION => transformNamedArgumentExpression(exp.getNamedArgumentExpression) + case proto.Expression.ExprTypeCase.MERGE_ACTION => + transformMergeAction(exp.getMergeAction) case _ => throw InvalidPlanInput( s"Expression with ID: ${exp.getExprTypeCase.getNumber} is not supported") @@ -2586,6 +2588,36 @@ class SparkConnectPlanner( } } + private def transformMergeAction(action: proto.MergeAction): MergeAction = { + val condition = if (action.hasCondition) { + Some(transformExpression(action.getCondition)) + } else { + None + } + val assignments = action.getAssignmentsList.asScala.map { assignment => + val key = transformExpression(assignment.getKey) + val value = transformExpression(assignment.getValue) + Assignment(key, value) + }.toSeq + action.getActionType match { + case proto.MergeAction.ActionType.ACTION_TYPE_DELETE => + assert(assignments.isEmpty, "Delete action should not have assignment.") + DeleteAction(condition) + case proto.MergeAction.ActionType.ACTION_TYPE_INSERT => + InsertAction(condition, assignments) + case proto.MergeAction.ActionType.ACTION_TYPE_INSERT_STAR => + assert(assignments.isEmpty, "InsertStar action should not have assignment.") + InsertStarAction(condition) + case proto.MergeAction.ActionType.ACTION_TYPE_UPDATE => + UpdateAction(condition, assignments) + case proto.MergeAction.ActionType.ACTION_TYPE_UPDATE_STAR => + assert(assignments.isEmpty, "UpdateStar action should not have assignment.") + UpdateStarAction(condition) + case _ => + throw InvalidPlanInput(s"Unsupported merge action type ${action.getActionType}.") + } + } + def process( command: proto.Command, responseObserver: StreamObserver[ExecutePlanResponse]): Unit = { @@ -2629,6 +2661,8 @@ class SparkConnectPlanner( handleCheckpointCommand(command.getCheckpointCommand, responseObserver) case proto.Command.CommandTypeCase.REMOVE_CACHED_REMOTE_RELATION_COMMAND => handleRemoveCachedRemoteRelationCommand(command.getRemoveCachedRemoteRelationCommand) + case proto.Command.CommandTypeCase.MERGE_INTO_TABLE_COMMAND => + handleMergeIntoTableCommand(command.getMergeIntoTableCommand) case _ => throw new UnsupportedOperationException(s"$command not supported.") } @@ -3596,6 +3630,30 @@ class SparkConnectPlanner( executeHolder.eventsManager.postFinished() } + private def handleMergeIntoTableCommand(cmd: proto.MergeIntoTableCommand): Unit = { + def transformActions(actions: java.util.List[proto.Expression]): Seq[MergeAction] = + actions.asScala.map(transformExpression).map(_.asInstanceOf[MergeAction]).toSeq + + val matchedActions = transformActions(cmd.getMatchActionsList) + val notMatchedActions = transformActions(cmd.getNotMatchedActionsList) + val notMatchedBySourceActions = transformActions(cmd.getNotMatchedBySourceActionsList) + + val sourceDs = Dataset.ofRows(session, transformRelation(cmd.getSourceTablePlan)) + var mergeInto = sourceDs + .mergeInto(cmd.getTargetTableName, Column(transformExpression(cmd.getMergeCondition))) + .withNewMatchedActions(matchedActions: _*) + .withNewNotMatchedActions(notMatchedActions: _*) + .withNewNotMatchedBySourceActions(notMatchedBySourceActions: _*) + + mergeInto = if (cmd.getWithSchemaEvolution) { + mergeInto.withSchemaEvolution() + } else { + mergeInto + } + mergeInto.merge() + executeHolder.eventsManager.postFinished() + } + private val emptyLocalRelation = LocalRelation( output = AttributeReference("value", StringType, false)() :: Nil, data = Seq.empty) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index e831c264e632a..d6cea3723f956 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders._ import org.apache.spark.sql.catalyst.expressions.OrderUtils import org.apache.spark.sql.connect.client.SparkResult import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, StorageLevelProtoConverter, UdfUtils} +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId import org.apache.spark.sql.expressions.ScalaUserDefinedFunction import org.apache.spark.sql.functions.{struct, to_json} import org.apache.spark.sql.streaming.DataStreamWriter @@ -3212,6 +3213,38 @@ class Dataset[T] private[sql] ( new DataFrameWriterV2[T](table, this) } + /** + * Merges a set of updates, insertions, and deletions based on a source table into a target + * table. + * + * Scala Examples: + * {{{ + * spark.table("source") + * .mergeInto("target", $"source.id" === $"target.id") + * .whenMatched($"salary" === 100) + * .delete() + * .whenNotMatched() + * .insertAll() + * .whenNotMatchedBySource($"salary" === 100) + * .update(Map( + * "salary" -> lit(200) + * )) + * .merge() + * }}} + * + * @group basic + * @since 4.0.0 + */ + def mergeInto(table: String, condition: Column): MergeIntoWriter[T] = { + if (isStreaming) { + throw new AnalysisException( + errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED", + messageParameters = Map("methodName" -> toSQLId("mergeInto"))) + } + + new MergeIntoWriter[T](table, this, condition) + } + /** * Interface for saving the content of the streaming Dataset out into external storage. * diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala new file mode 100644 index 0000000000000..197cfac32a4b0 --- /dev/null +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala @@ -0,0 +1,411 @@ +/* + * 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 + +import scala.jdk.CollectionConverters._ + +import org.apache.spark.SparkRuntimeException +import org.apache.spark.annotation.Experimental +import org.apache.spark.connect.proto +import org.apache.spark.connect.proto.{Expression, MergeIntoTableCommand} +import org.apache.spark.connect.proto.MergeAction +import org.apache.spark.sql.MergeIntoWriter.buildMergeAction +import org.apache.spark.sql.functions.expr + +/** + * `MergeIntoWriter` provides methods to define and execute merge actions based on specified + * conditions. + * + * @tparam T + * the type of data in the Dataset. + * @param table + * the name of the target table for the merge operation. + * @param ds + * the source Dataset to merge into the target table. + * @param on + * the merge condition. + * @param schemaEvolutionEnabled + * whether to enable automatic schema evolution for this merge operation. Default is `false`. + * + * @since 4.0.0 + */ +@Experimental +class MergeIntoWriter[T] private[sql] ( + table: String, + ds: Dataset[T], + on: Column, + schemaEvolutionEnabled: Boolean = false) { + + private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction] + private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction] + private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction] + + /** + * Initialize a `WhenMatched` action without any condition. + * + * This `WhenMatched` action will be executed when a source row matches a target table row based + * on the merge condition. + * + * This `WhenMatched` can be followed by one of the following merge actions: + * - `updateAll`: Update all the matched target table rows with source dataset rows. + * - `update(Map)`: Update all the matched target table rows while changing only a subset of + * columns based on the provided assignment. + * - `delete`: Delete all target rows that have a match in the source table. + * + * @return + * a new `WhenMatched` object. + */ + def whenMatched(): WhenMatched[T] = { + new WhenMatched[T](this, None) + } + + /** + * Initialize a `WhenMatched` action with a condition. + * + * This `WhenMatched` action will be executed when a source row matches a target table row based + * on the merge condition and the specified `condition` is satisfied. + * + * This `WhenMatched` can be followed by one of the following merge actions: + * - `updateAll`: Update all the matched target table rows with source dataset rows. + * - `update(Map)`: Update all the matched target table rows while changing only a subset of + * columns based on the provided assignment. + * - `delete`: Delete all target rows that have a match in the source table. + * + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenMatched` object configured with the specified condition. + */ + def whenMatched(condition: Column): WhenMatched[T] = { + new WhenMatched[T](this, Some(condition.expr)) + } + + /** + * Initialize a `WhenNotMatched` action without any condition. + * + * This `WhenNotMatched` action will be executed when a source row does not match any target row + * based on the merge condition. + * + * This `WhenNotMatched` can be followed by one of the following merge actions: + * - `insertAll`: Insert all rows from the source that are not already in the target table. + * - `insert(Map)`: Insert all rows from the source that are not already in the target table, + * with the specified columns based on the provided assignment. + * + * @return + * a new `WhenNotMatched` object. + */ + def whenNotMatched(): WhenNotMatched[T] = { + new WhenNotMatched[T](this, None) + } + + /** + * Initialize a `WhenNotMatched` action with a condition. + * + * This `WhenNotMatched` action will be executed when a source row does not match any target row + * based on the merge condition and the specified `condition` is satisfied. + * + * This `WhenNotMatched` can be followed by one of the following merge actions: + * - `insertAll`: Insert all rows from the source that are not already in the target table. + * - `insert(Map)`: Insert all rows from the source that are not already in the target table, + * with the specified columns based on the provided assignment. + * + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenNotMatched` object configured with the specified condition. + */ + def whenNotMatched(condition: Column): WhenNotMatched[T] = { + new WhenNotMatched[T](this, Some(condition.expr)) + } + + /** + * Initialize a `WhenNotMatchedBySource` action without any condition. + * + * This `WhenNotMatchedBySource` action will be executed when a target row does not match any + * rows in the source table based on the merge condition. + * + * This `WhenNotMatchedBySource` can be followed by one of the following merge actions: + * - `updateAll`: Update all the not matched target table rows with source dataset rows. + * - `update(Map)`: Update all the not matched target table rows while changing only the + * specified columns based on the provided assignment. + * - `delete`: Delete all target rows that have no matches in the source table. + * + * @return + * a new `WhenNotMatchedBySource` object. + */ + def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = { + new WhenNotMatchedBySource[T](this, None) + } + + /** + * Initialize a `WhenNotMatchedBySource` action with a condition. + * + * This `WhenNotMatchedBySource` action will be executed when a target row does not match any + * rows in the source table based on the merge condition and the specified `condition` is + * satisfied. + * + * This `WhenNotMatchedBySource` can be followed by one of the following merge actions: + * - `updateAll`: Update all the not matched target table rows with source dataset rows. + * - `update(Map)`: Update all the not matched target table rows while changing only the + * specified columns based on the provided assignment. + * - `delete`: Delete all target rows that have no matches in the source table. + * + * @param condition + * a `Column` representing the condition to be evaluated for the action. + * @return + * a new `WhenNotMatchedBySource` object configured with the specified condition. + */ + def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = { + new WhenNotMatchedBySource[T](this, Some(condition.expr)) + } + + /** + * Enable automatic schema evolution for this merge operation. + * @return + * A `MergeIntoWriter` instance with schema evolution enabled. + */ + def withSchemaEvolution(): MergeIntoWriter[T] = { + new MergeIntoWriter[T](this.table, this.ds, this.on, schemaEvolutionEnabled = true) + .withNewMatchedActions(this.matchedActions: _*) + .withNewNotMatchedActions(this.notMatchedActions: _*) + .withNewNotMatchedBySourceActions(this.notMatchedBySourceActions: _*) + } + + /** + * Executes the merge operation. + */ + def merge(): Unit = { + if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) { + throw new SparkRuntimeException( + errorClass = "NO_MERGE_ACTION_SPECIFIED", + messageParameters = Map.empty) + } + + val matchedActionExpressions = + matchedActions.map(Expression.newBuilder().setMergeAction(_)).map(_.build()) + val notMatchedActionExpressions = + notMatchedActions.map(Expression.newBuilder().setMergeAction(_)).map(_.build()) + val notMatchedBySourceActionExpressions = + notMatchedBySourceActions.map(Expression.newBuilder().setMergeAction(_)).map(_.build()) + val mergeIntoCommand = MergeIntoTableCommand + .newBuilder() + .setTargetTableName(table) + .setSourceTablePlan(ds.plan.getRoot) + .setMergeCondition(on.expr) + .addAllMatchActions(matchedActionExpressions.asJava) + .addAllNotMatchedActions(notMatchedActionExpressions.asJava) + .addAllNotMatchedBySourceActions(notMatchedBySourceActionExpressions.asJava) + .setWithSchemaEvolution(schemaEvolutionEnabled) + .build() + + ds.sparkSession.execute( + proto.Command + .newBuilder() + .setMergeIntoTableCommand(mergeIntoCommand) + .build()) + } + + private[sql] def withNewMatchedActions(action: MergeAction*): MergeIntoWriter[T] = { + this.matchedActions = this.matchedActions :++ action + this + } + + private[sql] def withNewNotMatchedActions(action: MergeAction*): MergeIntoWriter[T] = { + this.notMatchedActions = this.notMatchedActions :++ action + this + } + + private[sql] def withNewNotMatchedBySourceActions(action: MergeAction*): MergeIntoWriter[T] = { + this.notMatchedBySourceActions = this.notMatchedBySourceActions :++ action + this + } +} + +/** + * A class for defining actions to be taken when matching rows in a DataFrame during a merge + * operation. + * + * @param mergeIntoWriter + * The MergeIntoWriter instance responsible for writing data to a target DataFrame. + * @param condition + * An optional condition Expression that specifies when the actions should be applied. If the + * condition is None, the actions will be applied to all matched rows. + * + * @tparam T + * The type of data in the MergeIntoWriter. + */ +case class WhenMatched[T] private[sql] ( + mergeIntoWriter: MergeIntoWriter[T], + condition: Option[Expression]) { + + /** + * Specifies an action to update all matched rows in the DataFrame. + * + * @return + * The MergeIntoWriter instance with the update all action configured. + */ + def updateAll(): MergeIntoWriter[T] = { + mergeIntoWriter.withNewMatchedActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_UPDATE_STAR, condition)) + } + + /** + * Specifies an action to update matched rows in the DataFrame with the provided column + * assignments. + * + * @param map + * A Map of column names to Column expressions representing the updates to be applied. + * @return + * The MergeIntoWriter instance with the update action configured. + */ + def update(map: Map[String, Column]): MergeIntoWriter[T] = { + mergeIntoWriter.withNewMatchedActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_UPDATE, condition, Some(map))) + } + + /** + * Specifies an action to delete matched rows from the DataFrame. + * + * @return + * The MergeIntoWriter instance with the delete action configured. + */ + def delete(): MergeIntoWriter[T] = { + mergeIntoWriter.withNewMatchedActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_DELETE, condition)) + } +} + +/** + * A class for defining actions to be taken when no matching rows are found in a DataFrame during + * a merge operation. + * + * @param mergeIntoWriter + * The MergeIntoWriter instance responsible for writing data to a target DataFrame. + * @param condition + * An optional condition Expression that specifies when the actions defined in this + * configuration should be applied. If the condition is None, the actions will be applied when + * there are no matching rows. + * @tparam T + * The type of data in the MergeIntoWriter. + */ +case class WhenNotMatched[T] private[sql] ( + mergeIntoWriter: MergeIntoWriter[T], + condition: Option[Expression]) { + + /** + * Specifies an action to insert all non-matched rows into the DataFrame. + * + * @return + * The MergeIntoWriter instance with the insert all action configured. + */ + def insertAll(): MergeIntoWriter[T] = { + mergeIntoWriter.withNewNotMatchedActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_INSERT_STAR, condition)) + } + + /** + * Specifies an action to insert non-matched rows into the DataFrame with the provided column + * assignments. + * + * @param map + * A Map of column names to Column expressions representing the values to be inserted. + * @return + * The MergeIntoWriter instance with the insert action configured. + */ + def insert(map: Map[String, Column]): MergeIntoWriter[T] = { + mergeIntoWriter.withNewNotMatchedActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_INSERT, condition, Some(map))) + + } +} + +/** + * A class for defining actions to be performed when there is no match by source during a merge + * operation in a MergeIntoWriter. + * + * @param mergeIntoWriter + * the MergeIntoWriter instance to which the merge actions will be applied. + * @param condition + * an optional condition to be used with the merge actions. + * @tparam T + * the type parameter for the MergeIntoWriter. + */ +case class WhenNotMatchedBySource[T] private[sql] ( + mergeIntoWriter: MergeIntoWriter[T], + condition: Option[Expression]) { + + /** + * Specifies an action to update all non-matched rows in the target DataFrame when not matched + * by the source. + * + * @return + * The MergeIntoWriter instance with the update all action configured. + */ + def updateAll(): MergeIntoWriter[T] = { + mergeIntoWriter.withNewNotMatchedBySourceActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_UPDATE_STAR, condition)) + } + + /** + * Specifies an action to update non-matched rows in the target DataFrame with the provided + * column assignments when not matched by the source. + * + * @param map + * A Map of column names to Column expressions representing the updates to be applied. + * @return + * The MergeIntoWriter instance with the update action configured. + */ + def update(map: Map[String, Column]): MergeIntoWriter[T] = { + mergeIntoWriter.withNewNotMatchedBySourceActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_UPDATE, condition, Some(map))) + } + + /** + * Specifies an action to delete non-matched rows from the target DataFrame when not matched by + * the source. + * + * @return + * The MergeIntoWriter instance with the delete action configured. + */ + def delete(): MergeIntoWriter[T] = { + mergeIntoWriter.withNewNotMatchedBySourceActions( + buildMergeAction(MergeAction.ActionType.ACTION_TYPE_DELETE, condition)) + } +} + +private object MergeIntoWriter { + private[sql] def buildMergeAction( + actionType: MergeAction.ActionType, + conditionOpt: Option[Expression], + assignmentsOpt: Option[Map[String, Column]] = None): MergeAction = { + val assignmentsProtoOpt = assignmentsOpt.map { + _.map { case (k, v) => + MergeAction.Assignment + .newBuilder() + .setKey(expr(k).expr) + .setValue(v.expr) + .build() + }.toSeq.asJava + } + + var builder = MergeAction.newBuilder().setActionType(actionType) + builder = conditionOpt.map(builder.setCondition).getOrElse(builder) + builder = assignmentsProtoOpt.map(builder.addAllAssignments).getOrElse(builder) + builder.build() + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/MergeIntoE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/MergeIntoE2ETestSuite.scala new file mode 100644 index 0000000000000..cdb72e3baf0e9 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/MergeIntoE2ETestSuite.scala @@ -0,0 +1,187 @@ +/* + * 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 + +import org.apache.spark.sql.functions.lit +import org.apache.spark.sql.test.{ConnectFunSuite, RemoteSparkSession} + +class MergeIntoE2ETestSuite extends ConnectFunSuite with RemoteSparkSession { + + lazy val session: SparkSession = spark + + override def beforeAll(): Unit = { + super.beforeAll() + spark.conf.set( + "spark.sql.catalog.testcat", + "org.apache.spark.sql.connector.catalog.InMemoryRowLevelOperationTableCatalog") + } + + private val sourceRows = Seq((1, 100, "hr"), (2, 200, "finance"), (3, 300, "hr")) + + private def withSourceView(f: String => Unit): Unit = { + import session.implicits._ + + val name = "source" + sourceRows.toDF("pk", "salary", "dep").createOrReplaceTempView(name) + try { + f(name) + } finally { + spark.sql(s"DROP VIEW IF EXISTS $name") + } + } + + private def withTargetTable(f: String => Unit): Unit = + withTargetTable("pk INT NOT NULL, salary INT, dep STRING")(f) + + private def withTargetTable(schema: String)(f: String => Unit): Unit = { + val name = "testcat.ns1.target" + spark.sql(s"CREATE OR REPLACE TABLE $name ($schema)") + try { + f(name) + } finally { + spark.sql(s"DROP TABLE IF EXISTS $name") + } + } + + private def checkTable(target: String, rows: Seq[(Int, Int, String)]): Unit = { + val actual = spark.sql(s"select * from $target").collect().sortBy(_.getInt(0)) + val expected = rows.sortBy(_._1).map(Row.fromTuple).toArray + assert(actual === expected) + } + + test("insert and update and delete") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (1, 100, 'hr'), (2, 200, 'finance')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenMatched($"$source.pk" === 2) + .delete() + .whenMatched() + .update(Map("salary" -> lit(999999))) + .whenNotMatched() + .insert(Map("pk" -> $"pk", "salary" -> $"salary", "dep" -> $"dep")) + .merge() + + checkTable(target, Seq((1, 999999, "hr"), (3, 300, "hr"))) + } + } + } + + test("insertAll and updateAll") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (1, 100, 'hr'), (2, 200, 'mgr')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenMatched() + .updateAll() + .whenNotMatched() + .insertAll() + .merge() + + checkTable(target, Seq((1, 100, "hr"), (2, 200, "finance"), (3, 300, "hr"))) + } + } + } + + test("insertAll and delete") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (1, 100, 'hr'), (2, 200, 'finance')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenMatched() + .delete() + .whenNotMatched() + .insertAll() + .merge() + + checkTable(target, Seq((3, 300, "hr"))) + } + } + } + + test("insertAll and update with condition") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (1, 100, 'hr'), (2, 200, 'finance')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenMatched($"$source.pk" === 1) + .update(Map("salary" -> lit(999999))) + .whenNotMatched() + .insertAll() + .merge() + + checkTable(target, Seq((1, 999999, "hr"), (2, 200, "finance"), (3, 300, "hr"))) + } + } + } + + test("insertAll and delete with condition") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (1, 100, 'hr'), (2, 200, 'finance')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenMatched($"$source.pk" === 1) + .delete() + .whenNotMatched() + .insertAll() + .merge() + + checkTable(target, Seq((2, 200, "finance"), (3, 300, "hr"))) + } + } + } + + test("when not matched by source") { + import session.implicits._ + + withSourceView { source => + withTargetTable { target => + spark.sql(s"INSERT INTO $target VALUES (9, 99, 'eng')") + spark + .table("source") + .mergeInto(target, $"$source.pk" === $"$target.pk") + .whenNotMatched() + .insertAll() + .whenNotMatchedBySource() + .delete() + .merge() + + checkTable(target, Seq((1, 100, "hr"), (2, 200, "finance"), (3, 300, "hr"))) + } + } + } +} diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala index 25e7050d3078e..a6be271b5c7d1 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala @@ -143,6 +143,21 @@ object CheckConnectJvmClientCompatibility { checkMiMaCompatibility(clientJar, protobufJar, includedRules, excludeRules) } + private lazy val mergeIntoWriterExcludeRules: Seq[ProblemFilter] = { + // Exclude some auto-generated methods in [[MergeIntoWriter]] classes. + // The incompatible changes are due to the uses of [[proto.Expression]] instead + // of [[catalyst.Expression]] in the method signature. + val classNames = Seq("WhenMatched", "WhenNotMatched", "WhenNotMatchedBySource") + val methodNames = Seq("apply", "condition", "copy", "copy$*", "unapply") + + classNames.flatMap { className => + methodNames.map { methodName => + ProblemFilters.exclude[IncompatibleSignatureProblem]( + s"org.apache.spark.sql.$className.$methodName") + } + } + } + private def checkMiMaCompatibilityWithSqlModule( clientJar: File, sqlJar: File): List[Problem] = { @@ -300,17 +315,9 @@ object CheckConnectJvmClientCompatibility { ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.artifact.util.ArtifactUtils"), ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.artifact.util.ArtifactUtils$"), - - // MergeIntoWriter - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.MergeIntoWriter"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.MergeIntoWriter$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenMatched"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenMatched$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenNotMatched"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenNotMatched$"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenNotMatchedBySource"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.WhenNotMatchedBySource$")) + "org.apache.spark.sql.artifact.util.ArtifactUtils$")) ++ + mergeIntoWriterExcludeRules + checkMiMaCompatibility(clientJar, sqlJar, includedRules, excludeRules) } @@ -464,7 +471,8 @@ object CheckConnectJvmClientCompatibility { // Encoders are in the wrong JAR ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders"), - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$")) + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$")) ++ + mergeIntoWriterExcludeRules checkMiMaCompatibility(sqlJar, clientJar, includedRules, excludeRules) } diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 34656ab23e0fd..8f4ed6014d77f 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -483,6 +483,7 @@ def __hash__(self): "pyspark.sql.group", "pyspark.sql.functions.builtin", "pyspark.sql.functions.partitioning", + "pyspark.sql.merge", "pyspark.sql.readwriter", "pyspark.sql.streaming.query", "pyspark.sql.streaming.readwriter", @@ -997,6 +998,7 @@ def __hash__(self): "pyspark.sql.connect.session", "pyspark.sql.connect.window", "pyspark.sql.connect.column", + "pyspark.sql.connect.merge", "pyspark.sql.connect.readwriter", "pyspark.sql.connect.dataframe", "pyspark.sql.connect.functions.builtin", diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 92cadb598c590..dd093fc19b343 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -23,7 +23,6 @@ ) from pyspark.resource import ResourceProfile from pyspark.sql.connect.utils import check_dependencies -from pyspark.sql.merge import MergeIntoWriter check_dependencies(__name__) @@ -74,6 +73,7 @@ import pyspark.sql.connect.plan as plan from pyspark.sql.connect.conversion import ArrowTableToRowsConversion from pyspark.sql.connect.group import GroupedData +from pyspark.sql.connect.merge import MergeIntoWriter from pyspark.sql.connect.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.connect.streaming.readwriter import DataStreamWriter from pyspark.sql.column import Column @@ -2188,9 +2188,11 @@ def cb(ei: "ExecutionInfo") -> None: return DataFrameWriterV2(self._plan, self._session, table, cb) def mergeInto(self, table: str, condition: Column) -> MergeIntoWriter: - raise PySparkNotImplementedError( - error_class="NOT_IMPLEMENTED", - message_parameters={"feature": "mergeInto"}, + def cb(ei: "ExecutionInfo") -> None: + self._execution_info = ei + + return MergeIntoWriter( + self._plan, self._session, table, condition, cb # type: ignore[arg-type] ) def offset(self, n: int) -> ParentDataFrame: diff --git a/python/pyspark/sql/connect/merge.py b/python/pyspark/sql/connect/merge.py new file mode 100644 index 0000000000000..9c3b3e4370a40 --- /dev/null +++ b/python/pyspark/sql/connect/merge.py @@ -0,0 +1,253 @@ +# +# 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. +# + +import sys +from typing import Dict, Optional, TYPE_CHECKING, List, Callable + +from pyspark.sql.connect import proto +from pyspark.sql.connect.column import Column +from pyspark.sql.connect.functions import expr +from pyspark.sql.merge import MergeIntoWriter as PySparkMergeIntoWriter + +if TYPE_CHECKING: + from pyspark.sql.connect.client import SparkConnectClient + from pyspark.sql.connect.plan import LogicalPlan + from pyspark.sql.connect.session import SparkSession + from pyspark.sql.metrics import ExecutionInfo + +__all__ = ["MergeIntoWriter"] + + +def _build_merge_action( + client: "SparkConnectClient", + action_type: proto.MergeAction.ActionType.ValueType, + condition: Optional[Column] = None, + assignments: Optional[Dict[str, Column]] = None, +) -> proto.MergeAction: + if assignments is None: + proto_assignments = None + else: + proto_assignments = [ + proto.MergeAction.Assignment( + key=expr(k).to_plan(client), value=v.to_plan(client) # type: ignore[operator] + ) + for k, v in assignments.items() + ] + return proto.MergeAction( + action_type=action_type, + condition=None if condition is None else condition.to_plan(client), + assignments=proto_assignments, + ) + + +class MergeIntoWriter: + def __init__( + self, + plan: "LogicalPlan", + session: "SparkSession", + table: str, + condition: Column, + callback: Optional[Callable[["ExecutionInfo"], None]] = None, + ): + self._client = session.client + self._target_table = table + self._source_plan = plan + self._condition = condition + + self._callback = callback if callback is not None else lambda _: None + self._schema_evolution_enabled = False + self._matched_actions = list() # type: List[proto.MergeAction] + self._not_matched_actions = list() # type: List[proto.MergeAction] + self._not_matched_by_source_actions = list() # type: List[proto.MergeAction] + + def whenMatched(self, condition: Optional[Column] = None) -> "MergeIntoWriter.WhenMatched": + return self.WhenMatched(self, condition) + + whenMatched.__doc__ = PySparkMergeIntoWriter.whenMatched.__doc__ + + def whenNotMatched( + self, condition: Optional[Column] = None + ) -> "MergeIntoWriter.WhenNotMatched": + return self.WhenNotMatched(self, condition) + + whenNotMatched.__doc__ = PySparkMergeIntoWriter.whenNotMatched.__doc__ + + def whenNotMatchedBySource( + self, condition: Optional[Column] = None + ) -> "MergeIntoWriter.WhenNotMatchedBySource": + return self.WhenNotMatchedBySource(self, condition) + + whenNotMatchedBySource.__doc__ = PySparkMergeIntoWriter.whenNotMatchedBySource.__doc__ + + def withSchemaEvolution(self) -> "MergeIntoWriter": + self._schema_evolution_enabled = True + return self + + withSchemaEvolution.__doc__ = PySparkMergeIntoWriter.withSchemaEvolution.__doc__ + + def merge(self) -> None: + def a2e(a: proto.MergeAction) -> proto.Expression: + return proto.Expression(merge_action=a) + + merge = proto.MergeIntoTableCommand( + target_table_name=self._target_table, + source_table_plan=self._source_plan.plan(self._client), + merge_condition=self._condition.to_plan(self._client), + match_actions=[a2e(a) for a in self._matched_actions], + not_matched_actions=[a2e(a) for a in self._not_matched_actions], + not_matched_by_source_actions=[a2e(a) for a in self._not_matched_by_source_actions], + with_schema_evolution=self._schema_evolution_enabled, + ) + _, _, ei = self._client.execute_command( + proto.Command(merge_into_table_command=merge), self._source_plan.observations + ) + self._callback(ei) + + merge.__doc__ = PySparkMergeIntoWriter.merge.__doc__ + + class WhenMatched: + def __init__(self, writer: "MergeIntoWriter", condition: Optional[Column]): + self.writer = writer + self._condition = condition + + def updateAll(self) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, proto.MergeAction.ACTION_TYPE_UPDATE_STAR, self._condition + ) + self.writer._matched_actions.append(action) + return self.writer + + updateAll.__doc__ = PySparkMergeIntoWriter.WhenMatched.updateAll.__doc__ + + def update(self, assignments: Dict[str, Column]) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, + proto.MergeAction.ACTION_TYPE_UPDATE, + self._condition, + assignments, + ) + self.writer._matched_actions.append(action) + return self.writer + + update.__doc__ = PySparkMergeIntoWriter.WhenMatched.update.__doc__ + + def delete(self) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, proto.MergeAction.ACTION_TYPE_DELETE, self._condition + ) + self.writer._matched_actions.append(action) + return self.writer + + delete.__doc__ = PySparkMergeIntoWriter.WhenMatched.delete.__doc__ + + WhenMatched.__doc__ = PySparkMergeIntoWriter.WhenMatched.__doc__ + + class WhenNotMatched: + def __init__(self, writer: "MergeIntoWriter", condition: Optional[Column]): + self.writer = writer + self._condition = condition + + def insertAll(self) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, proto.MergeAction.ACTION_TYPE_INSERT_STAR, self._condition + ) + self.writer._not_matched_actions.append(action) + return self.writer + + insertAll.__doc__ = PySparkMergeIntoWriter.WhenNotMatched.insertAll.__doc__ + + def insert(self, assignments: Dict[str, Column]) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, + proto.MergeAction.ACTION_TYPE_INSERT, + self._condition, + assignments, + ) + self.writer._not_matched_actions.append(action) + return self.writer + + insert.__doc__ = PySparkMergeIntoWriter.WhenNotMatched.insert.__doc__ + + WhenNotMatched.__doc__ = PySparkMergeIntoWriter.WhenNotMatched.__doc__ + + class WhenNotMatchedBySource: + def __init__(self, writer: "MergeIntoWriter", condition: Optional[Column]): + self.writer = writer + self._condition = condition + + def updateAll(self) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, proto.MergeAction.ACTION_TYPE_UPDATE_STAR, self._condition + ) + self.writer._not_matched_by_source_actions.append(action) + return self.writer + + updateAll.__doc__ = PySparkMergeIntoWriter.WhenNotMatchedBySource.updateAll.__doc__ + + def update(self, assignments: Dict[str, Column]) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, + proto.MergeAction.ACTION_TYPE_UPDATE, + self._condition, + assignments, + ) + self.writer._not_matched_by_source_actions.append(action) + return self.writer + + update.__doc__ = PySparkMergeIntoWriter.WhenNotMatchedBySource.update.__doc__ + + def delete(self) -> "MergeIntoWriter": + action = _build_merge_action( + self.writer._client, proto.MergeAction.ACTION_TYPE_DELETE, self._condition + ) + self.writer._not_matched_by_source_actions.append(action) + return self.writer + + delete.__doc__ = PySparkMergeIntoWriter.WhenNotMatchedBySource.delete.__doc__ + + WhenNotMatchedBySource.__doc__ = PySparkMergeIntoWriter.WhenNotMatchedBySource.__doc__ + + +MergeIntoWriter.__doc__ = PySparkMergeIntoWriter.__doc__ + + +def _test() -> None: + import doctest + import os + from pyspark.sql import SparkSession as PySparkSession + import pyspark.sql.connect.merge + + os.chdir(os.environ["SPARK_HOME"]) + + globs = pyspark.sql.connect.merge.__dict__.copy() + globs["spark"] = ( + PySparkSession.builder.appName("sql.connect.dataframe tests") + .remote(os.environ.get("SPARK_CONNECT_TESTING_REMOTE", "local[4]")) + .getOrCreate() + ) + (failure_count, test_count) = doctest.testmod( + pyspark.sql.merge, + globs=globs, + optionflags=doctest.ELLIPSIS | doctest.NORMALIZE_WHITESPACE | doctest.REPORT_NDIFF, + ) + globs["spark"].stop() + if failure_count: + sys.exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/pyspark/sql/connect/proto/commands_pb2.py b/python/pyspark/sql/connect/proto/commands_pb2.py index 8f67f817c3f00..c24840908f34d 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.py +++ b/python/pyspark/sql/connect/proto/commands_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\xaf\x0c\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"t\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1cspark/connect/commands.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto"\x90\r\n\x07\x43ommand\x12]\n\x11register_function\x18\x01 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x10registerFunction\x12H\n\x0fwrite_operation\x18\x02 \x01(\x0b\x32\x1d.spark.connect.WriteOperationH\x00R\x0ewriteOperation\x12_\n\x15\x63reate_dataframe_view\x18\x03 \x01(\x0b\x32).spark.connect.CreateDataFrameViewCommandH\x00R\x13\x63reateDataframeView\x12O\n\x12write_operation_v2\x18\x04 \x01(\x0b\x32\x1f.spark.connect.WriteOperationV2H\x00R\x10writeOperationV2\x12<\n\x0bsql_command\x18\x05 \x01(\x0b\x32\x19.spark.connect.SqlCommandH\x00R\nsqlCommand\x12k\n\x1cwrite_stream_operation_start\x18\x06 \x01(\x0b\x32(.spark.connect.WriteStreamOperationStartH\x00R\x19writeStreamOperationStart\x12^\n\x17streaming_query_command\x18\x07 \x01(\x0b\x32$.spark.connect.StreamingQueryCommandH\x00R\x15streamingQueryCommand\x12X\n\x15get_resources_command\x18\x08 \x01(\x0b\x32".spark.connect.GetResourcesCommandH\x00R\x13getResourcesCommand\x12t\n\x1fstreaming_query_manager_command\x18\t \x01(\x0b\x32+.spark.connect.StreamingQueryManagerCommandH\x00R\x1cstreamingQueryManagerCommand\x12m\n\x17register_table_function\x18\n \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R\x15registerTableFunction\x12\x81\x01\n$streaming_query_listener_bus_command\x18\x0b \x01(\x0b\x32/.spark.connect.StreamingQueryListenerBusCommandH\x00R streamingQueryListenerBusCommand\x12\x64\n\x14register_data_source\x18\x0c \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R\x12registerDataSource\x12t\n\x1f\x63reate_resource_profile_command\x18\r \x01(\x0b\x32+.spark.connect.CreateResourceProfileCommandH\x00R\x1c\x63reateResourceProfileCommand\x12Q\n\x12\x63heckpoint_command\x18\x0e \x01(\x0b\x32 .spark.connect.CheckpointCommandH\x00R\x11\x63heckpointCommand\x12\x84\x01\n%remove_cached_remote_relation_command\x18\x0f \x01(\x0b\x32\x30.spark.connect.RemoveCachedRemoteRelationCommandH\x00R!removeCachedRemoteRelationCommand\x12_\n\x18merge_into_table_command\x18\x10 \x01(\x0b\x32$.spark.connect.MergeIntoTableCommandH\x00R\x15mergeIntoTableCommand\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x0e\n\x0c\x63ommand_type"\xaa\x04\n\nSqlCommand\x12\x14\n\x03sql\x18\x01 \x01(\tB\x02\x18\x01R\x03sql\x12;\n\x04\x61rgs\x18\x02 \x03(\x0b\x32#.spark.connect.SqlCommand.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12Z\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32-.spark.connect.SqlCommand.NamedArgumentsEntryB\x02\x18\x01R\x0enamedArguments\x12\x42\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionB\x02\x18\x01R\x0cposArguments\x12-\n\x05input\x18\x06 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"\x96\x01\n\x1a\x43reateDataFrameViewCommand\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1b\n\tis_global\x18\x03 \x01(\x08R\x08isGlobal\x12\x18\n\x07replace\x18\x04 \x01(\x08R\x07replace"\xca\x08\n\x0eWriteOperation\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1b\n\x06source\x18\x02 \x01(\tH\x01R\x06source\x88\x01\x01\x12\x14\n\x04path\x18\x03 \x01(\tH\x00R\x04path\x12?\n\x05table\x18\x04 \x01(\x0b\x32\'.spark.connect.WriteOperation.SaveTableH\x00R\x05table\x12:\n\x04mode\x18\x05 \x01(\x0e\x32&.spark.connect.WriteOperation.SaveModeR\x04mode\x12*\n\x11sort_column_names\x18\x06 \x03(\tR\x0fsortColumnNames\x12\x31\n\x14partitioning_columns\x18\x07 \x03(\tR\x13partitioningColumns\x12\x43\n\tbucket_by\x18\x08 \x01(\x0b\x32&.spark.connect.WriteOperation.BucketByR\x08\x62ucketBy\x12\x44\n\x07options\x18\t \x03(\x0b\x32*.spark.connect.WriteOperation.OptionsEntryR\x07options\x12-\n\x12\x63lustering_columns\x18\n \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x82\x02\n\tSaveTable\x12\x1d\n\ntable_name\x18\x01 \x01(\tR\ttableName\x12X\n\x0bsave_method\x18\x02 \x01(\x0e\x32\x37.spark.connect.WriteOperation.SaveTable.TableSaveMethodR\nsaveMethod"|\n\x0fTableSaveMethod\x12!\n\x1dTABLE_SAVE_METHOD_UNSPECIFIED\x10\x00\x12#\n\x1fTABLE_SAVE_METHOD_SAVE_AS_TABLE\x10\x01\x12!\n\x1dTABLE_SAVE_METHOD_INSERT_INTO\x10\x02\x1a[\n\x08\x42ucketBy\x12.\n\x13\x62ucket_column_names\x18\x01 \x03(\tR\x11\x62ucketColumnNames\x12\x1f\n\x0bnum_buckets\x18\x02 \x01(\x05R\nnumBuckets"\x89\x01\n\x08SaveMode\x12\x19\n\x15SAVE_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10SAVE_MODE_APPEND\x10\x01\x12\x17\n\x13SAVE_MODE_OVERWRITE\x10\x02\x12\x1d\n\x19SAVE_MODE_ERROR_IF_EXISTS\x10\x03\x12\x14\n\x10SAVE_MODE_IGNORE\x10\x04\x42\x0b\n\tsave_typeB\t\n\x07_source"\xdc\x06\n\x10WriteOperationV2\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\ntable_name\x18\x02 \x01(\tR\ttableName\x12\x1f\n\x08provider\x18\x03 \x01(\tH\x00R\x08provider\x88\x01\x01\x12L\n\x14partitioning_columns\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13partitioningColumns\x12\x46\n\x07options\x18\x05 \x03(\x0b\x32,.spark.connect.WriteOperationV2.OptionsEntryR\x07options\x12_\n\x10table_properties\x18\x06 \x03(\x0b\x32\x34.spark.connect.WriteOperationV2.TablePropertiesEntryR\x0ftableProperties\x12\x38\n\x04mode\x18\x07 \x01(\x0e\x32$.spark.connect.WriteOperationV2.ModeR\x04mode\x12J\n\x13overwrite_condition\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x12overwriteCondition\x12-\n\x12\x63lustering_columns\x18\t \x03(\tR\x11\x63lusteringColumns\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x42\n\x14TablePropertiesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"\x9f\x01\n\x04Mode\x12\x14\n\x10MODE_UNSPECIFIED\x10\x00\x12\x0f\n\x0bMODE_CREATE\x10\x01\x12\x12\n\x0eMODE_OVERWRITE\x10\x02\x12\x1d\n\x19MODE_OVERWRITE_PARTITIONS\x10\x03\x12\x0f\n\x0bMODE_APPEND\x10\x04\x12\x10\n\x0cMODE_REPLACE\x10\x05\x12\x1a\n\x16MODE_CREATE_OR_REPLACE\x10\x06\x42\x0b\n\t_provider"\xa0\x06\n\x19WriteStreamOperationStart\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06\x66ormat\x18\x02 \x01(\tR\x06\x66ormat\x12O\n\x07options\x18\x03 \x03(\x0b\x32\x35.spark.connect.WriteStreamOperationStart.OptionsEntryR\x07options\x12:\n\x19partitioning_column_names\x18\x04 \x03(\tR\x17partitioningColumnNames\x12:\n\x18processing_time_interval\x18\x05 \x01(\tH\x00R\x16processingTimeInterval\x12%\n\ravailable_now\x18\x06 \x01(\x08H\x00R\x0c\x61vailableNow\x12\x14\n\x04once\x18\x07 \x01(\x08H\x00R\x04once\x12\x46\n\x1e\x63ontinuous_checkpoint_interval\x18\x08 \x01(\tH\x00R\x1c\x63ontinuousCheckpointInterval\x12\x1f\n\x0boutput_mode\x18\t \x01(\tR\noutputMode\x12\x1d\n\nquery_name\x18\n \x01(\tR\tqueryName\x12\x14\n\x04path\x18\x0b \x01(\tH\x01R\x04path\x12\x1f\n\ntable_name\x18\x0c \x01(\tH\x01R\ttableName\x12N\n\x0e\x66oreach_writer\x18\r \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\rforeachWriter\x12L\n\rforeach_batch\x18\x0e \x01(\x0b\x32\'.spark.connect.StreamingForeachFunctionR\x0c\x66oreachBatch\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07triggerB\x12\n\x10sink_destination"\xb3\x01\n\x18StreamingForeachFunction\x12\x43\n\x0fpython_function\x18\x01 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x0epythonFunction\x12\x46\n\x0escala_function\x18\x02 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\rscalaFunctionB\n\n\x08\x66unction"\xd4\x01\n\x1fWriteStreamOperationStartResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12<\n\x18query_started_event_json\x18\x03 \x01(\tH\x00R\x15queryStartedEventJson\x88\x01\x01\x42\x1b\n\x19_query_started_event_json"A\n\x18StreamingQueryInstanceId\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id\x12\x15\n\x06run_id\x18\x02 \x01(\tR\x05runId"\xf8\x04\n\x15StreamingQueryCommand\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12\x18\n\x06status\x18\x02 \x01(\x08H\x00R\x06status\x12%\n\rlast_progress\x18\x03 \x01(\x08H\x00R\x0clastProgress\x12)\n\x0frecent_progress\x18\x04 \x01(\x08H\x00R\x0erecentProgress\x12\x14\n\x04stop\x18\x05 \x01(\x08H\x00R\x04stop\x12\x34\n\x15process_all_available\x18\x06 \x01(\x08H\x00R\x13processAllAvailable\x12O\n\x07\x65xplain\x18\x07 \x01(\x0b\x32\x33.spark.connect.StreamingQueryCommand.ExplainCommandH\x00R\x07\x65xplain\x12\x1e\n\texception\x18\x08 \x01(\x08H\x00R\texception\x12k\n\x11\x61wait_termination\x18\t \x01(\x0b\x32<.spark.connect.StreamingQueryCommand.AwaitTerminationCommandH\x00R\x10\x61waitTermination\x1a,\n\x0e\x45xplainCommand\x12\x1a\n\x08\x65xtended\x18\x01 \x01(\x08R\x08\x65xtended\x1aL\n\x17\x41waitTerminationCommand\x12"\n\ntimeout_ms\x18\x02 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_msB\t\n\x07\x63ommand"\xf5\x08\n\x1bStreamingQueryCommandResult\x12\x42\n\x08query_id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x07queryId\x12Q\n\x06status\x18\x02 \x01(\x0b\x32\x37.spark.connect.StreamingQueryCommandResult.StatusResultH\x00R\x06status\x12j\n\x0frecent_progress\x18\x03 \x01(\x0b\x32?.spark.connect.StreamingQueryCommandResult.RecentProgressResultH\x00R\x0erecentProgress\x12T\n\x07\x65xplain\x18\x04 \x01(\x0b\x32\x38.spark.connect.StreamingQueryCommandResult.ExplainResultH\x00R\x07\x65xplain\x12Z\n\texception\x18\x05 \x01(\x0b\x32:.spark.connect.StreamingQueryCommandResult.ExceptionResultH\x00R\texception\x12p\n\x11\x61wait_termination\x18\x06 \x01(\x0b\x32\x41.spark.connect.StreamingQueryCommandResult.AwaitTerminationResultH\x00R\x10\x61waitTermination\x1a\xaa\x01\n\x0cStatusResult\x12%\n\x0estatus_message\x18\x01 \x01(\tR\rstatusMessage\x12*\n\x11is_data_available\x18\x02 \x01(\x08R\x0fisDataAvailable\x12*\n\x11is_trigger_active\x18\x03 \x01(\x08R\x0fisTriggerActive\x12\x1b\n\tis_active\x18\x04 \x01(\x08R\x08isActive\x1aH\n\x14RecentProgressResult\x12\x30\n\x14recent_progress_json\x18\x05 \x03(\tR\x12recentProgressJson\x1a\'\n\rExplainResult\x12\x16\n\x06result\x18\x01 \x01(\tR\x06result\x1a\xc5\x01\n\x0f\x45xceptionResult\x12\x30\n\x11\x65xception_message\x18\x01 \x01(\tH\x00R\x10\x65xceptionMessage\x88\x01\x01\x12$\n\x0b\x65rror_class\x18\x02 \x01(\tH\x01R\nerrorClass\x88\x01\x01\x12$\n\x0bstack_trace\x18\x03 \x01(\tH\x02R\nstackTrace\x88\x01\x01\x42\x14\n\x12_exception_messageB\x0e\n\x0c_error_classB\x0e\n\x0c_stack_trace\x1a\x38\n\x16\x41waitTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminatedB\r\n\x0bresult_type"\xbd\x06\n\x1cStreamingQueryManagerCommand\x12\x18\n\x06\x61\x63tive\x18\x01 \x01(\x08H\x00R\x06\x61\x63tive\x12\x1d\n\tget_query\x18\x02 \x01(\tH\x00R\x08getQuery\x12|\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32\x46.spark.connect.StreamingQueryManagerCommand.AwaitAnyTerminationCommandH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12n\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0b\x61\x64\x64Listener\x12t\n\x0fremove_listener\x18\x06 \x01(\x0b\x32I.spark.connect.StreamingQueryManagerCommand.StreamingQueryListenerCommandH\x00R\x0eremoveListener\x12\'\n\x0elist_listeners\x18\x07 \x01(\x08H\x00R\rlistListeners\x1aO\n\x1a\x41waitAnyTerminationCommand\x12"\n\ntimeout_ms\x18\x01 \x01(\x03H\x00R\ttimeoutMs\x88\x01\x01\x42\r\n\x0b_timeout_ms\x1a\xcd\x01\n\x1dStreamingQueryListenerCommand\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x12U\n\x17python_listener_payload\x18\x02 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\x15pythonListenerPayload\x88\x01\x01\x12\x0e\n\x02id\x18\x03 \x01(\tR\x02idB\x1a\n\x18_python_listener_payloadB\t\n\x07\x63ommand"\xb4\x08\n"StreamingQueryManagerCommandResult\x12X\n\x06\x61\x63tive\x18\x01 \x01(\x0b\x32>.spark.connect.StreamingQueryManagerCommandResult.ActiveResultH\x00R\x06\x61\x63tive\x12`\n\x05query\x18\x02 \x01(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceH\x00R\x05query\x12\x81\x01\n\x15\x61wait_any_termination\x18\x03 \x01(\x0b\x32K.spark.connect.StreamingQueryManagerCommandResult.AwaitAnyTerminationResultH\x00R\x13\x61waitAnyTermination\x12+\n\x10reset_terminated\x18\x04 \x01(\x08H\x00R\x0fresetTerminated\x12#\n\x0c\x61\x64\x64_listener\x18\x05 \x01(\x08H\x00R\x0b\x61\x64\x64Listener\x12)\n\x0fremove_listener\x18\x06 \x01(\x08H\x00R\x0eremoveListener\x12{\n\x0elist_listeners\x18\x07 \x01(\x0b\x32R.spark.connect.StreamingQueryManagerCommandResult.ListStreamingQueryListenerResultH\x00R\rlistListeners\x1a\x7f\n\x0c\x41\x63tiveResult\x12o\n\x0e\x61\x63tive_queries\x18\x01 \x03(\x0b\x32H.spark.connect.StreamingQueryManagerCommandResult.StreamingQueryInstanceR\ractiveQueries\x1as\n\x16StreamingQueryInstance\x12\x37\n\x02id\x18\x01 \x01(\x0b\x32\'.spark.connect.StreamingQueryInstanceIdR\x02id\x12\x17\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x88\x01\x01\x42\x07\n\x05_name\x1a;\n\x19\x41waitAnyTerminationResult\x12\x1e\n\nterminated\x18\x01 \x01(\x08R\nterminated\x1aK\n\x1eStreamingQueryListenerInstance\x12)\n\x10listener_payload\x18\x01 \x01(\x0cR\x0flistenerPayload\x1a\x45\n ListStreamingQueryListenerResult\x12!\n\x0clistener_ids\x18\x01 \x03(\tR\x0blistenerIdsB\r\n\x0bresult_type"\xad\x01\n StreamingQueryListenerBusCommand\x12;\n\x19\x61\x64\x64_listener_bus_listener\x18\x01 \x01(\x08H\x00R\x16\x61\x64\x64ListenerBusListener\x12\x41\n\x1cremove_listener_bus_listener\x18\x02 \x01(\x08H\x00R\x19removeListenerBusListenerB\t\n\x07\x63ommand"\x83\x01\n\x1bStreamingQueryListenerEvent\x12\x1d\n\nevent_json\x18\x01 \x01(\tR\teventJson\x12\x45\n\nevent_type\x18\x02 \x01(\x0e\x32&.spark.connect.StreamingQueryEventTypeR\teventType"\xcc\x01\n"StreamingQueryListenerEventsResult\x12\x42\n\x06\x65vents\x18\x01 \x03(\x0b\x32*.spark.connect.StreamingQueryListenerEventR\x06\x65vents\x12\x42\n\x1blistener_bus_listener_added\x18\x02 \x01(\x08H\x00R\x18listenerBusListenerAdded\x88\x01\x01\x42\x1e\n\x1c_listener_bus_listener_added"\x15\n\x13GetResourcesCommand"\xd4\x01\n\x19GetResourcesCommandResult\x12U\n\tresources\x18\x01 \x03(\x0b\x32\x37.spark.connect.GetResourcesCommandResult.ResourcesEntryR\tresources\x1a`\n\x0eResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.ResourceInformationR\x05value:\x02\x38\x01"X\n\x1c\x43reateResourceProfileCommand\x12\x38\n\x07profile\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ResourceProfileR\x07profile"C\n"CreateResourceProfileCommandResult\x12\x1d\n\nprofile_id\x18\x01 \x01(\x05R\tprofileId"d\n!RemoveCachedRemoteRelationCommand\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation"t\n\x11\x43heckpointCommand\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x14\n\x05local\x18\x02 \x01(\x08R\x05local\x12\x14\n\x05\x65\x61ger\x18\x03 \x01(\x08R\x05\x65\x61ger"\xe8\x03\n\x15MergeIntoTableCommand\x12*\n\x11target_table_name\x18\x01 \x01(\tR\x0ftargetTableName\x12\x43\n\x11source_table_plan\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x0fsourceTablePlan\x12\x42\n\x0fmerge_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0emergeCondition\x12>\n\rmatch_actions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cmatchActions\x12I\n\x13not_matched_actions\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x11notMatchedActions\x12[\n\x1dnot_matched_by_source_actions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x19notMatchedBySourceActions\x12\x32\n\x15with_schema_evolution\x18\x07 \x01(\x08R\x13withSchemaEvolution*\x85\x01\n\x17StreamingQueryEventType\x12\x1e\n\x1aQUERY_PROGRESS_UNSPECIFIED\x10\x00\x12\x18\n\x14QUERY_PROGRESS_EVENT\x10\x01\x12\x1a\n\x16QUERY_TERMINATED_EVENT\x10\x02\x12\x14\n\x10QUERY_IDLE_EVENT\x10\x03\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -71,102 +71,104 @@ _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_options = b"8\001" _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._options = None _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_options = b"8\001" - _STREAMINGQUERYEVENTTYPE._serialized_start = 10518 - _STREAMINGQUERYEVENTTYPE._serialized_end = 10651 + _STREAMINGQUERYEVENTTYPE._serialized_start = 11106 + _STREAMINGQUERYEVENTTYPE._serialized_end = 11239 _COMMAND._serialized_start = 167 - _COMMAND._serialized_end = 1750 - _SQLCOMMAND._serialized_start = 1753 - _SQLCOMMAND._serialized_end = 2307 - _SQLCOMMAND_ARGSENTRY._serialized_start = 2123 - _SQLCOMMAND_ARGSENTRY._serialized_end = 2213 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 2215 - _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 2307 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 2310 - _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 2460 - _WRITEOPERATION._serialized_start = 2463 - _WRITEOPERATION._serialized_end = 3561 - _WRITEOPERATION_OPTIONSENTRY._serialized_start = 2985 - _WRITEOPERATION_OPTIONSENTRY._serialized_end = 3043 - _WRITEOPERATION_SAVETABLE._serialized_start = 3046 - _WRITEOPERATION_SAVETABLE._serialized_end = 3304 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 3180 - _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 3304 - _WRITEOPERATION_BUCKETBY._serialized_start = 3306 - _WRITEOPERATION_BUCKETBY._serialized_end = 3397 - _WRITEOPERATION_SAVEMODE._serialized_start = 3400 - _WRITEOPERATION_SAVEMODE._serialized_end = 3537 - _WRITEOPERATIONV2._serialized_start = 3564 - _WRITEOPERATIONV2._serialized_end = 4424 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 2985 - _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 3043 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 4183 - _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 4249 - _WRITEOPERATIONV2_MODE._serialized_start = 4252 - _WRITEOPERATIONV2_MODE._serialized_end = 4411 - _WRITESTREAMOPERATIONSTART._serialized_start = 4427 - _WRITESTREAMOPERATIONSTART._serialized_end = 5227 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 2985 - _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 3043 - _STREAMINGFOREACHFUNCTION._serialized_start = 5230 - _STREAMINGFOREACHFUNCTION._serialized_end = 5409 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 5412 - _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 5624 - _STREAMINGQUERYINSTANCEID._serialized_start = 5626 - _STREAMINGQUERYINSTANCEID._serialized_end = 5691 - _STREAMINGQUERYCOMMAND._serialized_start = 5694 - _STREAMINGQUERYCOMMAND._serialized_end = 6326 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 6193 - _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 6237 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 6239 - _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 6315 - _STREAMINGQUERYCOMMANDRESULT._serialized_start = 6329 - _STREAMINGQUERYCOMMANDRESULT._serialized_end = 7470 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 6912 - _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 7082 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 7084 - _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 7156 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 7158 - _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 7197 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 7200 - _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 7397 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 7399 - _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 7455 - _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 7473 - _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 8302 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 8004 - _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 8083 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 8086 - _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 8291 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 8305 - _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 9381 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 8913 - _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 9040 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 9042 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 9157 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 9159 - _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 9218 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 9220 - _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 9295 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 9297 - _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 9366 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 9384 - _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 9557 - _STREAMINGQUERYLISTENEREVENT._serialized_start = 9560 - _STREAMINGQUERYLISTENEREVENT._serialized_end = 9691 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9694 - _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9898 - _GETRESOURCESCOMMAND._serialized_start = 9900 - _GETRESOURCESCOMMAND._serialized_end = 9921 - _GETRESOURCESCOMMANDRESULT._serialized_start = 9924 - _GETRESOURCESCOMMANDRESULT._serialized_end = 10136 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 10040 - _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 10136 - _CREATERESOURCEPROFILECOMMAND._serialized_start = 10138 - _CREATERESOURCEPROFILECOMMAND._serialized_end = 10226 - _CREATERESOURCEPROFILECOMMANDRESULT._serialized_start = 10228 - _CREATERESOURCEPROFILECOMMANDRESULT._serialized_end = 10295 - _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_start = 10297 - _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_end = 10397 - _CHECKPOINTCOMMAND._serialized_start = 10399 - _CHECKPOINTCOMMAND._serialized_end = 10515 + _COMMAND._serialized_end = 1847 + _SQLCOMMAND._serialized_start = 1850 + _SQLCOMMAND._serialized_end = 2404 + _SQLCOMMAND_ARGSENTRY._serialized_start = 2220 + _SQLCOMMAND_ARGSENTRY._serialized_end = 2310 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_start = 2312 + _SQLCOMMAND_NAMEDARGUMENTSENTRY._serialized_end = 2404 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_start = 2407 + _CREATEDATAFRAMEVIEWCOMMAND._serialized_end = 2557 + _WRITEOPERATION._serialized_start = 2560 + _WRITEOPERATION._serialized_end = 3658 + _WRITEOPERATION_OPTIONSENTRY._serialized_start = 3082 + _WRITEOPERATION_OPTIONSENTRY._serialized_end = 3140 + _WRITEOPERATION_SAVETABLE._serialized_start = 3143 + _WRITEOPERATION_SAVETABLE._serialized_end = 3401 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_start = 3277 + _WRITEOPERATION_SAVETABLE_TABLESAVEMETHOD._serialized_end = 3401 + _WRITEOPERATION_BUCKETBY._serialized_start = 3403 + _WRITEOPERATION_BUCKETBY._serialized_end = 3494 + _WRITEOPERATION_SAVEMODE._serialized_start = 3497 + _WRITEOPERATION_SAVEMODE._serialized_end = 3634 + _WRITEOPERATIONV2._serialized_start = 3661 + _WRITEOPERATIONV2._serialized_end = 4521 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_start = 3082 + _WRITEOPERATIONV2_OPTIONSENTRY._serialized_end = 3140 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_start = 4280 + _WRITEOPERATIONV2_TABLEPROPERTIESENTRY._serialized_end = 4346 + _WRITEOPERATIONV2_MODE._serialized_start = 4349 + _WRITEOPERATIONV2_MODE._serialized_end = 4508 + _WRITESTREAMOPERATIONSTART._serialized_start = 4524 + _WRITESTREAMOPERATIONSTART._serialized_end = 5324 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_start = 3082 + _WRITESTREAMOPERATIONSTART_OPTIONSENTRY._serialized_end = 3140 + _STREAMINGFOREACHFUNCTION._serialized_start = 5327 + _STREAMINGFOREACHFUNCTION._serialized_end = 5506 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_start = 5509 + _WRITESTREAMOPERATIONSTARTRESULT._serialized_end = 5721 + _STREAMINGQUERYINSTANCEID._serialized_start = 5723 + _STREAMINGQUERYINSTANCEID._serialized_end = 5788 + _STREAMINGQUERYCOMMAND._serialized_start = 5791 + _STREAMINGQUERYCOMMAND._serialized_end = 6423 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_start = 6290 + _STREAMINGQUERYCOMMAND_EXPLAINCOMMAND._serialized_end = 6334 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_start = 6336 + _STREAMINGQUERYCOMMAND_AWAITTERMINATIONCOMMAND._serialized_end = 6412 + _STREAMINGQUERYCOMMANDRESULT._serialized_start = 6426 + _STREAMINGQUERYCOMMANDRESULT._serialized_end = 7567 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_start = 7009 + _STREAMINGQUERYCOMMANDRESULT_STATUSRESULT._serialized_end = 7179 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_start = 7181 + _STREAMINGQUERYCOMMANDRESULT_RECENTPROGRESSRESULT._serialized_end = 7253 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_start = 7255 + _STREAMINGQUERYCOMMANDRESULT_EXPLAINRESULT._serialized_end = 7294 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_start = 7297 + _STREAMINGQUERYCOMMANDRESULT_EXCEPTIONRESULT._serialized_end = 7494 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_start = 7496 + _STREAMINGQUERYCOMMANDRESULT_AWAITTERMINATIONRESULT._serialized_end = 7552 + _STREAMINGQUERYMANAGERCOMMAND._serialized_start = 7570 + _STREAMINGQUERYMANAGERCOMMAND._serialized_end = 8399 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_start = 8101 + _STREAMINGQUERYMANAGERCOMMAND_AWAITANYTERMINATIONCOMMAND._serialized_end = 8180 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_start = 8183 + _STREAMINGQUERYMANAGERCOMMAND_STREAMINGQUERYLISTENERCOMMAND._serialized_end = 8388 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_start = 8402 + _STREAMINGQUERYMANAGERCOMMANDRESULT._serialized_end = 9478 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_start = 9010 + _STREAMINGQUERYMANAGERCOMMANDRESULT_ACTIVERESULT._serialized_end = 9137 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_start = 9139 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYINSTANCE._serialized_end = 9254 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_start = 9256 + _STREAMINGQUERYMANAGERCOMMANDRESULT_AWAITANYTERMINATIONRESULT._serialized_end = 9315 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_start = 9317 + _STREAMINGQUERYMANAGERCOMMANDRESULT_STREAMINGQUERYLISTENERINSTANCE._serialized_end = 9392 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_start = 9394 + _STREAMINGQUERYMANAGERCOMMANDRESULT_LISTSTREAMINGQUERYLISTENERRESULT._serialized_end = 9463 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_start = 9481 + _STREAMINGQUERYLISTENERBUSCOMMAND._serialized_end = 9654 + _STREAMINGQUERYLISTENEREVENT._serialized_start = 9657 + _STREAMINGQUERYLISTENEREVENT._serialized_end = 9788 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_start = 9791 + _STREAMINGQUERYLISTENEREVENTSRESULT._serialized_end = 9995 + _GETRESOURCESCOMMAND._serialized_start = 9997 + _GETRESOURCESCOMMAND._serialized_end = 10018 + _GETRESOURCESCOMMANDRESULT._serialized_start = 10021 + _GETRESOURCESCOMMANDRESULT._serialized_end = 10233 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_start = 10137 + _GETRESOURCESCOMMANDRESULT_RESOURCESENTRY._serialized_end = 10233 + _CREATERESOURCEPROFILECOMMAND._serialized_start = 10235 + _CREATERESOURCEPROFILECOMMAND._serialized_end = 10323 + _CREATERESOURCEPROFILECOMMANDRESULT._serialized_start = 10325 + _CREATERESOURCEPROFILECOMMANDRESULT._serialized_end = 10392 + _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_start = 10394 + _REMOVECACHEDREMOTERELATIONCOMMAND._serialized_end = 10494 + _CHECKPOINTCOMMAND._serialized_start = 10496 + _CHECKPOINTCOMMAND._serialized_end = 10612 + _MERGEINTOTABLECOMMAND._serialized_start = 10615 + _MERGEINTOTABLECOMMAND._serialized_end = 11103 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/commands_pb2.pyi b/python/pyspark/sql/connect/proto/commands_pb2.pyi index 04d50d5b5e4f4..03a31ecdfedfe 100644 --- a/python/pyspark/sql/connect/proto/commands_pb2.pyi +++ b/python/pyspark/sql/connect/proto/commands_pb2.pyi @@ -103,6 +103,7 @@ class Command(google.protobuf.message.Message): CREATE_RESOURCE_PROFILE_COMMAND_FIELD_NUMBER: builtins.int CHECKPOINT_COMMAND_FIELD_NUMBER: builtins.int REMOVE_CACHED_REMOTE_RELATION_COMMAND_FIELD_NUMBER: builtins.int + MERGE_INTO_TABLE_COMMAND_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def register_function( @@ -143,6 +144,8 @@ class Command(google.protobuf.message.Message): self, ) -> global___RemoveCachedRemoteRelationCommand: ... @property + def merge_into_table_command(self) -> global___MergeIntoTableCommand: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary Commands they can add them here. During the planning the correct resolution is done. @@ -170,6 +173,7 @@ class Command(google.protobuf.message.Message): checkpoint_command: global___CheckpointCommand | None = ..., remove_cached_remote_relation_command: global___RemoveCachedRemoteRelationCommand | None = ..., + merge_into_table_command: global___MergeIntoTableCommand | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -187,6 +191,8 @@ class Command(google.protobuf.message.Message): b"extension", "get_resources_command", b"get_resources_command", + "merge_into_table_command", + b"merge_into_table_command", "register_data_source", b"register_data_source", "register_function", @@ -226,6 +232,8 @@ class Command(google.protobuf.message.Message): b"extension", "get_resources_command", b"get_resources_command", + "merge_into_table_command", + b"merge_into_table_command", "register_data_source", b"register_data_source", "register_function", @@ -269,6 +277,7 @@ class Command(google.protobuf.message.Message): "create_resource_profile_command", "checkpoint_command", "remove_cached_remote_relation_command", + "merge_into_table_command", "extension", ] | None @@ -2197,3 +2206,92 @@ class CheckpointCommand(google.protobuf.message.Message): ) -> None: ... global___CheckpointCommand = CheckpointCommand + +class MergeIntoTableCommand(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + TARGET_TABLE_NAME_FIELD_NUMBER: builtins.int + SOURCE_TABLE_PLAN_FIELD_NUMBER: builtins.int + MERGE_CONDITION_FIELD_NUMBER: builtins.int + MATCH_ACTIONS_FIELD_NUMBER: builtins.int + NOT_MATCHED_ACTIONS_FIELD_NUMBER: builtins.int + NOT_MATCHED_BY_SOURCE_ACTIONS_FIELD_NUMBER: builtins.int + WITH_SCHEMA_EVOLUTION_FIELD_NUMBER: builtins.int + target_table_name: builtins.str + """(Required) The name of the target table.""" + @property + def source_table_plan(self) -> pyspark.sql.connect.proto.relations_pb2.Relation: + """(Required) The relation of the source table.""" + @property + def merge_condition(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression: + """(Required) The condition to match the source and target.""" + @property + def match_actions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Optional) The actions to be taken when the condition is matched.""" + @property + def not_matched_actions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Optional) The actions to be taken when the condition is not matched.""" + @property + def not_matched_by_source_actions( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ]: + """(Optional) The actions to be taken when the condition is not matched by source.""" + with_schema_evolution: builtins.bool + """(Required) Whether to enable schema evolution.""" + def __init__( + self, + *, + target_table_name: builtins.str = ..., + source_table_plan: pyspark.sql.connect.proto.relations_pb2.Relation | None = ..., + merge_condition: pyspark.sql.connect.proto.expressions_pb2.Expression | None = ..., + match_actions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + not_matched_actions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + not_matched_by_source_actions: collections.abc.Iterable[ + pyspark.sql.connect.proto.expressions_pb2.Expression + ] + | None = ..., + with_schema_evolution: builtins.bool = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "merge_condition", b"merge_condition", "source_table_plan", b"source_table_plan" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "match_actions", + b"match_actions", + "merge_condition", + b"merge_condition", + "not_matched_actions", + b"not_matched_actions", + "not_matched_by_source_actions", + b"not_matched_by_source_actions", + "source_table_plan", + b"source_table_plan", + "target_table_name", + b"target_table_name", + "with_schema_evolution", + b"with_schema_evolution", + ], + ) -> None: ... + +global___MergeIntoTableCommand = MergeIntoTableCommand diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index c8a183105fd11..b4c4b48de2688 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -34,7 +34,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\x97/\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05valueB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xd8/\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_conditionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -47,69 +47,75 @@ b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" ) _EXPRESSION._serialized_start = 133 - _EXPRESSION._serialized_end = 6172 - _EXPRESSION_WINDOW._serialized_start = 1730 - _EXPRESSION_WINDOW._serialized_end = 2513 - _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 2020 - _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2513 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2287 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2432 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2434 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2513 - _EXPRESSION_SORTORDER._serialized_start = 2516 - _EXPRESSION_SORTORDER._serialized_end = 2941 - _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2746 - _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 2854 - _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 2856 - _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 2941 - _EXPRESSION_CAST._serialized_start = 2944 - _EXPRESSION_CAST._serialized_end = 3259 - _EXPRESSION_CAST_EVALMODE._serialized_start = 3145 - _EXPRESSION_CAST_EVALMODE._serialized_end = 3243 - _EXPRESSION_LITERAL._serialized_start = 3262 - _EXPRESSION_LITERAL._serialized_end = 4825 - _EXPRESSION_LITERAL_DECIMAL._serialized_start = 4097 - _EXPRESSION_LITERAL_DECIMAL._serialized_end = 4214 - _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 4216 - _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4314 - _EXPRESSION_LITERAL_ARRAY._serialized_start = 4317 - _EXPRESSION_LITERAL_ARRAY._serialized_end = 4447 - _EXPRESSION_LITERAL_MAP._serialized_start = 4450 - _EXPRESSION_LITERAL_MAP._serialized_end = 4677 - _EXPRESSION_LITERAL_STRUCT._serialized_start = 4680 - _EXPRESSION_LITERAL_STRUCT._serialized_end = 4809 - _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4828 - _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 5014 - _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 5017 - _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 5221 - _EXPRESSION_EXPRESSIONSTRING._serialized_start = 5223 - _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5273 - _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5275 - _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5399 - _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5401 - _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5487 - _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5490 - _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5622 - _EXPRESSION_UPDATEFIELDS._serialized_start = 5625 - _EXPRESSION_UPDATEFIELDS._serialized_end = 5812 - _EXPRESSION_ALIAS._serialized_start = 5814 - _EXPRESSION_ALIAS._serialized_end = 5934 - _EXPRESSION_LAMBDAFUNCTION._serialized_start = 5937 - _EXPRESSION_LAMBDAFUNCTION._serialized_end = 6095 - _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 6097 - _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 6159 - _EXPRESSIONCOMMON._serialized_start = 6174 - _EXPRESSIONCOMMON._serialized_end = 6239 - _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 6242 - _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6606 - _PYTHONUDF._serialized_start = 6609 - _PYTHONUDF._serialized_end = 6813 - _SCALARSCALAUDF._serialized_start = 6816 - _SCALARSCALAUDF._serialized_end = 7030 - _JAVAUDF._serialized_start = 7033 - _JAVAUDF._serialized_end = 7182 - _CALLFUNCTION._serialized_start = 7184 - _CALLFUNCTION._serialized_end = 7292 - _NAMEDARGUMENTEXPRESSION._serialized_start = 7294 - _NAMEDARGUMENTEXPRESSION._serialized_end = 7386 + _EXPRESSION._serialized_end = 6237 + _EXPRESSION_WINDOW._serialized_start = 1795 + _EXPRESSION_WINDOW._serialized_end = 2578 + _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 2085 + _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2578 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2352 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2497 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2499 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2578 + _EXPRESSION_SORTORDER._serialized_start = 2581 + _EXPRESSION_SORTORDER._serialized_end = 3006 + _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2811 + _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 2919 + _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 2921 + _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 3006 + _EXPRESSION_CAST._serialized_start = 3009 + _EXPRESSION_CAST._serialized_end = 3324 + _EXPRESSION_CAST_EVALMODE._serialized_start = 3210 + _EXPRESSION_CAST_EVALMODE._serialized_end = 3308 + _EXPRESSION_LITERAL._serialized_start = 3327 + _EXPRESSION_LITERAL._serialized_end = 4890 + _EXPRESSION_LITERAL_DECIMAL._serialized_start = 4162 + _EXPRESSION_LITERAL_DECIMAL._serialized_end = 4279 + _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 4281 + _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4379 + _EXPRESSION_LITERAL_ARRAY._serialized_start = 4382 + _EXPRESSION_LITERAL_ARRAY._serialized_end = 4512 + _EXPRESSION_LITERAL_MAP._serialized_start = 4515 + _EXPRESSION_LITERAL_MAP._serialized_end = 4742 + _EXPRESSION_LITERAL_STRUCT._serialized_start = 4745 + _EXPRESSION_LITERAL_STRUCT._serialized_end = 4874 + _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4893 + _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 5079 + _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 5082 + _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 5286 + _EXPRESSION_EXPRESSIONSTRING._serialized_start = 5288 + _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5338 + _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5340 + _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5464 + _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5466 + _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5552 + _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5555 + _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5687 + _EXPRESSION_UPDATEFIELDS._serialized_start = 5690 + _EXPRESSION_UPDATEFIELDS._serialized_end = 5877 + _EXPRESSION_ALIAS._serialized_start = 5879 + _EXPRESSION_ALIAS._serialized_end = 5999 + _EXPRESSION_LAMBDAFUNCTION._serialized_start = 6002 + _EXPRESSION_LAMBDAFUNCTION._serialized_end = 6160 + _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 6162 + _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 6224 + _EXPRESSIONCOMMON._serialized_start = 6239 + _EXPRESSIONCOMMON._serialized_end = 6304 + _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 6307 + _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6671 + _PYTHONUDF._serialized_start = 6674 + _PYTHONUDF._serialized_end = 6878 + _SCALARSCALAUDF._serialized_start = 6881 + _SCALARSCALAUDF._serialized_end = 7095 + _JAVAUDF._serialized_start = 7098 + _JAVAUDF._serialized_end = 7247 + _CALLFUNCTION._serialized_start = 7249 + _CALLFUNCTION._serialized_end = 7357 + _NAMEDARGUMENTEXPRESSION._serialized_start = 7359 + _NAMEDARGUMENTEXPRESSION._serialized_end = 7451 + _MERGEACTION._serialized_start = 7454 + _MERGEACTION._serialized_end = 7966 + _MERGEACTION_ASSIGNMENT._serialized_start = 7676 + _MERGEACTION_ASSIGNMENT._serialized_end = 7782 + _MERGEACTION_ACTIONTYPE._serialized_start = 7785 + _MERGEACTION_ACTIONTYPE._serialized_end = 7952 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 42031d47bb851..2c80be6c8fb5a 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1182,6 +1182,7 @@ class Expression(google.protobuf.message.Message): COMMON_INLINE_USER_DEFINED_FUNCTION_FIELD_NUMBER: builtins.int CALL_FUNCTION_FIELD_NUMBER: builtins.int NAMED_ARGUMENT_EXPRESSION_FIELD_NUMBER: builtins.int + MERGE_ACTION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def common(self) -> global___ExpressionCommon: ... @@ -1222,6 +1223,8 @@ class Expression(google.protobuf.message.Message): @property def named_argument_expression(self) -> global___NamedArgumentExpression: ... @property + def merge_action(self) -> global___MergeAction: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -1248,6 +1251,7 @@ class Expression(google.protobuf.message.Message): common_inline_user_defined_function: global___CommonInlineUserDefinedFunction | None = ..., call_function: global___CallFunction | None = ..., named_argument_expression: global___NamedArgumentExpression | None = ..., + merge_action: global___MergeAction | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -1273,6 +1277,8 @@ class Expression(google.protobuf.message.Message): b"lambda_function", "literal", b"literal", + "merge_action", + b"merge_action", "named_argument_expression", b"named_argument_expression", "sort_order", @@ -1318,6 +1324,8 @@ class Expression(google.protobuf.message.Message): b"lambda_function", "literal", b"literal", + "merge_action", + b"merge_action", "named_argument_expression", b"named_argument_expression", "sort_order", @@ -1361,6 +1369,7 @@ class Expression(google.protobuf.message.Message): "common_inline_user_defined_function", "call_function", "named_argument_expression", + "merge_action", "extension", ] | None @@ -1662,3 +1671,103 @@ class NamedArgumentExpression(google.protobuf.message.Message): ) -> None: ... global___NamedArgumentExpression = NamedArgumentExpression + +class MergeAction(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class _ActionType: + ValueType = typing.NewType("ValueType", builtins.int) + V: typing_extensions.TypeAlias = ValueType + + class _ActionTypeEnumTypeWrapper( + google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ + MergeAction._ActionType.ValueType + ], + builtins.type, + ): # noqa: F821 + DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor + ACTION_TYPE_INVALID: MergeAction._ActionType.ValueType # 0 + ACTION_TYPE_DELETE: MergeAction._ActionType.ValueType # 1 + ACTION_TYPE_INSERT: MergeAction._ActionType.ValueType # 2 + ACTION_TYPE_INSERT_STAR: MergeAction._ActionType.ValueType # 3 + ACTION_TYPE_UPDATE: MergeAction._ActionType.ValueType # 4 + ACTION_TYPE_UPDATE_STAR: MergeAction._ActionType.ValueType # 5 + + class ActionType(_ActionType, metaclass=_ActionTypeEnumTypeWrapper): ... + ACTION_TYPE_INVALID: MergeAction.ActionType.ValueType # 0 + ACTION_TYPE_DELETE: MergeAction.ActionType.ValueType # 1 + ACTION_TYPE_INSERT: MergeAction.ActionType.ValueType # 2 + ACTION_TYPE_INSERT_STAR: MergeAction.ActionType.ValueType # 3 + ACTION_TYPE_UPDATE: MergeAction.ActionType.ValueType # 4 + ACTION_TYPE_UPDATE_STAR: MergeAction.ActionType.ValueType # 5 + + class Assignment(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + @property + def key(self) -> global___Expression: + """(Required) The key of the assignment.""" + @property + def value(self) -> global___Expression: + """(Required) The value of the assignment.""" + def __init__( + self, + *, + key: global___Expression | None = ..., + value: global___Expression | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + ACTION_TYPE_FIELD_NUMBER: builtins.int + CONDITION_FIELD_NUMBER: builtins.int + ASSIGNMENTS_FIELD_NUMBER: builtins.int + action_type: global___MergeAction.ActionType.ValueType + """(Required) The action type of the merge action.""" + @property + def condition(self) -> global___Expression: + """(Optional) The condition expression of the merge action.""" + @property + def assignments( + self, + ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ + global___MergeAction.Assignment + ]: + """(Optional) The assignments of the merge action. Required for ActionTypes INSERT and UPDATE.""" + def __init__( + self, + *, + action_type: global___MergeAction.ActionType.ValueType = ..., + condition: global___Expression | None = ..., + assignments: collections.abc.Iterable[global___MergeAction.Assignment] | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_condition", b"_condition", "condition", b"condition" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_condition", + b"_condition", + "action_type", + b"action_type", + "assignments", + b"assignments", + "condition", + b"condition", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_condition", b"_condition"] + ) -> typing_extensions.Literal["condition"] | None: ... + +global___MergeAction = MergeAction diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d31d8fa85ea14..cdacc719aaca8 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -6018,6 +6018,10 @@ def mergeInto(self, table: str, condition: Column) -> MergeIntoWriter: ... .whenNotMatched().insertAll() ... .whenNotMatchedBySource().delete() ... .merge()) + + Notes + ----- + This method does not support streaming queries. """ ... diff --git a/python/pyspark/sql/merge.py b/python/pyspark/sql/merge.py index c5ddecbee1d1a..bff903cd43926 100644 --- a/python/pyspark/sql/merge.py +++ b/python/pyspark/sql/merge.py @@ -35,7 +35,6 @@ class MergeIntoWriter: """ def __init__(self, df: "DataFrame", table: str, condition: Column): - self._df = df self._spark = df.sparkSession from pyspark.sql.classic.column import _to_java_column diff --git a/python/pyspark/sql/tests/connect/test_parity_dataframe.py b/python/pyspark/sql/tests/connect/test_parity_dataframe.py index 23a55281354be..343f485553a98 100644 --- a/python/pyspark/sql/tests/connect/test_parity_dataframe.py +++ b/python/pyspark/sql/tests/connect/test_parity_dataframe.py @@ -30,10 +30,6 @@ def test_help_command(self): def test_toDF_with_schema_string(self): super().test_toDF_with_schema_string() - @unittest.skip("Spark Connect does not support mergeInto at the moment.") - def test_df_merge_into(self): - super().test_df_merge_into() - if __name__ == "__main__": import unittest From 745d602940a74790b0356da09f4d0e773f0739d5 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Fri, 12 Jul 2024 17:01:39 +0800 Subject: [PATCH 54/64] [SPARK-48876][BUILD] Upgrade Guava used by the connect module to 33.2.1-jre MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? The pr aims to upgrade Guava used by the `connect` module to `33.2.1-jre`. ### Why are the changes needed? The new version bring some fixes and changes as follows: - Changed InetAddress-String conversion methods to preserve the IPv6 scope ID if present. The scope ID can be necessary for IPv6-capable devices with multiple network interfaces. - Added HttpHeaders constants Ad-Auction-Allowed, Permissions-Policy-Report-Only, and Sec-GPC - Fixed a potential NullPointerException in ImmutableMap.Builder on a rare code path。 The full release notes: - https://github.com/google/guava/releases/tag/v33.2.0 - https://github.com/google/guava/releases/tag/v33.2.1 ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Pass GA. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47296 from LuciferYang/connect-guava-33.2.1. Authored-by: yangjie01 Signed-off-by: Kent Yao --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index afeb46e2450a6..a70b4eae4442d 100644 --- a/pom.xml +++ b/pom.xml @@ -288,7 +288,7 @@ true - 33.1.0-jre + 33.2.1-jre 1.0.2 1.62.2 1.1.3 From 918ca333a900ac999351ee06855f17cc7b7d9ad5 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 12 Jul 2024 17:52:55 +0800 Subject: [PATCH 55/64] [SPARK-48878][PYTHON][DOCS] Add doctests for `options` in json functions ### What changes were proposed in this pull request? Add doctests for `options` in json functions ### Why are the changes needed? test coverage, we never test `options` in `from_json` and `to_json` before since it is a new underlying implementation in Spark Connect, we should explicitly test it ### Does this PR introduce _any_ user-facing change? doc changes ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #47319 from zhengruifeng/from_json_option. Lead-authored-by: Kent Yao Co-authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 42 +++++++++++++++++++++---- 1 file changed, 36 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 0b464aa207103..9e0c0700ae04c 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15793,6 +15793,20 @@ def from_json( +---------+ |[1, 2, 3]| +---------+ + + Example 6: Parsing JSON with specified options + + >>> import pyspark.sql.functions as sf + >>> df = spark.createDataFrame([(1, '''{a:123}'''), (2, '''{"a":456}''')], ("key", "value")) + >>> parsed1 = sf.from_json(df.value, "a INT") + >>> parsed2 = sf.from_json(df.value, "a INT", {"allowUnquotedFieldNames": "true"}) + >>> df.select("value", parsed1, parsed2).show() + +---------+----------------+----------------+ + | value|from_json(value)|from_json(value)| + +---------+----------------+----------------+ + | {a:123}| {NULL}| {123}| + |{"a":456}| {456}| {456}| + +---------+----------------+----------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -16113,6 +16127,19 @@ def to_json(col: "ColumnOrName", options: Optional[Dict[str, str]] = None) -> Co +---------------+ |["Alice","Bob"]| +---------------+ + + Example 6: Converting to JSON with specified options + + >>> import pyspark.sql.functions as sf + >>> df = spark.sql("SELECT (DATE('2022-02-22'), 1) AS date") + >>> json1 = sf.to_json(df.date) + >>> json2 = sf.to_json(df.date, {"dateFormat": "yyyy/MM/dd"}) + >>> df.select("date", json1, json2).show(truncate=False) + +---------------+------------------------------+------------------------------+ + |date |to_json(date) |to_json(date) | + +---------------+------------------------------+------------------------------+ + |{2022-02-22, 1}|{"col1":"2022-02-22","col2":1}|{"col1":"2022/02/22","col2":1}| + +---------------+------------------------------+------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -16150,12 +16177,15 @@ def schema_of_json(json: Union[Column, str], options: Optional[Dict[str, str]] = Examples -------- - >>> df = spark.range(1) - >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect() - [Row(json='STRUCT')] - >>> schema = schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'}) - >>> df.select(schema.alias("json")).collect() - [Row(json='STRUCT')] + >>> import pyspark.sql.functions as sf + >>> parsed1 = sf.schema_of_json(sf.lit('{"a": 0}')) + >>> parsed2 = sf.schema_of_json('{a: 1}', {'allowUnquotedFieldNames':'true'}) + >>> spark.range(1).select(parsed1, parsed2).show() + +------------------------+----------------------+ + |schema_of_json({"a": 0})|schema_of_json({a: 1})| + +------------------------+----------------------+ + | STRUCT| STRUCT| + +------------------------+----------------------+ """ from pyspark.sql.classic.column import _create_column_from_literal, _to_java_column From d74785359c50bf966cfe892d3a9eae1a06341db2 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 12 Jul 2024 19:08:24 +0900 Subject: [PATCH 56/64] [SPARK-48666][SQL] Do not push down filter if it contains PythonUDFs ### What changes were proposed in this pull request? This PR proposes to prevent pushing down Python UDFs. This PR uses the same approach as https://github.com/apache/spark/pull/47033, therefore added the author as a co-author, but simplifies the change. Extracting filters to push down happens first https://github.com/apache/spark/blob/cbe6846c477bc8b6d94385ddd0097c4e97b05d41/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala#L46 https://github.com/apache/spark/blob/cbe6846c477bc8b6d94385ddd0097c4e97b05d41/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala#L211 https://github.com/apache/spark/blob/cbe6846c477bc8b6d94385ddd0097c4e97b05d41/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala#L51 Before extracting Python UDFs https://github.com/apache/spark/blob/cbe6846c477bc8b6d94385ddd0097c4e97b05d41/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala#L80 Here is full stacktrace: ``` [INTERNAL_ERROR] Cannot evaluate expression: pyUDF(cast(input[0, bigint, true] as string)) SQLSTATE: XX000 org.apache.spark.SparkException: [INTERNAL_ERROR] Cannot evaluate expression: pyUDF(cast(input[0, bigint, true] as string)) SQLSTATE: XX000 at org.apache.spark.SparkException$.internalError(SparkException.scala:92) at org.apache.spark.SparkException$.internalError(SparkException.scala:96) at org.apache.spark.sql.errors.QueryExecutionErrors$.cannotEvaluateExpressionError(QueryExecutionErrors.scala:65) at org.apache.spark.sql.catalyst.expressions.FoldableUnevaluable.eval(Expression.scala:387) at org.apache.spark.sql.catalyst.expressions.FoldableUnevaluable.eval$(Expression.scala:386) at org.apache.spark.sql.catalyst.expressions.PythonUDF.eval(PythonUDF.scala:72) at org.apache.spark.sql.catalyst.expressions.UnaryExpression.eval(Expression.scala:563) at org.apache.spark.sql.catalyst.expressions.IsNotNull.eval(nullExpressions.scala:403) at org.apache.spark.sql.catalyst.expressions.InterpretedPredicate.eval(predicates.scala:53) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.$anonfun$prunePartitionsByFilter$1(ExternalCatalogUtils.scala:189) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.$anonfun$prunePartitionsByFilter$1$adapted(ExternalCatalogUtils.scala:188) at scala.collection.immutable.List.filter(List.scala:516) at scala.collection.immutable.List.filter(List.scala:79) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.prunePartitionsByFilter(ExternalCatalogUtils.scala:188) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.listPartitionsByFilter(InMemoryCatalog.scala:604) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener.listPartitionsByFilter(ExternalCatalogWithListener.scala:262) at org.apache.spark.sql.catalyst.catalog.SessionCatalog.listPartitionsByFilter(SessionCatalog.scala:1358) at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.listPartitionsByFilter(ExternalCatalogUtils.scala:168) at org.apache.spark.sql.execution.datasources.CatalogFileIndex.filterPartitions(CatalogFileIndex.scala:74) at org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(PruneFileSourcePartitions.scala:72) at org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions$$anonfun$apply$1.applyOrElse(PruneFileSourcePartitions.scala:50) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$1(TreeNode.scala:470) at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(origin.scala:84) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:470) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$transformDownWithPruning$3(TreeNode.scala:475) at org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren(TreeNode.scala:1251) at org.apache.spark.sql.catalyst.trees.BinaryLike.mapChildren$(TreeNode.scala:1250) at org.apache.spark.sql.catalyst.plans.logical.Join.mapChildren(basicLogicalOperators.scala:552) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDownWithPruning(TreeNode.scala:475) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.org$apache$spark$sql$catalyst$plans$logical$AnalysisHelper$$super$transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning(AnalysisHelper.scala:330) at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper.transformDownWithPruning$(AnalysisHelper.scala:326) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan.transformDownWithPruning(LogicalPlan.scala:37) at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:446) at org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions$.apply(PruneFileSourcePartitions.scala:50) at org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions$.apply(PruneFileSourcePartitions.scala:35) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$2(RuleExecutor.scala:226) at scala.collection.LinearSeqOps.foldLeft(LinearSeq.scala:183) at scala.collection.LinearSeqOps.foldLeft$(LinearSeq.scala:179) at scala.collection.immutable.List.foldLeft(List.scala:79) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1(RuleExecutor.scala:223) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$execute$1$adapted(RuleExecutor.scala:215) at scala.collection.immutable.List.foreach(List.scala:334) at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:215) at org.apache.spark.sql.catalyst.rules.RuleExecutor.$anonfun$executeAndTrack$1(RuleExecutor.scala:186) at org.apache.spark.sql.catalyst.QueryPlanningTracker$.withTracker(QueryPlanningTracker.scala:89) at org.apache.spark.sql.catalyst.rules.RuleExecutor.executeAndTrack(RuleExecutor.scala:186) at org.apache.spark.sql.execution.QueryExecution.$anonfun$optimizedPlan$1(QueryExecution.scala:167) at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:138) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$2(QueryExecution.scala:234) at org.apache.spark.sql.execution.QueryExecution$.withInternalError(QueryExecution.scala:608) at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:234) at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:923) at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:233) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:163) at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:159) at org.apache.spark.sql.execution.python.PythonUDFSuite.$anonfun$new$19(PythonUDFSuite.scala:136) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally(SparkErrorUtils.scala:64) at org.apache.spark.util.SparkErrorUtils.tryWithSafeFinally$(SparkErrorUtils.scala:61) at org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:99) at org.apache.spark.sql.test.SQLTestUtilsBase.withTable(SQLTestUtils.scala:307) at org.apache.spark.sql.test.SQLTestUtilsBase.withTable$(SQLTestUtils.scala:305) at org.apache.spark.sql.execution.python.PythonUDFSuite.withTable(PythonUDFSuite.scala:25) at org.apache.spark.sql.execution.python.PythonUDFSuite.$anonfun$new$18(PythonUDFSuite.scala:130) at scala.runtime.java8.JFunction0$mcV$sp.apply(JFunction0$mcV$sp.scala:18) at org.scalatest.enablers.Timed$$anon$1.timeoutAfter(Timed.scala:127) at org.scalatest.concurrent.TimeLimits$.failAfterImpl(TimeLimits.scala:282) at org.scalatest.concurrent.TimeLimits.failAfter(TimeLimits.scala:231) at org.scalatest.concurrent.TimeLimits.failAfter$(TimeLimits.scala:230) at org.apache.spark.SparkFunSuite.failAfter(SparkFunSuite.scala:69) at org.apache.spark.SparkFunSuite.$anonfun$test$2(SparkFunSuite.scala:155) at org.scalatest.OutcomeOf.outcomeOf(OutcomeOf.scala:85) at org.scalatest.OutcomeOf.outcomeOf$(OutcomeOf.scala:83) at org.scalatest.OutcomeOf$.outcomeOf(OutcomeOf.scala:104) at org.scalatest.Transformer.apply(Transformer.scala:22) at org.scalatest.Transformer.apply(Transformer.scala:20) at org.scalatest.funsuite.AnyFunSuiteLike$$anon$1.apply(AnyFunSuiteLike.scala:226) at org.apache.spark.SparkFunSuite.withFixture(SparkFunSuite.scala:227) at org.scalatest.funsuite.AnyFunSuiteLike.invokeWithFixture$1(AnyFunSuiteLike.scala:224) at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTest$1(AnyFunSuiteLike.scala:236) at org.scalatest.SuperEngine.runTestImpl(Engine.scala:306) at org.scalatest.funsuite.AnyFunSuiteLike.runTest(AnyFunSuiteLike.scala:236) at org.scalatest.funsuite.AnyFunSuiteLike.runTest$(AnyFunSuiteLike.scala:218) at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterEach$$super$runTest(SparkFunSuite.scala:69) at org.scalatest.BeforeAndAfterEach.runTest(BeforeAndAfterEach.scala:234) at org.scalatest.BeforeAndAfterEach.runTest$(BeforeAndAfterEach.scala:227) at org.apache.spark.SparkFunSuite.runTest(SparkFunSuite.scala:69) at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$runTests$1(AnyFunSuiteLike.scala:269) at org.scalatest.SuperEngine.$anonfun$runTestsInBranch$1(Engine.scala:413) at scala.collection.immutable.List.foreach(List.scala:334) at org.scalatest.SuperEngine.traverseSubNodes$1(Engine.scala:401) at org.scalatest.SuperEngine.runTestsInBranch(Engine.scala:396) at org.scalatest.SuperEngine.runTestsImpl(Engine.scala:475) at org.scalatest.funsuite.AnyFunSuiteLike.runTests(AnyFunSuiteLike.scala:269) at org.scalatest.funsuite.AnyFunSuiteLike.runTests$(AnyFunSuiteLike.scala:268) at org.scalatest.funsuite.AnyFunSuite.runTests(AnyFunSuite.scala:1564) at org.scalatest.Suite.run(Suite.scala:1114) at org.scalatest.Suite.run$(Suite.scala:1096) at org.scalatest.funsuite.AnyFunSuite.org$scalatest$funsuite$AnyFunSuiteLike$$super$run(AnyFunSuite.scala:1564) at org.scalatest.funsuite.AnyFunSuiteLike.$anonfun$run$1(AnyFunSuiteLike.scala:273) at org.scalatest.SuperEngine.runImpl(Engine.scala:535) at org.scalatest.funsuite.AnyFunSuiteLike.run(AnyFunSuiteLike.scala:273) at org.scalatest.funsuite.AnyFunSuiteLike.run$(AnyFunSuiteLike.scala:272) at org.apache.spark.SparkFunSuite.org$scalatest$BeforeAndAfterAll$$super$run(SparkFunSuite.scala:69) at org.scalatest.BeforeAndAfterAll.liftedTree1$1(BeforeAndAfterAll.scala:213) at org.scalatest.BeforeAndAfterAll.run(BeforeAndAfterAll.scala:210) at org.scalatest.BeforeAndAfterAll.run$(BeforeAndAfterAll.scala:208) at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:69) at org.scalatest.tools.SuiteRunner.run(SuiteRunner.scala:47) at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13(Runner.scala:1321) at org.scalatest.tools.Runner$.$anonfun$doRunRunRunDaDoRunRun$13$adapted(Runner.scala:1315) at scala.collection.immutable.List.foreach(List.scala:334) at org.scalatest.tools.Runner$.doRunRunRunDaDoRunRun(Runner.scala:1315) at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24(Runner.scala:992) at org.scalatest.tools.Runner$.$anonfun$runOptionallyWithPassFailReporter$24$adapted(Runner.scala:970) at org.scalatest.tools.Runner$.withClassLoaderAndDispatchReporter(Runner.scala:1481) at org.scalatest.tools.Runner$.runOptionallyWithPassFailReporter(Runner.scala:970) at org.scalatest.tools.Runner$.run(Runner.scala:798) at org.scalatest.tools.Runner.run(Runner.scala) at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.runScalaTest2or3(ScalaTestRunner.java:43) at org.jetbrains.plugins.scala.testingSupport.scalaTest.ScalaTestRunner.main(ScalaTestRunner.java:26) ``` ### Why are the changes needed? In order for end users to use Python UDFs against partitioned columns. ### Does this PR introduce _any_ user-facing change? Yes, this fixes a bug - this PR allows to use Python UDF in partitioned columns. ### How was this patch tested? Unittest added. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47033 Closes #47313 from HyukjinKwon/SPARK-48666. Lead-authored-by: Hyukjin Kwon Co-authored-by: Wei Zheng Signed-off-by: Hyukjin Kwon --- .../datasources/PruneFileSourcePartitions.scala | 7 ++++++- .../datasources/v2/FileScanBuilder.scala | 7 +++++-- .../sql/execution/python/PythonUDFSuite.scala | 16 ++++++++++++++-- .../execution/PruneHiveTablePartitions.scala | 9 +++++++-- 4 files changed, 32 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 1dffea4e1bc87..d5923a577daac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -63,7 +63,12 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _)) if filters.nonEmpty && fsRelation.partitionSchema.nonEmpty => val normalizedFilters = DataSourceStrategy.normalizeExprs( - filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), + filters.filter { f => + f.deterministic && + !SubqueryExpression.hasSubquery(f) && + // Python UDFs might exist because this rule is applied before ``ExtractPythonUDFs``. + !f.exists(_.isInstanceOf[PythonUDF]) + }, logicalRelation.output) val (partitionKeyFilters, _) = DataSourceUtils .getPartitionFiltersAndDataFilters(partitionSchema, normalizedFilters) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala index 447a36fe622c9..7e0bc25a9a1e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.v2 import scala.collection.mutable import org.apache.spark.sql.{sources, SparkSession} -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDF, SubqueryExpression} import org.apache.spark.sql.connector.expressions.filter.Predicate import org.apache.spark.sql.connector.read.{ScanBuilder, SupportsPushDownRequiredColumns} import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, DataSourceUtils, PartitioningAwareFileIndex, PartitioningUtils} @@ -73,7 +73,10 @@ abstract class FileScanBuilder( val (deterministicFilters, nonDeterminsticFilters) = filters.partition(_.deterministic) val (partitionFilters, dataFilters) = DataSourceUtils.getPartitionFiltersAndDataFilters(partitionSchema, deterministicFilters) - this.partitionFilters = partitionFilters + this.partitionFilters = partitionFilters.filter { f => + // Python UDFs might exist because this rule is applied before ``ExtractPythonUDFs``. + !SubqueryExpression.hasSubquery(f) && !f.exists(_.isInstanceOf[PythonUDF]) + } this.dataFilters = dataFilters val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] for (filterExpr <- dataFilters) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala index 2e56ad0ab4160..b5d7c4cf08670 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonUDFSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.execution.python -import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest} -import org.apache.spark.sql.functions.{array, count, transform} +import org.apache.spark.sql.{AnalysisException, IntegratedUDFTestUtils, QueryTest, Row} +import org.apache.spark.sql.functions.{array, col, count, transform} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types.LongType @@ -124,4 +124,16 @@ class PythonUDFSuite extends QueryTest with SharedSparkSession { context = ExpectedContext( "transform", s".*${this.getClass.getSimpleName}.*")) } + + test("SPARK-48666: Python UDF execution against partitioned column") { + assume(shouldTestPythonUDFs) + withTable("t") { + spark.range(1).selectExpr("id AS t", "(id + 1) AS p").write.partitionBy("p").saveAsTable("t") + val table = spark.table("t") + val newTable = table.withColumn("new_column", pythonTestUDF(table("p"))) + val df = newTable.as("t1").join( + newTable.as("t2"), col("t1.new_column") === col("t2.new_column")) + checkAnswer(df, Row(0, 1, 1, 0, 1, 1)) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala index 395ee86579e57..779562bed5b0f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/PruneHiveTablePartitions.scala @@ -22,7 +22,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.analysis.CastSupport import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, ExpressionSet, PredicateHelper, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeSet, Expression, ExpressionSet, PredicateHelper, PythonUDF, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.FilterEstimation @@ -50,7 +50,12 @@ private[sql] class PruneHiveTablePartitions(session: SparkSession) filters: Seq[Expression], relation: HiveTableRelation): ExpressionSet = { val normalizedFilters = DataSourceStrategy.normalizeExprs( - filters.filter(f => f.deterministic && !SubqueryExpression.hasSubquery(f)), relation.output) + filters.filter { f => + f.deterministic && + !SubqueryExpression.hasSubquery(f) && + // Python UDFs might exist because this rule is applied before ``ExtractPythonUDFs``. + !f.exists(_.isInstanceOf[PythonUDF]) + }, relation.output) val partitionColumnSet = AttributeSet(relation.partitionCols) ExpressionSet( normalizedFilters.flatMap(extractPredicatesWithinOutputSet(_, partitionColumnSet))) From 236d95738b6e50bc9ec54955e86d01b6dcf11c0e Mon Sep 17 00:00:00 2001 From: jackylee-ch Date: Fri, 12 Jul 2024 18:15:56 +0800 Subject: [PATCH 57/64] [SPARK-48845][SQL] GenericUDF catch exceptions from children ### What changes were proposed in this pull request? This pr is trying to fix the syntax issues with GenericUDF since 3.5.0. The problem arose from DeferredObject currently passing a value instead of a function, which prevented users from catching exceptions in GenericUDF, resulting in semantic differences. Here is an example case we encountered. Originally, the semantics were that udf_exception would throw an exception, while udf_catch_exception could catch the exception and return a null value. However, currently, any exception encountered by udf_exception will cause the program to fail. ``` select udf_catch_exception(udf_exception(col1)) from table ``` ### Why are the changes needed? For before Spark 3.5, we directly made the GenericUDF's DeferredObject lazy and evaluated the children in `function.evaluate(deferredObjects)`. Now, we would run the children's code first. If an exception is thrown, we would make it lazy to GenericUDF's DeferredObject. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Newly added UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47268 from jackylee-ch/generic_udf_catch_exception_from_child_func. Lead-authored-by: jackylee-ch Co-authored-by: Kent Yao Signed-off-by: Kent Yao --- .../spark/sql/hive/hiveUDFEvaluators.scala | 12 +++-- .../org/apache/spark/sql/hive/hiveUDFs.scala | 22 +++++--- .../sql/hive/execution/UDFCatchException.java | 51 +++++++++++++++++++ .../sql/hive/execution/UDFThrowException.java | 26 ++++++++++ .../sql/hive/execution/HiveUDFSuite.scala | 23 +++++++++ 5 files changed, 124 insertions(+), 10 deletions(-) create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFCatchException.java create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFThrowException.java diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFEvaluators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFEvaluators.scala index 2ebe724f399a7..409be67f7af4c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFEvaluators.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFEvaluators.scala @@ -129,7 +129,11 @@ class HiveGenericUDFEvaluator( override def returnType: DataType = inspectorToDataType(returnInspector) def setArg(index: Int, arg: Any): Unit = - deferredObjects(index).asInstanceOf[DeferredObjectAdapter].set(arg) + deferredObjects(index).asInstanceOf[DeferredObjectAdapter].set(() => arg) + + def setException(index: Int, exp: Throwable): Unit = { + deferredObjects(index).asInstanceOf[DeferredObjectAdapter].set(() => throw exp) + } override def doEvaluate(): Any = unwrapper(function.evaluate(deferredObjects)) } @@ -139,10 +143,10 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector, dataType: DataTyp extends DeferredObject with HiveInspectors { private val wrapper = wrapperFor(oi, dataType) - private var func: Any = _ - def set(func: Any): Unit = { + private var func: () => Any = _ + def set(func: () => Any): Unit = { this.func = func } override def prepare(i: Int): Unit = {} - override def get(): AnyRef = wrapper(func).asInstanceOf[AnyRef] + override def get(): AnyRef = wrapper(func()).asInstanceOf[AnyRef] } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 6efdb676ccbdc..227c6a618e3d4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -136,7 +136,13 @@ private[hive] case class HiveGenericUDF( override def eval(input: InternalRow): Any = { children.zipWithIndex.foreach { - case (child, idx) => evaluator.setArg(idx, child.eval(input)) + case (child, idx) => + try { + evaluator.setArg(idx, child.eval(input)) + } catch { + case t: Throwable => + evaluator.setException(idx, t) + } } evaluator.evaluate() } @@ -157,10 +163,15 @@ private[hive] case class HiveGenericUDF( val setValues = evals.zipWithIndex.map { case (eval, i) => s""" - |if (${eval.isNull}) { - | $refEvaluator.setArg($i, null); - |} else { - | $refEvaluator.setArg($i, ${eval.value}); + |try { + | ${eval.code} + | if (${eval.isNull}) { + | $refEvaluator.setArg($i, null); + | } else { + | $refEvaluator.setArg($i, ${eval.value}); + | } + |} catch (Throwable t) { + | $refEvaluator.setException($i, t); |} |""".stripMargin } @@ -169,7 +180,6 @@ private[hive] case class HiveGenericUDF( val resultTerm = ctx.freshName("result") ev.copy(code = code""" - |${evals.map(_.code).mkString("\n")} |${setValues.mkString("\n")} |$resultType $resultTerm = ($resultType) $refEvaluator.evaluate(); |boolean ${ev.isNull} = $resultTerm == null; diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFCatchException.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFCatchException.java new file mode 100644 index 0000000000000..242dbeaa63c94 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFCatchException.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDFArgumentException; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDF; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory; + +public class UDFCatchException extends GenericUDF { + + @Override + public ObjectInspector initialize(ObjectInspector[] args) throws UDFArgumentException { + if (args.length != 1) { + throw new UDFArgumentException("Exactly one argument is expected."); + } + return PrimitiveObjectInspectorFactory.javaStringObjectInspector; + } + + @Override + public Object evaluate(GenericUDF.DeferredObject[] args) { + if (args == null) { + return null; + } + try { + return args[0].get(); + } catch (Exception e) { + return null; + } + } + + @Override + public String getDisplayString(String[] children) { + return null; + } +} diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFThrowException.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFThrowException.java new file mode 100644 index 0000000000000..5d6ff6ca40ae5 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFThrowException.java @@ -0,0 +1,26 @@ +/* + * 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.hive.execution; + +import org.apache.hadoop.hive.ql.exec.UDF; + +public class UDFThrowException extends UDF { + public String evaluate(String data) { + return Integer.valueOf(data).toString(); + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index d73f2be3b3f50..2e88b13f0963d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.io.{LongWritable, Writable} import org.apache.spark.{SparkException, SparkFiles, TestUtils} import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.functions.{call_function, max} @@ -801,6 +802,28 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } } } + + test("SPARK-48845: GenericUDF catch exceptions from child UDFs") { + withTable("test_catch_exception") { + withUserDefinedFunction("udf_throw_exception" -> true, "udf_catch_exception" -> true) { + Seq("9", "9-1").toDF("a").write.saveAsTable("test_catch_exception") + sql("CREATE TEMPORARY FUNCTION udf_throw_exception AS " + + s"'${classOf[UDFThrowException].getName}'") + sql("CREATE TEMPORARY FUNCTION udf_catch_exception AS " + + s"'${classOf[UDFCatchException].getName}'") + Seq( + CodegenObjectFactoryMode.FALLBACK.toString, + CodegenObjectFactoryMode.NO_CODEGEN.toString + ).foreach { codegenMode => + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenMode) { + val df = sql( + "SELECT udf_catch_exception(udf_throw_exception(a)) FROM test_catch_exception") + checkAnswer(df, Seq(Row("9"), Row(null))) + } + } + } + } + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { From 8d3d4f9b900dadede3b8e33af830e5ef66682923 Mon Sep 17 00:00:00 2001 From: Ted Jenks Date: Fri, 12 Jul 2024 18:33:20 +0800 Subject: [PATCH 58/64] [SPARK-47307][SQL] Add a config to optionally chunk base64 strings Follow up #45408 ### What changes were proposed in this pull request? [[SPARK-47307](https://issues.apache.org/jira/browse/SPARK-47307)] Add a config to optionally chunk base64 strings ### Why are the changes needed? In #35110, it was incorrectly asserted that: > ApacheCommonBase64 obeys http://www.ietf.org/rfc/rfc2045.txt This is not true as the previous code called: ```java public static byte[] encodeBase64(byte[] binaryData) ``` Which states: > Encodes binary data using the base64 algorithm but does not chunk the output. However, the RFC 2045 (MIME) base64 encoder does chunk by default. This now means that any Spark encoded base64 strings cannot be decoded by encoders that do not implement RFC 2045. The docs state RFC 4648. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing test suite. ### Was this patch authored or co-authored using generative AI tooling? No Closes #47303 from wForget/SPARK-47307. Lead-authored-by: Ted Jenks Co-authored-by: wforget <643348094@qq.com> Co-authored-by: Kent Yao Co-authored-by: Ted Chester Jenks Signed-off-by: Kent Yao --- .../explain-results/function_base64.explain | 2 +- .../expressions/stringExpressions.scala | 40 +++++++++++++------ .../apache/spark/sql/internal/SQLConf.scala | 11 +++++ .../expressions/StringExpressionsSuite.scala | 13 ++++++ 4 files changed, 53 insertions(+), 13 deletions(-) diff --git a/connect/common/src/test/resources/query-tests/explain-results/function_base64.explain b/connect/common/src/test/resources/query-tests/explain-results/function_base64.explain index f80f3522190d8..d3a250919ea5d 100644 --- a/connect/common/src/test/resources/query-tests/explain-results/function_base64.explain +++ b/connect/common/src/test/resources/query-tests/explain-results/function_base64.explain @@ -1,2 +1,2 @@ -Project [base64(cast(g#0 as binary)) AS base64(CAST(g AS BINARY))#0] +Project [static_invoke(Base64.encode(cast(g#0 as binary), false)) AS base64(CAST(g AS BINARY))#0] +- LocalRelation , [id#0L, a#0, b#0, d#0, e#0, f#0, g#0] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index f25f58731c8cd..b188b9c2630fa 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -2682,24 +2682,40 @@ case class Chr(child: Expression) """, since = "1.5.0", group = "string_funcs") -case class Base64(child: Expression) - extends UnaryExpression with ImplicitCastInputTypes with NullIntolerant { +case class Base64(child: Expression, chunkBase64: Boolean) + extends UnaryExpression with RuntimeReplaceable with ImplicitCastInputTypes { + + def this(expr: Expression) = this(expr, SQLConf.get.chunkBase64StringEnabled) override def dataType: DataType = SQLConf.get.defaultStringType override def inputTypes: Seq[DataType] = Seq(BinaryType) - protected override def nullSafeEval(bytes: Any): Any = { - UTF8String.fromBytes(JBase64.getMimeEncoder.encode(bytes.asInstanceOf[Array[Byte]])) - } + override def replacement: Expression = StaticInvoke( + classOf[Base64], + dataType, + "encode", + Seq(child, Literal(chunkBase64, BooleanType)), + Seq(BinaryType, BooleanType)) - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, (child) => { - s"""${ev.value} = UTF8String.fromBytes( - ${classOf[JBase64].getName}.getMimeEncoder().encode($child)); - """}) - } + override def toString: String = s"$prettyName($child)" - override protected def withNewChildInternal(newChild: Expression): Base64 = copy(child = newChild) + override protected def withNewChildInternal(newChild: Expression): Expression = + copy(child = newChild) +} + +object Base64 { + def apply(expr: Expression): Base64 = new Base64(expr) + + private lazy val nonChunkEncoder = JBase64.getMimeEncoder(-1, Array()) + + def encode(input: Array[Byte], chunkBase64: Boolean): UTF8String = { + val encoder = if (chunkBase64) { + JBase64.getMimeEncoder + } else { + nonChunkEncoder + } + UTF8String.fromBytes(encoder.encode(input)) + } } /** 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 6ca831f99304b..65beb21d59d92 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 @@ -3525,6 +3525,15 @@ object SQLConf { .booleanConf .createWithDefault(false) + val CHUNK_BASE64_STRING_ENABLED = buildConf("spark.sql.legacy.chunkBase64String.enabled") + .internal() + .doc("Whether to truncate string generated by the `Base64` function. When true, base64" + + " strings generated by the base64 function are chunked into lines of at most 76" + + " characters. When false, the base64 strings are not chunked.") + .version("3.5.2") + .booleanConf + .createWithDefault(false) + val ENABLE_DEFAULT_COLUMNS = buildConf("spark.sql.defaultColumn.enabled") .internal() @@ -5856,6 +5865,8 @@ class SQLConf extends Serializable with Logging with SqlApiConf { def ansiRelationPrecedence: Boolean = ansiEnabled && getConf(ANSI_RELATION_PRECEDENCE) + def chunkBase64StringEnabled: Boolean = getConf(CHUNK_BASE64_STRING_ENABLED) + def timestampType: AtomicType = getConf(TIMESTAMP_TYPE) match { case "TIMESTAMP_LTZ" => // For historical reason, the TimestampType maps to TIMESTAMP WITH LOCAL TIME ZONE diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala index ebd7245434819..2ad8652f2b314 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringExpressionsSuite.scala @@ -509,6 +509,19 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { GenerateUnsafeProjection.generate(StringDecode(b, Literal("\"quote")).replacement :: Nil) } + test("SPARK-47307: base64 encoding without chunking") { + val longString = "a" * 58 + val encoded = "YWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYQ==" + withSQLConf(SQLConf.CHUNK_BASE64_STRING_ENABLED.key -> "false") { + checkEvaluation(Base64(Literal(longString.getBytes)), encoded) + } + val chunkEncoded = + s"YWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFhYWFh\r\nYQ==" + withSQLConf(SQLConf.CHUNK_BASE64_STRING_ENABLED.key -> "true") { + checkEvaluation(Base64(Literal(longString.getBytes)), chunkEncoded) + } + } + test("initcap unit test") { checkEvaluation(InitCap(Literal.create(null, StringType)), null) checkEvaluation(InitCap(Literal("a b")), "A B") From e20db137d2de26594b38c7e257a3d863de882022 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Fri, 12 Jul 2024 20:21:03 +0900 Subject: [PATCH 59/64] [SPARK-48510] 2/2] Support UDAF `toColumn` API in Spark Connect ### What changes were proposed in this pull request? This PR follows https://github.com/apache/spark/pull/46245 to add support `udaf.toColumn` API in Spark Connect. Here we introduce a new Protobuf message, `proto.TypedAggregateExpression`, that includes a serialized UDF packet. On the server, we unpack it into an `Aggregator` object and generate a real `TypedAggregateExpression` instance with the encoder information passed along with the UDF. ### Why are the changes needed? Because the `toColumn` API is not supported in the previous PR. ### Does this PR introduce _any_ user-facing change? Yes, from now on users could create typed UDAF using `udaf.toColumn` API/. ### How was this patch tested? New tests. ### Was this patch authored or co-authored using generative AI tooling? Nope. Closes #46849 from xupefei/connect-udaf-tocolumn. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../protobuf/spark/connect/expressions.proto | 6 + .../connect/planner/SparkConnectPlanner.scala | 63 +++++++- .../spark/sql/expressions/Aggregator.scala | 53 ++++++- .../sql/expressions/UserDefinedFunction.scala | 2 +- .../sql/UserDefinedFunctionE2ETestSuite.scala | 72 +++++++-- .../sql/connect/proto/expressions_pb2.py | 146 +++++++++--------- .../sql/connect/proto/expressions_pb2.pyi | 30 ++++ 7 files changed, 278 insertions(+), 94 deletions(-) diff --git a/connect/common/src/main/protobuf/spark/connect/expressions.proto b/connect/common/src/main/protobuf/spark/connect/expressions.proto index 860e923576161..3a91371fd3b25 100644 --- a/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -51,6 +51,7 @@ message Expression { CallFunction call_function = 16; NamedArgumentExpression named_argument_expression = 17; MergeAction merge_action = 19; + TypedAggregateExpression typed_aggregate_expression = 20; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. @@ -402,6 +403,11 @@ message JavaUDF { bool aggregate = 3; } +message TypedAggregateExpression { + // (Required) The aggregate function object packed into bytes. + ScalarScalaUDF scalar_scala_udf = 1; +} + message CallFunction { // (Required) Unparsed name of the SQL function. string function_name = 1; diff --git a/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 449e923beae3a..4702f09a14c29 100644 --- a/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -51,7 +51,7 @@ import org.apache.spark.sql.catalyst.analysis.{GlobalTempView, LocalTempView, Mu import org.apache.spark.sql.catalyst.encoders.{AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.BloomFilterAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, BloomFilterAggregate} import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans.{Cross, FullOuter, Inner, JoinType, LeftAnti, LeftOuter, LeftSemi, RightOuter, UsingJoin} import org.apache.spark.sql.catalyst.plans.logical @@ -67,6 +67,7 @@ import org.apache.spark.sql.connect.service.{ExecuteHolder, SessionHolder, Spark import org.apache.spark.sql.connect.utils.MetricGenerator import org.apache.spark.sql.errors.QueryCompilationErrors import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.execution.command.CreateViewCommand import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -1455,7 +1456,7 @@ class SparkConnectPlanner( } val projection = rel.getExpressionsList.asScala.toSeq - .map(transformExpression) + .map(transformExpression(_, Some(baseRel))) .map(toNamedExpression) logical.Project(projectList = projection, child = baseRel) @@ -1472,21 +1473,40 @@ class SparkConnectPlanner( * Catalyst expression */ @DeveloperApi - def transformExpression(exp: proto.Expression): Expression = if (exp.hasCommon) { + def transformExpression(exp: proto.Expression): Expression = transformExpression(exp, None) + + /** + * Transforms an input protobuf expression into the Catalyst expression. This is usually not + * called directly. Typically the planner will traverse the expressions automatically, only + * plugins are expected to manually perform expression transformations. + * + * @param exp + * the input expression + * @param baseRelationOpt + * inputs of the base relation that contains this expression + * @return + * Catalyst expression + */ + @DeveloperApi + def transformExpression( + exp: proto.Expression, + baseRelationOpt: Option[LogicalPlan]): Expression = if (exp.hasCommon) { try { val origin = exp.getCommon.getOrigin PySparkCurrentOrigin.set( origin.getPythonOrigin.getFragment, origin.getPythonOrigin.getCallSite) - withOrigin { doTransformExpression(exp) } + withOrigin { doTransformExpression(exp, baseRelationOpt) } } finally { PySparkCurrentOrigin.clear() } } else { - doTransformExpression(exp) + doTransformExpression(exp, baseRelationOpt) } - private def doTransformExpression(exp: proto.Expression): Expression = { + private def doTransformExpression( + exp: proto.Expression, + baseRelationOpt: Option[LogicalPlan]): Expression = { exp.getExprTypeCase match { case proto.Expression.ExprTypeCase.LITERAL => transformLiteral(exp.getLiteral) case proto.Expression.ExprTypeCase.UNRESOLVED_ATTRIBUTE => @@ -1523,6 +1543,8 @@ class SparkConnectPlanner( transformNamedArgumentExpression(exp.getNamedArgumentExpression) case proto.Expression.ExprTypeCase.MERGE_ACTION => transformMergeAction(exp.getMergeAction) + case proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION => + transformTypedAggregateExpression(exp.getTypedAggregateExpression, baseRelationOpt) case _ => throw InvalidPlanInput( s"Expression with ID: ${exp.getExprTypeCase.getNumber} is not supported") @@ -2584,8 +2606,35 @@ class SparkConnectPlanner( if expr.getUnresolvedFunction.getFunctionName == "reduce" => // The reduce func needs the input data attribute, thus handle it specially here transformTypedReduceExpression(expr.getUnresolvedFunction, plan.output) - case _ => transformExpression(expr) + case _ => transformExpression(expr, Some(plan)) + } + } + + private def transformTypedAggregateExpression( + expr: proto.TypedAggregateExpression, + baseRelationOpt: Option[LogicalPlan]): AggregateExpression = { + val udf = expr.getScalarScalaUdf + assert(udf.getAggregate) + + val udfPacket = unpackScalaUDF[UdfPacket](udf) + assert(udfPacket.inputEncoders.size == 1, "UDAF should have exactly one input encoder") + + val aggregator = udfPacket.function.asInstanceOf[Aggregator[Any, Any, Any]] + val tae = + TypedAggregateExpression(aggregator)(aggregator.bufferEncoder, aggregator.outputEncoder) + val taeWithInput = baseRelationOpt match { + case Some(baseRelation) => + val inputEncoder = TypedScalaUdf.encoderFor( + udfPacket.inputEncoders.head, + "input", + Some(baseRelation.output)) + TypedAggUtils + .withInputType(tae, inputEncoder, baseRelation.output) + .asInstanceOf[TypedAggregateExpression] + case _ => + tae } + taeWithInput.toAggregateExpression() } private def transformMergeAction(action: proto.MergeAction): MergeAction = { diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index 91c8fb57c31bf..3dabcdef1567e 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,7 +17,11 @@ package org.apache.spark.sql.expressions -import org.apache.spark.sql.{Encoder, TypedColumn} +import scala.reflect.runtime.universe._ + +import org.apache.spark.connect.proto +import org.apache.spark.sql.{encoderFor, Encoder, TypedColumn} +import org.apache.spark.sql.catalyst.ScalaReflection /** * A base class for user-defined aggregations, which can be used in `Dataset` operations to take @@ -92,9 +96,52 @@ abstract class Aggregator[-IN, BUF, OUT] extends Serializable { def outputEncoder: Encoder[OUT] /** - * Returns this `Aggregator` as a `TypedColumn` that can be used in `Dataset`. operations. + * Returns this `Aggregator` as a `TypedColumn` that can be used in `Dataset` operations. + * @since 4.0.0 */ def toColumn: TypedColumn[IN, OUT] = { - throw new UnsupportedOperationException("toColumn is not implemented.") + val ttpe = getInputTypeTag[IN] + val inputEncoder = ScalaReflection.encoderFor(ttpe) + val udaf = + ScalaUserDefinedFunction( + this, + Seq(inputEncoder), + encoderFor(outputEncoder), + aggregate = true) + + val builder = proto.TypedAggregateExpression.newBuilder() + builder.setScalarScalaUdf(udaf.udf) + val expr = proto.Expression.newBuilder().setTypedAggregateExpression(builder).build() + + new TypedColumn(expr, encoderFor(outputEncoder)) + } + + private final def getInputTypeTag[T]: TypeTag[T] = { + val mirror = runtimeMirror(this.getClass.getClassLoader) + val tpe = mirror.classSymbol(this.getClass).toType + // Find the most generic (last in the tree) Aggregator class + val baseAgg = + tpe.baseClasses + .findLast(_.asClass.toType <:< typeOf[Aggregator[_, _, _]]) + .getOrElse(throw new IllegalStateException("Could not find the Aggregator base class.")) + val typeArgs = tpe.baseType(baseAgg).typeArgs + assert( + typeArgs.length == 3, + s"Aggregator should have 3 type arguments, " + + s"but found ${typeArgs.length}: ${typeArgs.mkString}.") + val inType = typeArgs.head + + import scala.reflect.api._ + TypeTag( + mirror, + new TypeCreator { + def apply[U <: Universe with Singleton](m: Mirror[U]): U#Type = + if (m eq mirror) { + inType.asInstanceOf[U#Type] + } else { + throw new IllegalArgumentException( + s"Type tag defined in $mirror cannot be migrated to other mirrors.") + } + }) } } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index f4499858306a1..dcf7f67551d30 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -107,7 +107,7 @@ case class ScalaUserDefinedFunction private[sql] ( aggregate: Boolean) extends UserDefinedFunction { - private[this] lazy val udf = { + private[expressions] lazy val udf = { val scalaUdfBuilder = proto.ScalarScalaUDF .newBuilder() .setPayload(ByteString.copyFrom(serializedUdfPacket)) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala index 4032a9499c448..4aec0e6348c09 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/UserDefinedFunctionE2ETestSuite.scala @@ -367,17 +367,7 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession test("UDAF custom Aggregator - case class as input types") { val session: SparkSession = spark import session.implicits._ - val agg = new Aggregator[UdafTestInput, (Long, Long), Long] { - override def zero: (Long, Long) = (0L, 0L) - override def reduce(b: (Long, Long), a: UdafTestInput): (Long, Long) = - (b._1 + a.id, b._2 + a.extra) - override def merge(b1: (Long, Long), b2: (Long, Long)): (Long, Long) = - (b1._1 + b2._1, b1._2 + b2._2) - override def finish(reduction: (Long, Long)): Long = reduction._1 + reduction._2 - override def bufferEncoder: Encoder[(Long, Long)] = - Encoders.tuple(Encoders.scalaLong, Encoders.scalaLong) - override def outputEncoder: Encoder[Long] = Encoders.scalaLong - } + val agg = new CompleteUdafTestInputAggregator() spark.udf.register("agg", udaf(agg)) val result = spark .range(10) @@ -388,6 +378,66 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession .head() assert(result == 135) // 45 + 90 } + + test("UDAF custom Aggregator - toColumn") { + val session: SparkSession = spark + import session.implicits._ + val aggCol = new CompleteUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).as[UdafTestInput] + + assert(ds.select(aggCol).head() == 135) // 45 + 90 + assert(ds.agg(aggCol).head().getLong(0) == 135) // 45 + 90 + } + + test("UDAF custom Aggregator - multiple extends - toColumn") { + val session: SparkSession = spark + import session.implicits._ + val aggCol = new CompleteGrandChildUdafTestInputAggregator().toColumn + val ds = spark.range(10).withColumn("extra", col("id") * 2).as[UdafTestInput] + + assert(ds.select(aggCol).head() == 540) // (45 + 90) * 4 + assert(ds.agg(aggCol).head().getLong(0) == 540) // (45 + 90) * 4 + } } case class UdafTestInput(id: Long, extra: Long) + +// An Aggregator that takes [[UdafTestInput]] as input. +final class CompleteUdafTestInputAggregator + extends Aggregator[UdafTestInput, (Long, Long), Long] { + override def zero: (Long, Long) = (0L, 0L) + override def reduce(b: (Long, Long), a: UdafTestInput): (Long, Long) = + (b._1 + a.id, b._2 + a.extra) + override def merge(b1: (Long, Long), b2: (Long, Long)): (Long, Long) = + (b1._1 + b2._1, b1._2 + b2._2) + override def finish(reduction: (Long, Long)): Long = reduction._1 + reduction._2 + override def bufferEncoder: Encoder[(Long, Long)] = + Encoders.tuple(Encoders.scalaLong, Encoders.scalaLong) + override def outputEncoder: Encoder[Long] = Encoders.scalaLong +} + +// Same as [[CompleteUdafTestInputAggregator]] but the input type is not defined. +abstract class IncompleteUdafTestInputAggregator[T] extends Aggregator[T, (Long, Long), Long] { + override def zero: (Long, Long) = (0L, 0L) + override def reduce(b: (Long, Long), a: T): (Long, Long) // Incomplete! + override def merge(b1: (Long, Long), b2: (Long, Long)): (Long, Long) = + (b1._1 + b2._1, b1._2 + b2._2) + override def finish(reduction: (Long, Long)): Long = reduction._1 + reduction._2 + override def bufferEncoder: Encoder[(Long, Long)] = + Encoders.tuple(Encoders.scalaLong, Encoders.scalaLong) + override def outputEncoder: Encoder[Long] = Encoders.scalaLong +} + +// A layer over [[IncompleteUdafTestInputAggregator]] but the input type is still not defined. +abstract class IncompleteChildUdafTestInputAggregator[T] + extends IncompleteUdafTestInputAggregator[T] { + override def finish(reduction: (Long, Long)): Long = (reduction._1 + reduction._2) * 2 +} + +// Another layer that finally defines the input type. +final class CompleteGrandChildUdafTestInputAggregator + extends IncompleteChildUdafTestInputAggregator[UdafTestInput] { + override def reduce(b: (Long, Long), a: UdafTestInput): (Long, Long) = + (b._1 + a.id, b._2 + a.extra) + override def finish(reduction: (Long, Long)): Long = (reduction._1 + reduction._2) * 4 +} diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index b4c4b48de2688..1c1ad2b6ecec5 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -34,7 +34,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xd8/\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_conditionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xc1\x30\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_conditionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -47,75 +47,77 @@ b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" ) _EXPRESSION._serialized_start = 133 - _EXPRESSION._serialized_end = 6237 - _EXPRESSION_WINDOW._serialized_start = 1795 - _EXPRESSION_WINDOW._serialized_end = 2578 - _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 2085 - _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2578 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2352 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2497 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2499 - _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2578 - _EXPRESSION_SORTORDER._serialized_start = 2581 - _EXPRESSION_SORTORDER._serialized_end = 3006 - _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2811 - _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 2919 - _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 2921 - _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 3006 - _EXPRESSION_CAST._serialized_start = 3009 - _EXPRESSION_CAST._serialized_end = 3324 - _EXPRESSION_CAST_EVALMODE._serialized_start = 3210 - _EXPRESSION_CAST_EVALMODE._serialized_end = 3308 - _EXPRESSION_LITERAL._serialized_start = 3327 - _EXPRESSION_LITERAL._serialized_end = 4890 - _EXPRESSION_LITERAL_DECIMAL._serialized_start = 4162 - _EXPRESSION_LITERAL_DECIMAL._serialized_end = 4279 - _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 4281 - _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4379 - _EXPRESSION_LITERAL_ARRAY._serialized_start = 4382 - _EXPRESSION_LITERAL_ARRAY._serialized_end = 4512 - _EXPRESSION_LITERAL_MAP._serialized_start = 4515 - _EXPRESSION_LITERAL_MAP._serialized_end = 4742 - _EXPRESSION_LITERAL_STRUCT._serialized_start = 4745 - _EXPRESSION_LITERAL_STRUCT._serialized_end = 4874 - _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4893 - _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 5079 - _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 5082 - _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 5286 - _EXPRESSION_EXPRESSIONSTRING._serialized_start = 5288 - _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5338 - _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5340 - _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5464 - _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5466 - _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5552 - _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5555 - _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5687 - _EXPRESSION_UPDATEFIELDS._serialized_start = 5690 - _EXPRESSION_UPDATEFIELDS._serialized_end = 5877 - _EXPRESSION_ALIAS._serialized_start = 5879 - _EXPRESSION_ALIAS._serialized_end = 5999 - _EXPRESSION_LAMBDAFUNCTION._serialized_start = 6002 - _EXPRESSION_LAMBDAFUNCTION._serialized_end = 6160 - _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 6162 - _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 6224 - _EXPRESSIONCOMMON._serialized_start = 6239 - _EXPRESSIONCOMMON._serialized_end = 6304 - _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 6307 - _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6671 - _PYTHONUDF._serialized_start = 6674 - _PYTHONUDF._serialized_end = 6878 - _SCALARSCALAUDF._serialized_start = 6881 - _SCALARSCALAUDF._serialized_end = 7095 - _JAVAUDF._serialized_start = 7098 - _JAVAUDF._serialized_end = 7247 - _CALLFUNCTION._serialized_start = 7249 - _CALLFUNCTION._serialized_end = 7357 - _NAMEDARGUMENTEXPRESSION._serialized_start = 7359 - _NAMEDARGUMENTEXPRESSION._serialized_end = 7451 - _MERGEACTION._serialized_start = 7454 - _MERGEACTION._serialized_end = 7966 - _MERGEACTION_ASSIGNMENT._serialized_start = 7676 - _MERGEACTION_ASSIGNMENT._serialized_end = 7782 - _MERGEACTION_ACTIONTYPE._serialized_start = 7785 - _MERGEACTION_ACTIONTYPE._serialized_end = 7952 + _EXPRESSION._serialized_end = 6342 + _EXPRESSION_WINDOW._serialized_start = 1900 + _EXPRESSION_WINDOW._serialized_end = 2683 + _EXPRESSION_WINDOW_WINDOWFRAME._serialized_start = 2190 + _EXPRESSION_WINDOW_WINDOWFRAME._serialized_end = 2683 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_start = 2457 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY._serialized_end = 2602 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_start = 2604 + _EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE._serialized_end = 2683 + _EXPRESSION_SORTORDER._serialized_start = 2686 + _EXPRESSION_SORTORDER._serialized_end = 3111 + _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_start = 2916 + _EXPRESSION_SORTORDER_SORTDIRECTION._serialized_end = 3024 + _EXPRESSION_SORTORDER_NULLORDERING._serialized_start = 3026 + _EXPRESSION_SORTORDER_NULLORDERING._serialized_end = 3111 + _EXPRESSION_CAST._serialized_start = 3114 + _EXPRESSION_CAST._serialized_end = 3429 + _EXPRESSION_CAST_EVALMODE._serialized_start = 3315 + _EXPRESSION_CAST_EVALMODE._serialized_end = 3413 + _EXPRESSION_LITERAL._serialized_start = 3432 + _EXPRESSION_LITERAL._serialized_end = 4995 + _EXPRESSION_LITERAL_DECIMAL._serialized_start = 4267 + _EXPRESSION_LITERAL_DECIMAL._serialized_end = 4384 + _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_start = 4386 + _EXPRESSION_LITERAL_CALENDARINTERVAL._serialized_end = 4484 + _EXPRESSION_LITERAL_ARRAY._serialized_start = 4487 + _EXPRESSION_LITERAL_ARRAY._serialized_end = 4617 + _EXPRESSION_LITERAL_MAP._serialized_start = 4620 + _EXPRESSION_LITERAL_MAP._serialized_end = 4847 + _EXPRESSION_LITERAL_STRUCT._serialized_start = 4850 + _EXPRESSION_LITERAL_STRUCT._serialized_end = 4979 + _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_start = 4998 + _EXPRESSION_UNRESOLVEDATTRIBUTE._serialized_end = 5184 + _EXPRESSION_UNRESOLVEDFUNCTION._serialized_start = 5187 + _EXPRESSION_UNRESOLVEDFUNCTION._serialized_end = 5391 + _EXPRESSION_EXPRESSIONSTRING._serialized_start = 5393 + _EXPRESSION_EXPRESSIONSTRING._serialized_end = 5443 + _EXPRESSION_UNRESOLVEDSTAR._serialized_start = 5445 + _EXPRESSION_UNRESOLVEDSTAR._serialized_end = 5569 + _EXPRESSION_UNRESOLVEDREGEX._serialized_start = 5571 + _EXPRESSION_UNRESOLVEDREGEX._serialized_end = 5657 + _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_start = 5660 + _EXPRESSION_UNRESOLVEDEXTRACTVALUE._serialized_end = 5792 + _EXPRESSION_UPDATEFIELDS._serialized_start = 5795 + _EXPRESSION_UPDATEFIELDS._serialized_end = 5982 + _EXPRESSION_ALIAS._serialized_start = 5984 + _EXPRESSION_ALIAS._serialized_end = 6104 + _EXPRESSION_LAMBDAFUNCTION._serialized_start = 6107 + _EXPRESSION_LAMBDAFUNCTION._serialized_end = 6265 + _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_start = 6267 + _EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE._serialized_end = 6329 + _EXPRESSIONCOMMON._serialized_start = 6344 + _EXPRESSIONCOMMON._serialized_end = 6409 + _COMMONINLINEUSERDEFINEDFUNCTION._serialized_start = 6412 + _COMMONINLINEUSERDEFINEDFUNCTION._serialized_end = 6776 + _PYTHONUDF._serialized_start = 6779 + _PYTHONUDF._serialized_end = 6983 + _SCALARSCALAUDF._serialized_start = 6986 + _SCALARSCALAUDF._serialized_end = 7200 + _JAVAUDF._serialized_start = 7203 + _JAVAUDF._serialized_end = 7352 + _TYPEDAGGREGATEEXPRESSION._serialized_start = 7354 + _TYPEDAGGREGATEEXPRESSION._serialized_end = 7453 + _CALLFUNCTION._serialized_start = 7455 + _CALLFUNCTION._serialized_end = 7563 + _NAMEDARGUMENTEXPRESSION._serialized_start = 7565 + _NAMEDARGUMENTEXPRESSION._serialized_end = 7657 + _MERGEACTION._serialized_start = 7660 + _MERGEACTION._serialized_end = 8172 + _MERGEACTION_ASSIGNMENT._serialized_start = 7882 + _MERGEACTION_ASSIGNMENT._serialized_end = 7988 + _MERGEACTION_ACTIONTYPE._serialized_start = 7991 + _MERGEACTION_ACTIONTYPE._serialized_end = 8158 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.pyi b/python/pyspark/sql/connect/proto/expressions_pb2.pyi index 2c80be6c8fb5a..1566eb1b1e9e2 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1183,6 +1183,7 @@ class Expression(google.protobuf.message.Message): CALL_FUNCTION_FIELD_NUMBER: builtins.int NAMED_ARGUMENT_EXPRESSION_FIELD_NUMBER: builtins.int MERGE_ACTION_FIELD_NUMBER: builtins.int + TYPED_AGGREGATE_EXPRESSION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def common(self) -> global___ExpressionCommon: ... @@ -1225,6 +1226,8 @@ class Expression(google.protobuf.message.Message): @property def merge_action(self) -> global___MergeAction: ... @property + def typed_aggregate_expression(self) -> global___TypedAggregateExpression: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -1252,6 +1255,7 @@ class Expression(google.protobuf.message.Message): call_function: global___CallFunction | None = ..., named_argument_expression: global___NamedArgumentExpression | None = ..., merge_action: global___MergeAction | None = ..., + typed_aggregate_expression: global___TypedAggregateExpression | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -1283,6 +1287,8 @@ class Expression(google.protobuf.message.Message): b"named_argument_expression", "sort_order", b"sort_order", + "typed_aggregate_expression", + b"typed_aggregate_expression", "unresolved_attribute", b"unresolved_attribute", "unresolved_extract_value", @@ -1330,6 +1336,8 @@ class Expression(google.protobuf.message.Message): b"named_argument_expression", "sort_order", b"sort_order", + "typed_aggregate_expression", + b"typed_aggregate_expression", "unresolved_attribute", b"unresolved_attribute", "unresolved_extract_value", @@ -1370,6 +1378,7 @@ class Expression(google.protobuf.message.Message): "call_function", "named_argument_expression", "merge_action", + "typed_aggregate_expression", "extension", ] | None @@ -1620,6 +1629,27 @@ class JavaUDF(google.protobuf.message.Message): global___JavaUDF = JavaUDF +class TypedAggregateExpression(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + SCALAR_SCALA_UDF_FIELD_NUMBER: builtins.int + @property + def scalar_scala_udf(self) -> global___ScalarScalaUDF: + """(Required) The aggregate function object packed into bytes.""" + def __init__( + self, + *, + scalar_scala_udf: global___ScalarScalaUDF | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["scalar_scala_udf", b"scalar_scala_udf"] + ) -> builtins.bool: ... + def ClearField( + self, field_name: typing_extensions.Literal["scalar_scala_udf", b"scalar_scala_udf"] + ) -> None: ... + +global___TypedAggregateExpression = TypedAggregateExpression + class CallFunction(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor From 0fa5787d0a6bd17ccd05ff561bc8dfa88af03312 Mon Sep 17 00:00:00 2001 From: Weichen Xu Date: Fri, 12 Jul 2024 22:20:37 +0800 Subject: [PATCH 60/64] [SPARK-48883][ML][R] Replace RDD read / write API invocation with Dataframe read / write API ### What changes were proposed in this pull request? Replace RDD read / write API invocation with Dataframe read / write API ### Why are the changes needed? In databricks runtime, RDD read / write API has some issue for certain storage types that requires the account key, but Dataframe read / write API works. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Unit tests ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47328 from WeichenXu123/ml-df-writer-save-2. Authored-by: Weichen Xu Signed-off-by: Weichen Xu --- .../apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala | 7 +++++-- .../src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala | 7 +++++-- .../org/apache/spark/ml/r/BisectingKMeansWrapper.scala | 7 +++++-- .../apache/spark/ml/r/DecisionTreeClassifierWrapper.scala | 7 +++++-- .../apache/spark/ml/r/DecisionTreeRegressorWrapper.scala | 7 +++++-- .../scala/org/apache/spark/ml/r/FMClassifierWrapper.scala | 7 +++++-- .../scala/org/apache/spark/ml/r/FMRegressorWrapper.scala | 7 +++++-- .../scala/org/apache/spark/ml/r/FPGrowthWrapper.scala | 4 +++- .../org/apache/spark/ml/r/GBTClassifierWrapper.scala | 7 +++++-- .../scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala | 7 +++++-- .../org/apache/spark/ml/r/GaussianMixtureWrapper.scala | 7 +++++-- .../spark/ml/r/GeneralizedLinearRegressionWrapper.scala | 7 +++++-- .../org/apache/spark/ml/r/IsotonicRegressionWrapper.scala | 7 +++++-- .../main/scala/org/apache/spark/ml/r/KMeansWrapper.scala | 7 +++++-- .../src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala | 7 +++++-- .../org/apache/spark/ml/r/LinearRegressionWrapper.scala | 7 +++++-- .../scala/org/apache/spark/ml/r/LinearSVCWrapper.scala | 7 +++++-- .../org/apache/spark/ml/r/LogisticRegressionWrapper.scala | 7 +++++-- .../ml/r/MultilayerPerceptronClassifierWrapper.scala | 4 +++- .../scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala | 7 +++++-- .../src/main/scala/org/apache/spark/ml/r/RWrappers.scala | 3 ++- .../apache/spark/ml/r/RandomForestClassifierWrapper.scala | 8 ++++++-- .../apache/spark/ml/r/RandomForestRegressorWrapper.scala | 8 ++++++-- .../main/scala/org/apache/spark/ml/util/ReadWrite.scala | 6 ++++-- 24 files changed, 114 insertions(+), 45 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala index 7eef3ced422e6..67057b3fcef69 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -129,7 +129,9 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg val rMetadata = ("class" -> instance.getClass.getName) ~ ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -142,7 +144,8 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala index 125cdf7259fef..5fc19450d2198 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/ALSWrapper.scala @@ -94,7 +94,9 @@ private[r] object ALSWrapper extends MLReadable[ALSWrapper] { val rMetadata = ("class" -> instance.getClass.getName) ~ ("ratingCol" -> instance.ratingCol) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.alsModel.save(modelPath) } @@ -107,7 +109,8 @@ private[r] object ALSWrapper extends MLReadable[ALSWrapper] { val rMetadataPath = new Path(path, "rMetadata").toString val modelPath = new Path(path, "model").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val ratingCol = (rMetadata \ "ratingCol").extract[String] val alsModel = ALSModel.load(modelPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala index d4486f1b80a10..c86a788ab3301 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala @@ -120,7 +120,9 @@ private[r] object BisectingKMeansWrapper extends MLReadable[BisectingKMeansWrapp ("size" -> instance.size.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -133,7 +135,8 @@ private[r] object BisectingKMeansWrapper extends MLReadable[BisectingKMeansWrapp val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] val size = (rMetadata \ "size").extract[Array[Long]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala index 992a0c18819fc..51b8f7ce869c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala @@ -131,7 +131,9 @@ private[r] object DecisionTreeClassifierWrapper extends MLReadable[DecisionTreeC ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -144,7 +146,8 @@ private[r] object DecisionTreeClassifierWrapper extends MLReadable[DecisionTreeC val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala index db421b5a1875e..40fb32daf42e5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala @@ -114,7 +114,9 @@ private[r] object DecisionTreeRegressorWrapper extends MLReadable[DecisionTreeRe ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -127,7 +129,8 @@ private[r] object DecisionTreeRegressorWrapper extends MLReadable[DecisionTreeRe val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala index 635af0563da0d..3ab631b9be0b3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala @@ -151,7 +151,9 @@ private[r] object FMClassifierWrapper ("features" -> instance.features.toImmutableArraySeq) ~ ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -164,7 +166,8 @@ private[r] object FMClassifierWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] val labels = (rMetadata \ "labels").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala index b036a1d102d97..e1ec2f47fb035 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala @@ -132,7 +132,9 @@ private[r] object FMRegressorWrapper val rMetadata = ("class" -> instance.getClass.getName) ~ ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -145,7 +147,8 @@ private[r] object FMRegressorWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala index b8151d8d90702..ff227e8b78129 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala @@ -78,7 +78,9 @@ private[r] object FPGrowthWrapper extends MLReadable[FPGrowthWrapper] { "class" -> instance.getClass.getName )) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.fpGrowthModel.save(modelPath) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala index 777191ef5e5c6..5bf021ca3bd4a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala @@ -138,7 +138,9 @@ private[r] object GBTClassifierWrapper extends MLReadable[GBTClassifierWrapper] ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -151,7 +153,8 @@ private[r] object GBTClassifierWrapper extends MLReadable[GBTClassifierWrapper] val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala index 6e5ca47fabae6..efc7ab21a77f2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala @@ -122,7 +122,9 @@ private[r] object GBTRegressorWrapper extends MLReadable[GBTRegressorWrapper] { ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -135,7 +137,8 @@ private[r] object GBTRegressorWrapper extends MLReadable[GBTRegressorWrapper] { val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala index 9a98a8b18b141..14a090690f8a0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GaussianMixtureWrapper.scala @@ -113,7 +113,9 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp ("logLikelihood" -> instance.logLikelihood) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -126,7 +128,8 @@ private[r] object GaussianMixtureWrapper extends MLReadable[GaussianMixtureWrapp val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val dim = (rMetadata \ "dim").extract[Int] val logLikelihood = (rMetadata \ "logLikelihood").extract[Double] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala index 60cf0631f91de..fb3a000713627 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala @@ -170,7 +170,9 @@ private[r] object GeneralizedLinearRegressionWrapper ("rAic" -> instance.rAic) ~ ("rNumIterations" -> instance.rNumIterations) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -184,7 +186,8 @@ private[r] object GeneralizedLinearRegressionWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val rFeatures = (rMetadata \ "rFeatures").extract[Array[String]] val rCoefficients = (rMetadata \ "rCoefficients").extract[Array[Double]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala index d4a3adea460fa..e2df133c1c1e8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala @@ -99,7 +99,9 @@ private[r] object IsotonicRegressionWrapper val rMetadata = ("class" -> instance.getClass.getName) ~ ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -112,7 +114,8 @@ private[r] object IsotonicRegressionWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala index 78c9a15aac597..4073b69e46b4f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala @@ -123,7 +123,9 @@ private[r] object KMeansWrapper extends MLReadable[KMeansWrapper] { ("size" -> instance.size.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } } @@ -136,7 +138,8 @@ private[r] object KMeansWrapper extends MLReadable[KMeansWrapper] { val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] val size = (rMetadata \ "size").extract[Array[Long]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala index 943c38178d6f0..26998edab271a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LDAWrapper.scala @@ -198,7 +198,9 @@ private[r] object LDAWrapper extends MLReadable[LDAWrapper] { ("logPerplexity" -> instance.logPerplexity) ~ ("vocabulary" -> instance.vocabulary.toList) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -211,7 +213,8 @@ private[r] object LDAWrapper extends MLReadable[LDAWrapper] { val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val logLikelihood = (rMetadata \ "logLikelihood").extract[Double] val logPerplexity = (rMetadata \ "logPerplexity").extract[Double] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala index 96b00fab7e344..3c720ed82b9fe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala @@ -127,7 +127,9 @@ private[r] object LinearRegressionWrapper val rMetadata = ("class" -> instance.getClass.getName) ~ ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -140,7 +142,8 @@ private[r] object LinearRegressionWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala index 3645af3e53115..5f77be9e5641b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala @@ -137,7 +137,9 @@ private[r] object LinearSVCWrapper ("features" -> instance.features.toImmutableArraySeq) ~ ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -150,7 +152,8 @@ private[r] object LinearSVCWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] val labels = (rMetadata \ "labels").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala index cac3d0609b209..429ca7ba04fce 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala @@ -192,7 +192,9 @@ private[r] object LogisticRegressionWrapper ("features" -> instance.features.toImmutableArraySeq) ~ ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -205,7 +207,8 @@ private[r] object LogisticRegressionWrapper val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val features = (rMetadata \ "features").extract[Array[String]] val labels = (rMetadata \ "labels").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala index 96c588acc1406..b8e466099af42 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/MultilayerPerceptronClassifierWrapper.scala @@ -142,7 +142,9 @@ private[r] object MultilayerPerceptronClassifierWrapper val rMetadata = "class" -> instance.getClass.getName val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala index d5e8e0ef4890a..30c5ddbe80d7b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -102,7 +102,9 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { ("labels" -> instance.labels.toImmutableArraySeq) ~ ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) instance.pipeline.save(pipelinePath) } @@ -115,7 +117,8 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { val rMetadataPath = new Path(path, "rMetadata").toString val pipelinePath = new Path(path, "pipeline").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val labels = (rMetadata \ "labels").extract[Array[String]] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala index 551c7514ee85f..3a7539e0937fe 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala @@ -33,7 +33,8 @@ private[r] object RWrappers extends MLReader[Object] { override def load(path: String): Object = { implicit val format = DefaultFormats val rMetadataPath = new Path(path, "rMetadata").toString - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val className = (rMetadata \ "class").extract[String] className match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala index 7c4175a6c5914..b3f040e2e95f6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala @@ -141,7 +141,10 @@ private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestC ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) + instance.pipeline.save(pipelinePath) } } @@ -154,7 +157,8 @@ private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestC val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala index 911571cac77de..9c583e2c53bd2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala @@ -124,7 +124,10 @@ private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRe ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) - sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) + sparkSession.createDataFrame( + Seq(Tuple1(rMetadataJson)) + ).repartition(1).write.text(rMetadataPath) + instance.pipeline.save(pipelinePath) } } @@ -137,7 +140,8 @@ private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRe val pipelinePath = new Path(path, "pipeline").toString val pipeline = PipelineModel.load(pipelinePath) - val rMetadataStr = sc.textFile(rMetadataPath, 1).first() + val rMetadataStr = sparkSession.read.text(rMetadataPath) + .first().getString(0) val rMetadata = parse(rMetadataStr) val formula = (rMetadata \ "formula").extract[String] val features = (rMetadata \ "features").extract[Array[String]] diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala index 9b26d0a911aca..e4a274ee1483a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/ReadWrite.scala @@ -411,7 +411,8 @@ private[ml] object DefaultParamsWriter { paramMap: Option[JValue] = None): Unit = { val metadataPath = new Path(path, "metadata").toString val metadataJson = getMetadataToSave(instance, sc, extraMetadata, paramMap) - sc.parallelize(Seq(metadataJson), 1).saveAsTextFile(metadataPath) + val spark = SparkSession.getActiveSession.get + spark.createDataFrame(Seq(Tuple1(metadataJson))).repartition(1).write.text(metadataPath) } /** @@ -585,7 +586,8 @@ private[ml] object DefaultParamsReader { */ def loadMetadata(path: String, sc: SparkContext, expectedClassName: String = ""): Metadata = { val metadataPath = new Path(path, "metadata").toString - val metadataStr = sc.textFile(metadataPath, 1).first() + val spark = SparkSession.getActiveSession.get + val metadataStr = spark.read.text(metadataPath).first().getString(0) parseMetadata(metadataStr, expectedClassName) } From 8e4bbdff80a1c069ccce71060751987e9e6c0b6b Mon Sep 17 00:00:00 2001 From: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Date: Fri, 12 Jul 2024 22:30:18 +0800 Subject: [PATCH 61/64] [SPARK-48440][SQL] Fix StringTranslate behaviour for non-UTF8_BINARY collations MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? String searching in UTF8_LCASE now works on character-level, rather than on byte-level. For example: `translate("İ", "i")` now returns `"İ"`, because there exists no **single character** in `"İ"` such that lowercased version of that character equals to `"i"`. Note, however, that there _is_ a byte subsequence of `"İ"` such that lowercased version of that UTF-8 byte sequence equals to `"i"` (so the new behaviour is different than the old behaviour). Also, translation for ICU collations works by repeatedly translating the longest possible substring that matches a key in the dictionary (under the specified collation), starting from the left side of the input string, until the entire string is translated. ### Why are the changes needed? Fix functions that give unusable results due to one-to-many case mapping when performing string search under UTF8_BINARY_LCASE (see example above). ### Does this PR introduce _any_ user-facing change? Yes, behaviour of `translate` expression is changed for edge cases with one-to-many case mapping. ### How was this patch tested? New unit tests in `CollationStringExpressionsSuite`. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46761 from uros-db/alter-translate. Authored-by: Uros Bojanic <157381213+uros-db@users.noreply.github.com> Signed-off-by: Wenchen Fan --- .../util/CollationAwareUTF8String.java | 218 +++++++++++++++--- .../sql/catalyst/util/CollationSupport.java | 25 +- .../unsafe/types/CollationSupportSuite.java | 192 ++++++++++++++- .../expressions/stringExpressions.scala | 30 ++- .../sql/CollationStringExpressionsSuite.scala | 51 +--- 5 files changed, 402 insertions(+), 114 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java index 23adc772b7f34..af152c87f88ce 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationAwareUTF8String.java @@ -18,6 +18,8 @@ import com.ibm.icu.lang.UCharacter; import com.ibm.icu.text.BreakIterator; +import com.ibm.icu.text.Collator; +import com.ibm.icu.text.RuleBasedCollator; import com.ibm.icu.text.StringSearch; import com.ibm.icu.util.ULocale; @@ -26,8 +28,12 @@ import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET; import static org.apache.spark.unsafe.Platform.copyMemory; +import static org.apache.spark.unsafe.types.UTF8String.CodePointIteratorType; +import java.text.CharacterIterator; +import java.text.StringCharacterIterator; import java.util.HashMap; +import java.util.Iterator; import java.util.Map; /** @@ -424,19 +430,50 @@ private static UTF8String toLowerCaseSlow(final UTF8String target, final int col * @param codePoint The code point to convert to lowercase. * @param sb The StringBuilder to append the lowercase character to. */ - private static void lowercaseCodePoint(final int codePoint, final StringBuilder sb) { - if (codePoint == 0x0130) { + private static void appendLowercaseCodePoint(final int codePoint, final StringBuilder sb) { + int lowercaseCodePoint = getLowercaseCodePoint(codePoint); + if (lowercaseCodePoint == CODE_POINT_COMBINED_LOWERCASE_I_DOT) { // Latin capital letter I with dot above is mapped to 2 lowercase characters. sb.appendCodePoint(0x0069); sb.appendCodePoint(0x0307); + } else { + // All other characters should follow context-unaware ICU single-code point case mapping. + sb.appendCodePoint(lowercaseCodePoint); + } + } + + /** + * `CODE_POINT_COMBINED_LOWERCASE_I_DOT` is an internal representation of the combined lowercase + * code point for ASCII lowercase letter i with an additional combining dot character (U+0307). + * This integer value is not a valid code point itself, but rather an artificial code point + * marker used to represent the two lowercase characters that are the result of converting the + * uppercase Turkish dotted letter I with a combining dot character (U+0130) to lowercase. + */ + private static final int CODE_POINT_LOWERCASE_I = 0x69; + private static final int CODE_POINT_COMBINING_DOT = 0x307; + private static final int CODE_POINT_COMBINED_LOWERCASE_I_DOT = + CODE_POINT_LOWERCASE_I << 16 | CODE_POINT_COMBINING_DOT; + + /** + * Returns the lowercase version of the provided code point, with special handling for + * one-to-many case mappings (i.e. characters that map to multiple characters in lowercase) and + * context-insensitive case mappings (i.e. characters that map to different characters based on + * the position in the string relative to other characters in lowercase). + */ + private static int getLowercaseCodePoint(final int codePoint) { + if (codePoint == 0x0130) { + // Latin capital letter I with dot above is mapped to 2 lowercase characters. + return CODE_POINT_COMBINED_LOWERCASE_I_DOT; } else if (codePoint == 0x03C2) { - // Greek final and non-final capital letter sigma should be mapped the same. - sb.appendCodePoint(0x03C3); + // Greek final and non-final letter sigma should be mapped the same. This is achieved by + // mapping Greek small final sigma (U+03C2) to Greek small non-final sigma (U+03C3). Capital + // letter sigma (U+03A3) is mapped to small non-final sigma (U+03C3) in the `else` branch. + return 0x03C3; } else { // All other characters should follow context-unaware ICU single-code point case mapping. - sb.appendCodePoint(UCharacter.toLowerCase(codePoint)); + return UCharacter.toLowerCase(codePoint); } } @@ -444,7 +481,7 @@ else if (codePoint == 0x03C2) { * Converts an entire string to lowercase using ICU rules, code point by code point, with * special handling for one-to-many case mappings (i.e. characters that map to multiple * characters in lowercase). Also, this method omits information about context-sensitive case - * mappings using special handling in the `lowercaseCodePoint` method. + * mappings using special handling in the `appendLowercaseCodePoint` method. * * @param target The target string to convert to lowercase. * @return The string converted to lowercase in a context-unaware manner. @@ -455,10 +492,11 @@ public static UTF8String lowerCaseCodePoints(final UTF8String target) { } private static UTF8String lowerCaseCodePointsSlow(final UTF8String target) { - String targetString = target.toValidString(); + Iterator targetIter = target.codePointIterator( + CodePointIteratorType.CODE_POINT_ITERATOR_MAKE_VALID); StringBuilder sb = new StringBuilder(); - for (int i = 0; i < targetString.length(); ++i) { - lowercaseCodePoint(targetString.codePointAt(i), sb); + while (targetIter.hasNext()) { + appendLowercaseCodePoint(targetIter.next(), sb); } return UTF8String.fromString(sb.toString()); } @@ -655,38 +693,152 @@ public static UTF8String lowercaseSubStringIndex(final UTF8String string, } } - public static Map getCollationAwareDict(UTF8String string, - Map dict, int collationId) { - // TODO(SPARK-48715): All UTF8String -> String conversions should use `makeValid` - String srcStr = string.toString(); + /** + * Converts the original translation dictionary (`dict`) to a dictionary with lowercased keys. + * This method is used to create a dictionary that can be used for the UTF8_LCASE collation. + * Note that `StringTranslate.buildDict` will ensure that all strings are validated properly. + * + * The method returns a map with lowercased code points as keys, while the values remain + * unchanged. Note that `dict` is constructed on a character by character basis, and the + * original keys are stored as strings. Keys in the resulting lowercase dictionary are stored + * as integers, which correspond only to single characters from the original `dict`. Also, + * there is special handling for the Turkish dotted uppercase letter I (U+0130). + */ + private static Map getLowercaseDict(final Map dict) { + // Replace all the keys in the dict with lowercased code points. + Map lowercaseDict = new HashMap<>(); + for (Map.Entry entry : dict.entrySet()) { + int codePoint = entry.getKey().codePointAt(0); + lowercaseDict.putIfAbsent(getLowercaseCodePoint(codePoint), entry.getValue()); + } + return lowercaseDict; + } + + /** + * Translates the `input` string using the translation map `dict`, for UTF8_LCASE collation. + * String translation is performed by iterating over the input string, from left to right, and + * repeatedly translating the longest possible substring that matches a key in the dictionary. + * For UTF8_LCASE, the method uses the lowercased substring to perform the lookup in the + * lowercased version of the translation map. + * + * @param input the string to be translated + * @param dict the lowercase translation dictionary + * @return the translated string + */ + public static UTF8String lowercaseTranslate(final UTF8String input, + final Map dict) { + // Iterator for the input string. + Iterator inputIter = input.codePointIterator( + CodePointIteratorType.CODE_POINT_ITERATOR_MAKE_VALID); + // Lowercased translation dictionary. + Map lowercaseDict = getLowercaseDict(dict); + // StringBuilder to store the translated string. + StringBuilder sb = new StringBuilder(); - Map collationAwareDict = new HashMap<>(); - for (String key : dict.keySet()) { - StringSearch stringSearch = - CollationFactory.getStringSearch(string, UTF8String.fromString(key), collationId); + // We use buffered code point iteration to handle one-to-many case mappings. We need to handle + // at most two code points at a time (for `CODE_POINT_COMBINED_LOWERCASE_I_DOT`), a buffer of + // size 1 enables us to match two codepoints in the input string with a single codepoint in + // the lowercase translation dictionary. + int codePointBuffer = -1, codePoint; + while (inputIter.hasNext()) { + if (codePointBuffer != -1) { + codePoint = codePointBuffer; + codePointBuffer = -1; + } else { + codePoint = inputIter.next(); + } + // Special handling for letter i (U+0069) followed by a combining dot (U+0307). By ensuring + // that `CODE_POINT_LOWERCASE_I` is buffered, we guarantee finding a max-length match. + if (lowercaseDict.containsKey(CODE_POINT_COMBINED_LOWERCASE_I_DOT) && + codePoint == CODE_POINT_LOWERCASE_I && inputIter.hasNext()) { + int nextCodePoint = inputIter.next(); + if (nextCodePoint == CODE_POINT_COMBINING_DOT) { + codePoint = CODE_POINT_COMBINED_LOWERCASE_I_DOT; + } else { + codePointBuffer = nextCodePoint; + } + } + // Translate the code point using the lowercased dictionary. + String translated = lowercaseDict.get(getLowercaseCodePoint(codePoint)); + if (translated == null) { + // Append the original code point if no translation is found. + sb.appendCodePoint(codePoint); + } else if (!"\0".equals(translated)) { + // Append the translated code point if the translation is not the null character. + sb.append(translated); + } + // Skip the code point if it maps to the null character. + } + // Append the last code point if it was buffered. + if (codePointBuffer != -1) sb.appendCodePoint(codePointBuffer); - int pos = 0; - while ((pos = stringSearch.next()) != StringSearch.DONE) { - int codePoint = srcStr.codePointAt(pos); - int charCount = Character.charCount(codePoint); - String newKey = srcStr.substring(pos, pos + charCount); + // Return the translated string. + return UTF8String.fromString(sb.toString()); + } - boolean exists = false; - for (String existingKey : collationAwareDict.keySet()) { - if (stringSearch.getCollator().compare(existingKey, newKey) == 0) { - collationAwareDict.put(newKey, collationAwareDict.get(existingKey)); - exists = true; - break; + /** + * Translates the `input` string using the translation map `dict`, for all ICU collations. + * String translation is performed by iterating over the input string, from left to right, and + * repeatedly translating the longest possible substring that matches a key in the dictionary. + * For ICU collations, the method uses the ICU `StringSearch` class to perform the lookup in + * the translation map, while respecting the rules of the specified ICU collation. + * + * @param input the string to be translated + * @param dict the collation aware translation dictionary + * @param collationId the collation ID to use for string translation + * @return the translated string + */ + public static UTF8String translate(final UTF8String input, + final Map dict, final int collationId) { + // Replace invalid UTF-8 sequences with the Unicode replacement character U+FFFD. + String inputString = input.toValidString(); + // Create a character iterator for the validated input string. This will be used for searching + // inside the string using ICU `StringSearch` class. We only need to do it once before the + // main loop of the translate algorithm. + CharacterIterator target = new StringCharacterIterator(inputString); + Collator collator = CollationFactory.fetchCollation(collationId).collator; + StringBuilder sb = new StringBuilder(); + // Index for the current character in the (validated) input string. This is the character we + // want to determine if we need to replace or not. + int charIndex = 0; + while (charIndex < inputString.length()) { + // We search the replacement dictionary to find a match. If there are more than one matches + // (which is possible for collated strings), we want to choose the match of largest length. + int longestMatchLen = 0; + String longestMatch = ""; + for (String key : dict.keySet()) { + StringSearch stringSearch = new StringSearch(key, target, (RuleBasedCollator) collator); + // Point `stringSearch` to start at the current character. + stringSearch.setIndex(charIndex); + int matchIndex = stringSearch.next(); + if (matchIndex == charIndex) { + // We have found a match (that is the current position matches with one of the characters + // in the dictionary). However, there might be other matches of larger length, so we need + // to continue searching against the characters in the dictionary and keep track of the + // match of largest length. + int matchLen = stringSearch.getMatchLength(); + if (matchLen > longestMatchLen) { + longestMatchLen = matchLen; + longestMatch = key; } } - - if (!exists) { - collationAwareDict.put(newKey, dict.get(key)); + } + if (longestMatchLen == 0) { + // No match was found, so output the current character. + sb.append(inputString.charAt(charIndex)); + // Move on to the next character in the input string. + ++charIndex; + } else { + // We have found at least one match. Append the match of longest match length to the output. + if (!"\0".equals(dict.get(longestMatch))) { + sb.append(dict.get(longestMatch)); } + // Skip as many characters as the longest match. + charIndex += longestMatchLen; } } - - return collationAwareDict; + // Return the translated string. + return UTF8String.fromString(sb.toString()); } public static UTF8String lowercaseTrim( diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java index 450a3eea1a3a0..f9ccd22f3f5c6 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/util/CollationSupport.java @@ -212,7 +212,7 @@ public static UTF8String exec(final UTF8String v, final int collationId, boolean return useICU ? execBinaryICU(v) : execBinary(v); } else if (collation.supportsLowercaseEquality) { return execLowercase(v); - } else { + } else { return execICU(v, collationId); } } @@ -224,7 +224,7 @@ public static String genCode(final String v, final int collationId, boolean useI return String.format(expr + "%s(%s)", funcName, v); } else if (collation.supportsLowercaseEquality) { return String.format(expr + "Lowercase(%s)", v); - } else { + } else { return String.format(expr + "ICU(%s, %d)", v, collationId); } } @@ -261,7 +261,7 @@ public static String genCode(final String v, final int collationId, boolean useI return String.format(expr + "%s(%s)", funcName, v); } else if (collation.supportsLowercaseEquality) { return String.format(expr + "Lowercase(%s)", v); - } else { + } else { return String.format(expr + "ICU(%s, %d)", v, collationId); } } @@ -522,26 +522,11 @@ public static UTF8String execBinary(final UTF8String source, Map return source.translate(dict); } public static UTF8String execLowercase(final UTF8String source, Map dict) { - String srcStr = source.toString(); - StringBuilder sb = new StringBuilder(); - int charCount = 0; - for (int k = 0; k < srcStr.length(); k += charCount) { - int codePoint = srcStr.codePointAt(k); - charCount = Character.charCount(codePoint); - String subStr = srcStr.substring(k, k + charCount); - String translated = dict.get(subStr.toLowerCase()); - if (null == translated) { - sb.append(subStr); - } else if (!"\0".equals(translated)) { - sb.append(translated); - } - } - return UTF8String.fromString(sb.toString()); + return CollationAwareUTF8String.lowercaseTranslate(source, dict); } public static UTF8String execICU(final UTF8String source, Map dict, final int collationId) { - return source.translate(CollationAwareUTF8String.getCollationAwareDict( - source, dict, collationId)); + return CollationAwareUTF8String.translate(source, dict, collationId); } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java index 9438484344d62..ce0cef3fef307 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CollationSupportSuite.java @@ -22,6 +22,9 @@ import org.apache.spark.sql.catalyst.util.CollationSupport; import org.junit.jupiter.api.Test; +import java.util.HashMap; +import java.util.Map; + import static org.junit.jupiter.api.Assertions.*; // checkstyle.off: AvoidEscapedUnicodeCharacters @@ -1378,19 +1381,186 @@ public void testStringTrim() throws SparkException { assertStringTrimRight("UTF8_LCASE", "Ëaaaẞ", "Ëẞ", "Ëaaa"); } - // TODO: Test more collation-aware string expressions. - - /** - * Collation-aware regexp expressions. - */ - - // TODO: Test more collation-aware regexp expressions. + private void assertStringTranslate( + String inputString, + String matchingString, + String replaceString, + String collationName, + String expectedResultString) throws SparkException { + int collationId = CollationFactory.collationNameToId(collationName); + Map dict = buildDict(matchingString, replaceString); + UTF8String source = UTF8String.fromString(inputString); + UTF8String result = CollationSupport.StringTranslate.exec(source, dict, collationId); + assertEquals(expectedResultString, result.toString()); + } - /** - * Other collation-aware expressions. - */ + @Test + public void testStringTranslate() throws SparkException { + // Basic tests - UTF8_BINARY. + assertStringTranslate("Translate", "Rnlt", "12", "UTF8_BINARY", "Tra2sae"); + assertStringTranslate("Translate", "Rn", "1234", "UTF8_BINARY", "Tra2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_BINARY", "Tra2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_BINARY", "TRaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_BINARY", "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_BINARY", "TXaxsXaxex"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_BINARY", "test大千世AX大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_BINARY", "大千世界test大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_BINARY", "Oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_BINARY", "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_BINARY", "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY", "Tr4234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_BINARY", "Tra2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", "UTF8_BINARY", "123f"); + // Basic tests - UTF8_LCASE. + assertStringTranslate("Translate", "Rnlt", "12", "UTF8_LCASE", "1a2sae"); + assertStringTranslate("Translate", "Rn", "1234", "UTF8_LCASE", "T1a2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", "UTF8_LCASE", "xXaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UTF8_LCASE", "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UTF8_LCASE", "xXaxsXaxeX"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UTF8_LCASE", "test大千世AB大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UTF8_LCASE", "大千世界abca大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UTF8_LCASE", "oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UTF8_LCASE", "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UTF8_LCASE", "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UTF8_LCASE", "14234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", "UTF8_LCASE", "41a2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", "UTF8_LCASE", "123f"); + // Basic tests - UNICODE. + assertStringTranslate("Translate", "Rnlt", "12", "UNICODE", "Tra2sae"); + assertStringTranslate("Translate", "Rn", "1234", "UNICODE", "Tra2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE", "Tra2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE", "TRaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE", "TxaxsXaxeX"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE", "TXaxsXaxex"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE", "test大千世AX大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE", "大千世界test大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE", "Oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE", "大千世界大千世界oesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE", "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE", "Tr4234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE", "Tra2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", "UNICODE", "123f"); + // Basic tests - UNICODE_CI. + assertStringTranslate("Translate", "Rnlt", "12", "UNICODE_CI", "1a2sae"); + assertStringTranslate("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate"); + assertStringTranslate("Translate", "Rnlt", "1234", "UNICODE_CI", "41a2s3a4e"); + assertStringTranslate("TRanslate", "rnlt", "XxXx", "UNICODE_CI", "xXaxsXaxe"); + assertStringTranslate("TRanslater", "Rrnlt", "xXxXx", "UNICODE_CI", "xxaxsXaxex"); + assertStringTranslate("TRanslater", "Rrnlt", "XxxXx", "UNICODE_CI", "xXaxsXaxeX"); + assertStringTranslate("test大千世界X大千世界", "界x", "AB", "UNICODE_CI", "test大千世AB大千世A"); + assertStringTranslate("大千世界test大千世界", "TEST", "abcd", "UNICODE_CI", "大千世界abca大千世界"); + assertStringTranslate("Test大千世界大千世界", "tT", "oO", "UNICODE_CI", "oeso大千世界大千世界"); + assertStringTranslate("大千世界大千世界tesT", "Tt", "Oo", "UNICODE_CI", "大千世界大千世界OesO"); + assertStringTranslate("大千世界大千世界tesT", "大千", "世世", "UNICODE_CI", "世世世界世世世界tesT"); + assertStringTranslate("Translate", "Rnlasdfjhgadt", "1234", "UNICODE_CI", "14234e"); + assertStringTranslate("Translate", "Rnlt", "123495834634", "UNICODE_CI", "41a2s3a4e"); + assertStringTranslate("abcdef", "abcde", "123", "UNICODE_CI", "123f"); + + // One-to-many case mapping - UTF8_BINARY. + assertStringTranslate("İ", "i\u0307", "xy", "UTF8_BINARY", "İ"); + assertStringTranslate("i\u0307", "İ", "xy", "UTF8_BINARY", "i\u0307"); + assertStringTranslate("i\u030A", "İ", "x", "UTF8_BINARY", "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", "UTF8_BINARY", "y\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_BINARY", "123"); + assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_BINARY", "1i\u0307"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_BINARY", "İ23"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_BINARY", "12bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_BINARY", "a2bcå"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UTF8_BINARY", "3\u030Aβφδ1\u0307"); + // One-to-many case mapping - UTF8_LCASE. + assertStringTranslate("İ", "i\u0307", "xy", "UTF8_LCASE", "İ"); + assertStringTranslate("i\u0307", "İ", "xy", "UTF8_LCASE", "x"); + assertStringTranslate("i\u030A", "İ", "x", "UTF8_LCASE", "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", "UTF8_LCASE", "y\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", "UTF8_LCASE", "11"); + assertStringTranslate("İi\u0307", "İyz", "123", "UTF8_LCASE", "11"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", "UTF8_LCASE", "İ23"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UTF8_LCASE", "12bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UTF8_LCASE", "12bc3"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UTF8_LCASE", "3\u030Aβφδ2"); + // One-to-many case mapping - UNICODE. + assertStringTranslate("İ", "i\u0307", "xy", "UNICODE", "İ"); + assertStringTranslate("i\u0307", "İ", "xy", "UNICODE", "i\u0307"); + assertStringTranslate("i\u030A", "İ", "x", "UNICODE", "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", "UNICODE", "i\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE", "1i\u0307"); + assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE", "1i\u0307"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE", "İi\u0307"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE", "3bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE", "a\u030Abcå"); + assertStringTranslate("a\u030AβφδI\u0307", "Iİaå", "1234", "UNICODE", "4βφδ2"); + // One-to-many case mapping - UNICODE_CI. + assertStringTranslate("İ", "i\u0307", "xy", "UNICODE_CI", "İ"); + assertStringTranslate("i\u0307", "İ", "xy", "UNICODE_CI", "x"); + assertStringTranslate("i\u030A", "İ", "x", "UNICODE_CI", "i\u030A"); + assertStringTranslate("i\u030A", "İi", "xy", "UNICODE_CI", "i\u030A"); + assertStringTranslate("İi\u0307", "İi\u0307", "123", "UNICODE_CI", "11"); + assertStringTranslate("İi\u0307", "İyz", "123", "UNICODE_CI", "11"); + assertStringTranslate("İi\u0307", "xi\u0307", "123", "UNICODE_CI", "İi\u0307"); + assertStringTranslate("a\u030Abcå", "a\u030Aå", "123", "UNICODE_CI", "3bc3"); + assertStringTranslate("a\u030Abcå", "A\u030AÅ", "123", "UNICODE_CI", "3bc3"); + assertStringTranslate("A\u030Aβφδi\u0307", "Iİaå", "1234", "UNICODE_CI", "4βφδ2"); + + // Greek sigmas - UTF8_BINARY. + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_BINARY", "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_BINARY", "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_BINARY", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_BINARY", "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_BINARY", "σιστιματικος"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_BINARY", "σιστιματικοσ"); + // Greek sigmas - UTF8_LCASE. + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UTF8_LCASE", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UTF8_LCASE", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("συστηματικος", "Συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", "UTF8_LCASE", "σιστιματικοσ"); + // Greek sigmas - UNICODE. + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE", "σΥσΤΗΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE", "ςΥςΤΗΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE", "ΣΥΣΤΗΜΑΤΙΚΟΣ"); + assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE", "σιστιματικος"); + assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE", "σιστιματικος"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE", "σιστιματικοσ"); + // Greek sigmas - UNICODE_CI. + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "σιι", "UNICODE_CI", "σισΤιΜΑΤΙΚΟσ"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "Συη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("ΣΥΣΤΗΜΑΤΙΚΟΣ", "ςυη", "ςιι", "UNICODE_CI", "ςιςΤιΜΑΤΙΚΟς"); + assertStringTranslate("συστηματικος", "Συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "συη", "σιι", "UNICODE_CI", "σιστιματικοσ"); + assertStringTranslate("συστηματικος", "ςυη", "σιι", "UNICODE_CI", "σιστιματικοσ"); + } - // TODO: Test other collation-aware expressions. + private Map buildDict(String matching, String replace) { + Map dict = new HashMap<>(); + int i = 0, j = 0; + while (i < matching.length()) { + String rep = "\u0000"; + if (j < replace.length()) { + int repCharCount = Character.charCount(replace.codePointAt(j)); + rep = replace.substring(j, j + repCharCount); + j += repCharCount; + } + int matchCharCount = Character.charCount(matching.codePointAt(i)); + String matchStr = matching.substring(i, i + matchCharCount); + dict.putIfAbsent(matchStr, rep); + i += matchCharCount; + } + return dict; + } } // checkstyle.on: AvoidEscapedUnicodeCharacters diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala index b188b9c2630fa..1302ca80e51a3 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringExpressions.scala @@ -1050,15 +1050,35 @@ case class Overlay(input: Expression, replace: Expression, pos: Expression, len: object StringTranslate { - def buildDict(matchingString: UTF8String, replaceString: UTF8String, collationId: Int) + /** + * Build a translation dictionary from UTF8Strings. First, this method converts the input strings + * to valid Java Strings. However, we avoid any behavior changes for the UTF8_BINARY collation, + * but ensure that all other collations use `UTF8String.toValidString` to achieve this step. + */ + def buildDict(matchingString: UTF8String, replaceString: UTF8String, collationId: Integer) : JMap[String, String] = { - val matching = if (CollationFactory.fetchCollation(collationId).supportsLowercaseEquality) { - matchingString.toString().toLowerCase() + val isCollationAware = collationId == CollationFactory.UTF8_BINARY_COLLATION_ID + val matching: String = if (isCollationAware) { + matchingString.toString + } else { + matchingString.toValidString + } + val replace: String = if (isCollationAware) { + replaceString.toString } else { - matchingString.toString() + replaceString.toValidString } + buildDict(matching, replace) + } - val replace = replaceString.toString() + /** + * Build a translation dictionary from Strings. This method assumes that the input strings are + * already valid. The result dictionary maps each character in `matching` to the corresponding + * character in `replace`. If `replace` is shorter than `matching`, the extra characters in + * `matching` will be mapped to null terminator, which causes characters to get deleted during + * translation. If `replace` is longer than `matching`, the extra characters will be ignored. + */ + private def buildDict(matching: String, replace: String): JMap[String, String] = { val dict = new HashMap[String, String]() var i = 0 var j = 0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala index 78aee5b80e549..5f722b2f01fb5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CollationStringExpressionsSuite.scala @@ -252,55 +252,16 @@ class CollationStringExpressionsSuite } assert(collationMismatch.getErrorClass === "COLLATION_MISMATCH.EXPLICIT") } - test("TRANSLATE check result on explicitly collated string") { + + test("Support StringTranslate string expression with collation") { // Supported collations case class TranslateTestCase[R](input: String, matchExpression: String, - replaceExpression: String, collation: String, result: R) + replaceExpression: String, collation: String, result: R) val testCases = Seq( + TranslateTestCase("Translate", "Rnlt", "12", "UTF8_BINARY", "Tra2sae"), TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"), - TranslateTestCase("Translate", "Rnlt", "1234", "UTF8_LCASE", "41a2s3a4e"), - TranslateTestCase("TRanslate", "rnlt", "XxXx", "UTF8_LCASE", "xXaxsXaxe"), - TranslateTestCase("TRanslater", "Rrnlt", "xXxXx", "UTF8_LCASE", "xxaxsXaxex"), - TranslateTestCase("TRanslater", "Rrnlt", "XxxXx", "UTF8_LCASE", "xXaxsXaxeX"), - // scalastyle:off - TranslateTestCase("test大千世界X大千世界", "界x", "AB", "UTF8_LCASE", "test大千世AB大千世A"), - TranslateTestCase("大千世界test大千世界", "TEST", "abcd", "UTF8_LCASE", "大千世界abca大千世界"), - TranslateTestCase("Test大千世界大千世界", "tT", "oO", "UTF8_LCASE", "oeso大千世界大千世界"), - TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UTF8_LCASE", "大千世界大千世界OesO"), - TranslateTestCase("大千世界大千世界tesT", "大千", "世世", "UTF8_LCASE", "世世世界世世世界tesT"), - // scalastyle:on - TranslateTestCase("Translate", "Rnlt", "1234", "UNICODE", "Tra2s3a4e"), - TranslateTestCase("TRanslate", "rnlt", "XxXx", "UNICODE", "TRaxsXaxe"), - TranslateTestCase("TRanslater", "Rrnlt", "xXxXx", "UNICODE", "TxaxsXaxeX"), - TranslateTestCase("TRanslater", "Rrnlt", "XxxXx", "UNICODE", "TXaxsXaxex"), - // scalastyle:off - TranslateTestCase("test大千世界X大千世界", "界x", "AB", "UNICODE", "test大千世AX大千世A"), - TranslateTestCase("Test大千世界大千世界", "tT", "oO", "UNICODE", "Oeso大千世界大千世界"), - TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UNICODE", "大千世界大千世界oesO"), - // scalastyle:on - TranslateTestCase("Translate", "Rnlt", "1234", "UNICODE_CI", "41a2s3a4e"), - TranslateTestCase("TRanslate", "rnlt", "XxXx", "UNICODE_CI", "xXaxsXaxe"), - TranslateTestCase("TRanslater", "Rrnlt", "xXxXx", "UNICODE_CI", "xxaxsXaxex"), - TranslateTestCase("TRanslater", "Rrnlt", "XxxXx", "UNICODE_CI", "xXaxsXaxeX"), - // scalastyle:off - TranslateTestCase("test大千世界X大千世界", "界x", "AB", "UNICODE_CI", "test大千世AB大千世A"), - TranslateTestCase("大千世界test大千世界", "TEST", "abcd", "UNICODE_CI", "大千世界abca大千世界"), - TranslateTestCase("Test大千世界大千世界", "tT", "oO", "UNICODE_CI", "oeso大千世界大千世界"), - TranslateTestCase("大千世界大千世界tesT", "Tt", "Oo", "UNICODE_CI", "大千世界大千世界OesO"), - TranslateTestCase("大千世界大千世界tesT", "大千", "世世", "UNICODE_CI", "世世世界世世世界tesT"), - // scalastyle:on - TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UTF8_LCASE", "14234e"), - TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UNICODE_CI", "14234e"), - TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UNICODE", "Tr4234e"), - TranslateTestCase("Translate", "Rnlasdfjhgadt", "1234", "UTF8_BINARY", "Tr4234e"), - TranslateTestCase("Translate", "Rnlt", "123495834634", "UTF8_LCASE", "41a2s3a4e"), - TranslateTestCase("Translate", "Rnlt", "123495834634", "UNICODE", "Tra2s3a4e"), - TranslateTestCase("Translate", "Rnlt", "123495834634", "UNICODE_CI", "41a2s3a4e"), - TranslateTestCase("Translate", "Rnlt", "123495834634", "UTF8_BINARY", "Tra2s3a4e"), - TranslateTestCase("abcdef", "abcde", "123", "UTF8_BINARY", "123f"), - TranslateTestCase("abcdef", "abcde", "123", "UTF8_LCASE", "123f"), - TranslateTestCase("abcdef", "abcde", "123", "UNICODE", "123f"), - TranslateTestCase("abcdef", "abcde", "123", "UNICODE_CI", "123f") + TranslateTestCase("Translate", "Rn", "\u0000\u0000", "UNICODE", "Traslate"), + TranslateTestCase("Translate", "Rn", "1234", "UNICODE_CI", "T1a2slate") ) testCases.foreach(t => { From 115c6e42586fcf4943430102f803160cd6e6e2d1 Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Fri, 12 Jul 2024 10:57:16 -0700 Subject: [PATCH 62/64] [SPARK-47911][SQL][FOLLOWUP] Rename UTF8 to UTF-8 in spark.sql.binaryOutputStyle ### What changes were proposed in this pull request? Make a follow-up for SPARK-47911 to rename UTF8 to UTF-8 of `spark.sql.binaryOutputStyle`, so that we could have a consistent name with `org.apache.spark.sql.catalyst.util.CharsetProvider.VALID_CHARSETS` and `java.nio.charset.StandardCharsets.UTF_8` ### Why are the changes needed? reduce cognitive cost for users ### Does this PR introduce _any_ user-facing change? no, unreleased feature ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #47322 from yaooqinn/SPARK-47911-FF. Authored-by: Kent Yao Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- .../scala/org/apache/spark/sql/execution/HiveResult.scala | 2 +- sql/core/src/test/resources/sql-tests/inputs/binary.sql | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) 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 65beb21d59d92..3103ced28214d 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 @@ -1522,12 +1522,12 @@ object SQLConf { * Output as UTF-8 string. * [83, 112, 97, 114, 107] -> "Spark" */ - UTF8, + UTF8: Value = Value("UTF-8") /** * Output as comma separated byte array string. * [83, 112, 97, 114, 107] -> [83, 112, 97, 114, 107] */ - BASIC, + val BASIC, /** * Output as base64 encoded string. * [83, 112, 97, 114, 107] -> U3Bhcmsg @@ -1546,7 +1546,7 @@ object SQLConf { } val BINARY_OUTPUT_STYLE = buildConf("spark.sql.binaryOutputStyle") - .doc("The output style used display binary data. Valid values are 'UTF8', " + + .doc("The output style used display binary data. Valid values are 'UTF-8', " + "'BASIC', 'BASE64', 'HEX', and 'HEX_DISCRETE'.") .version("4.0.0") .stringConf diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index 8df650ca39b7e..28c2ec4b5b7a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -51,7 +51,7 @@ object HiveResult extends SQLConfHelper { def getBinaryFormatter: BinaryFormatter = { if (conf.getConf(SQLConf.BINARY_OUTPUT_STYLE).isEmpty) { // Keep the legacy behavior for compatibility. - conf.setConf(SQLConf.BINARY_OUTPUT_STYLE, Some("UTF8")) + conf.setConf(SQLConf.BINARY_OUTPUT_STYLE, Some("UTF-8")) } ToStringBase.getBinaryFormatter(_).toString } diff --git a/sql/core/src/test/resources/sql-tests/inputs/binary.sql b/sql/core/src/test/resources/sql-tests/inputs/binary.sql index 8cd33eccaaf07..fc875b0afb0eb 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/binary.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/binary.sql @@ -1,4 +1,4 @@ ---SET spark.sql.binaryOutputStyle=UTF8 +--SET spark.sql.binaryOutputStyle=UTF-8 SELECT X''; SELECT X'4561736F6E2059616F20323031382D31312D31373A31333A33333A3333'; diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 760ee80260808..155acc98cb33b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -1067,7 +1067,7 @@ class DataFrameSuite extends QueryTest ) assert(df.getRows(10, 20) === expectedAnswer) } - withSQLConf(SQLConf.BINARY_OUTPUT_STYLE.key -> "UTF8") { + withSQLConf(SQLConf.BINARY_OUTPUT_STYLE.key -> "UTF-8") { val expectedAnswer = Seq( Seq("_1", "_2"), Seq("12", "ABC."), From e5e751b98f9ef5b8640079c07a9a342ef471d75d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 12 Jul 2024 14:54:13 -0700 Subject: [PATCH 63/64] [SPARK-48887][K8S] Enable `spark.kubernetes.executor.checkAllContainers` by default ### What changes were proposed in this pull request? This PR aims to enable `spark.kubernetes.executor.checkAllContainers` by default from Apache Spark 4.0.0. ### Why are the changes needed? Since Apache Spark 3.1.0, `spark.kubernetes.executor.checkAllContainers` is supported and useful because [sidecar pattern](https://kubernetes.io/docs/concepts/workloads/pods/sidecar-containers/) is used in many cases. Also, it prevents user mistakes which forget and ignore the sidecars' failures by always reporting sidecar failures via executor status. - https://github.com/apache/spark/pull/29924 ### Does this PR introduce _any_ user-facing change? - This configuration is no-op when there is no other container. - This will report user containers' error correctly when there exist other containers which are provided by the users. ### How was this patch tested? Both `true` and `false` are covered by our CI test coverage since Apache Spark 3.1.0. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #47337 from dongjoon-hyun/SPARK-48887. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/core-migration-guide.md | 2 ++ docs/running-on-kubernetes.md | 2 +- .../src/main/scala/org/apache/spark/deploy/k8s/Config.scala | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md index 26b0ff32cf5d9..aeb5118175ff0 100644 --- a/docs/core-migration-guide.md +++ b/docs/core-migration-guide.md @@ -38,6 +38,8 @@ license: | - Since Spark 4.0, Spark uses `ReadWriteOncePod` instead of `ReadWriteOnce` access mode in persistence volume claims. To restore the legacy behavior, you can set `spark.kubernetes.legacy.useReadWriteOnceAccessMode` to `true`. +- Since Spark 4.0, Spark reports its executor pod status by checking all containers of that pod. To restore the legacy behavior, you can set `spark.kubernetes.executor.checkAllContainers` to `false`. + - Since Spark 4.0, Spark uses `~/.ivy2.5.2` as Ivy user directory by default to isolate the existing systems from Apache Ivy's incompatibility. To restore the legacy behavior, you can set `spark.jars.ivy` to `~/.ivy2`. - Since Spark 4.0, Spark uses the external shuffle service for deleting shuffle blocks for deallocated executors when the shuffle is no longer needed. To restore the legacy behavior, you can set `spark.shuffle.service.removeShuffle` to `false`. diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index f9a4b645786b4..655b30756a298 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1327,7 +1327,7 @@ See the [configuration page](configuration.html) for information on Spark config spark.kubernetes.executor.checkAllContainers - false + true Specify whether executor pods should be check all containers (including sidecars) or only the executor container when determining the pod status. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index a25876de5aee5..2f9ee6943fe61 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -711,7 +711,7 @@ private[spark] object Config extends Logging { "executor status.") .version("3.1.0") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val KUBERNETES_EXECUTOR_MISSING_POD_DETECT_DELTA = ConfigBuilder("spark.kubernetes.executor.missingPodDetectDelta") From b4cd2ecf06169579bd2728649cc1b6dd407bc4c3 Mon Sep 17 00:00:00 2001 From: cashmand Date: Sat, 13 Jul 2024 12:31:40 +0900 Subject: [PATCH 64/64] [SPARK-48495][SQL][DOCS] Describe shredding scheme for Variant ### What changes were proposed in this pull request? For the Variant data type, we plan to add support for columnar storage formats (e.g. Parquet) to write the data shredded across multiple physical columns, and read only the data required for a given query. This PR merges a document describing the approach we plan to take. We can continue to update it as the implementation progresses. ### Why are the changes needed? When implemented, can allow much better performance when reading from columnar storage. More detail is given in the document. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? It is internal documentation, no testing should be needed. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #46831 from cashmand/SPARK-45891. Authored-by: cashmand Signed-off-by: Hyukjin Kwon --- common/variant/README.md | 4 + common/variant/shredding.md | 244 ++++++++++++++++++++++++++++++++++++ 2 files changed, 248 insertions(+) create mode 100644 common/variant/shredding.md diff --git a/common/variant/README.md b/common/variant/README.md index 0fc4d91f3f8a3..3e1b00c494755 100644 --- a/common/variant/README.md +++ b/common/variant/README.md @@ -375,3 +375,7 @@ Field names are case-sensitive. Field names are required to be unique for each o # Versions and extensions An implementation is not expected to parse a Variant value whose metadata version is higher than the version supported by the implementation. However, new types may be added to the specification without incrementing the version ID. In such a situation, an implementation should be able to read the rest of the Variant value if desired. + +# Shredding + +For columnar storage formats, a single Variant object may have poor read performance when only a small number of fields are needed. A better approach is to create separate columns for individual fields, referred to as shredding or subcolumnarization. [shredding.md](shredding.md) describes an approach to shredding Variant columns in Parquet and similar columnar formats. diff --git a/common/variant/shredding.md b/common/variant/shredding.md new file mode 100644 index 0000000000000..ba5e3f8cdcc2b --- /dev/null +++ b/common/variant/shredding.md @@ -0,0 +1,244 @@ +# Shredding Overview + +The Spark Variant type is designed to store and process semi-structured data efficiently, even with heterogeneous values. Query engines encode each variant value in a self-describing format, and store it as a group containing **value** and **metadata** binary fields in Parquet. Since data is often partially homogenous, it can be beneficial to extract certain fields into separate Parquet columns to further improve performance. We refer to this process as "shredding". Each Parquet file remains fully self-describing, with no additional metadata required to read or fully reconstruct the Variant data from the file. Combining shredding with a binary residual provides the flexibility to represent complex, evolving data with an unbounded number of unique fields while limiting the size of file schemas, and retaining the performance benefits of a columnar format. + +This document focuses on the shredding semantics, Parquet representation, implications for readers and writers, as well as the Variant reconstruction. For now, it does not discuss which fields to shred, user-facing API changes, or any engine-specific considerations like how to use shredded columns. The approach builds on top of the generic Spark Variant representation, and leverages the existing Parquet specification for maximum compatibility with the open-source ecosystem. + +At a high level, we replace the **value** and **metadata** of the Variant Parquet group with one or more fields called **object**, **array**, **typed_value** and **untyped_value**. These represent a fixed schema suitable for constructing the full Variant value for each row. + +Shredding lets Spark (or any other query engine) reap the full benefits of Parquet's columnar representation, such as more compact data encoding, min/max statistics for data skipping, and I/O and CPU savings from pruning unnecessary fields not accessed by a query (including the non-shredded Variant binary data). +Without shredding, any query that accesses a Variant column must fetch all bytes of the full binary buffer. With shredding, we can get nearly equivalent performance as in a relational (scalar) data model. + +For example, `select variant_get(variant_col, ‘$.field1.inner_field2’, ‘string’) from tbl` only needs to access `inner_field2`, and the file scan could avoid fetching the rest of the Variant value if this field was shredded into a separate column in the Parquet schema. Similarly, for the query `select * from tbl where variant_get(variant_col, ‘$.id’, ‘integer’) = 123`, the scan could first decode the shredded `id` column, and only fetch/decode the full Variant value for rows that pass the filter. + +# Parquet Example + +Consider the following Parquet schema together with how Variant values might be mapped to it. Notice that we represent each shredded field in **object** as a group of two fields, **typed_value** and **untyped_value**. We extract all homogenous data items of a certain path into **typed_value**, and set aside incompatible data items in **untyped_value**. Intuitively, incompatibilities within the same path may occur because we store the shredding schema per Parquet file, and each file can contain several row groups. Selecting a type for each field that is acceptable for all rows would be impractical because it would require buffering the contents of an entire file before writing. + +Typically, the expectation is that **untyped_value** exists at every level as an option, along with one of **object**, **array** or **typed_value**. If the actual Variant value contains a type that does not match the provided schema, it is stored in **untyped_value**. An **untyped_value** may also be populated if an object can be partially represented: any fields that are present in the schema must be written to those fields, and any missing fields are written to **untyped_valud**. + +``` +optional group variant_col { + optional binary untyped_value; + optional group object { + optional group a { + optional binary untyped_value; + optional int64 typed_value; + } + optional group b { + optional binary untyped_value; + optional group object { + optional group c { + optional binary untyped_value; + optional binary typed_value (STRING); + } + } + } + } +} +``` + +| Variant Value | Top-level untyped_value | b.untyped_value | Non-null in a | Non-null in b.c | +|---------------|--------------------------|---------------|---------------| +| {a: 123, b: {c: “hello”}} | null | null | typed_value | typed_value | +| {a: 1.23, b: {c: “123”}} | null | null | untyped_value | typed_value | +| {a: [1,2,3], b: {c: null}} | null | null | untyped_value | untyped_value | +| {a: 123, c: 456} | {c: 456} | null | typed_value | null | +| {a: 123, b: {c: "hello", d: 456}} | null | {d: 456} | typed_value | typed_value | +| [{a: 1, b: {c: 2}}, {a: 3, b: {c: 4}}] | [{a: 1, b: {c: 2}}, {a: 3, b: {c: 4}}] | null | null | null | + +# Parquet Layout + +The **array** and **object** fields represent Variant array and object types, respectively. Arrays must use the three-level list structure described in https://github.com/apache/parquet-format/blob/master/LogicalTypes.md. + +An **object** field must be a group. Each field name of this inner group corresponds to the Variant value's object field name. Each inner field's type is a recursively shredded variant value: that is, the fields of each object field must be one or more of **object**, **array**, **typed_value** or **untyped_value**. + +Similarly the elements of an **array** must be a group containing one or more of **object**, **array**, **typed_value** or **untyped_value**. + +Each leaf in the schema can store an arbitrary Variant value. It contains an **untyped_value** binary field and a **typed_value** field. If non-null, **untyped_value** represents the value stored as a Variant binary; the metadata and value of a normal Variant are concatenated. The **typed_value** field may be any type that has a corresponding Variant type. For each value in the data, at most one of the **typed_value** and **untyped_value** may be non-null. A writer may omit either field, which is equivalent to all rows being null. + +| typed_value | untyped_value | Meaning | +|-------------|----------------|---------| +| null | null | Field is missing in the reconstructed Variant. | +| null | non-null | Field may be any type in the reconstructed Variant. | +| non-null | null | Field has this column’s type in the reconstructed Variant. | +| non-null | non-null | Invalid | + +The **typed_value** may be absent from the Parquet schema for any field, which is equivalent to its value being always null (in which case the shredded field is always stored as a Variant binary). By the same token, **untyped_value** may be absent, which is equivalent to their value being always null (in which case the field will always be missing or have the type of the **typed_value** column). + +The full metadata and value can be reconstructed from **untyped_value** by treating the leading bytes as metadata, and using the header, dictionary size and final dictionary offset to determine the start of the Variant value section. (See the metadata description in the common/variant/README.md for more detail on how to interpret it.) For example, in the binary below, there is a one-element dictionary, and the final offset (`offset[1]`) indicates that the last dictionary entry ends at the second byte. Therefore the full metadata size is six bytes, and the rest is the value section of the Variant. + +``` + hdr sz offset[0] offset[1] bytes[0] bytes[1] value + -------------------------------------------------------------------- +| | | | | | | +| 0x01 | 0x01 | 0x00 | 0x02 | ‘h’ | ‘i’ | . . . . . . . . +|______|______|_________|_________|________|________|________________ +``` + +# Unshredded values + +If all values can be represented at a given level by whichever of **object**, **array** or **typed_value** is present, **untyped_value** is set to null. + +If a value cannot be represented by whichever of **object**, **array** or **typed_value** is present in the schema, then it is stored in **untyped_value**, and the other fields are set to null. In the Parquet example above, if field **a** was an object or array, or a non-integer scalar, it would be stored in **untyped_value**. + +If a value is an object, and the **object** field is present but does not contain all of the fields in the value, then any remaining fields are stored in an object in **untyped_value**. In the Parquet example above, if field **b** was an object of the form **{"c": 1, "d": 2}"**, then the object **{"d": 2}** would be stored in **untyped_value**, and the **c** field would be shredded recursively under **object.c**. + +Note that an array is always fully shredded if there is an **array** field, so the above consideration for **object** is not relevant for arrays: only one of **array** or **untyped_value** may be non-null at a given level. + +# Using untyped_value vs. typed_value + +In general, it is desirable to store values in the **typed_value** field rather than the **untyped_value** whenever possible. This will typically improve encoding efficiency, and allow the use of Parquet statistics to filter at the row group or page level. In the best case, the **untyped_value** fields are all null and the engine does not need to read them (or it can omit them from the schema on write entirely). There are two main motivations for including the **untyped_value** column: + +1) In a case where there are rare type mismatches (for example, a numeric field with rare strings like “n/a”), we allow the field to be shredded, which could still be a significant performance benefit compared to fetching and decoding the full value/metadata binary. +2) Since there is a single schema per file, there would be no easy way to recover from a type mismatch encountered late in a file write. Parquet files can be large, and buffering all file data before starting to write could be expensive. Including an untyped column for every field guarantees we can adhere to the requested shredding schema. + +The **untyped_value** is stored in a single binary column, rather than storing the value and metadata separately as is done in the unshredded binary format. The motivation for storing them separately for unshredded data is that this lets the engine encode and compress the metadata more efficiently when the fields are consistent across rows. We chose to combine them in the shredded fields: we expect the encoding/compression benefit to be lower, since in the case of uniform data, the values should be stored in typed columns. Using a single binary reduces the extra metadata required in the Parquet schema, which can be significant in some cases. + +# Data Skipping + +Shredded columns are expected to store statistics in the same format as a normal Parquet column. In general, the engine can only skip a row group or page if all rows in the **untyped_value** field are null, since it is possible for a `variant_get` expression to successfully cast a value from the **untyped_value** to the target type. For example, if **typed_value** is of type `int64`, then the string “123” might be contained in **untyped_value**, which would not be reflected in statistics, but could be retained by a filter like `where variant_get(col, “$.field”, “long”) = 123`. If **untyped_value** is all-null, then the engine can prune pages or row groups based on **typed_value**. This specification is not strict about what values may be stored in **untyped_value** rather than **typed_value**, so it is not safe to skip rows based on **typed_value** unless the corresponding **untyped_value** column is all-null, or the engine has specific knowledge of the behavior of the writer that produced the shredded data. + +# Shredding Semantics + +Variant defines a number of integer and decimal types of varying widths. When writing, it would be quite limiting to strictly enforce the mapping between Variant types and Parquet/Spark types. For example, if we chose to shred a field as `int64`, and encountered the value 123 encoded as `int32`, it seems preferable to write this to the **typed_value** column, even though it technically loses information about the type in the original Variant object, and would be reconstructed as an `int64`. + +On the other hand, storing arbitrarily casted values in the **typed_value** column could create inconsistent behavior before and after shredding, and could leak behavior from the writing engine to the reading engine. For example, double-to-string casts can produce different results in different engines. Performing such a cast while shredding (even if we somehow retained the knowledge that the original value was a `double`) could result in confusing behavior changes if shredding took place using a different tool from the query engine that wrote it. + +Our approach is a pragmatic compromise that allows the use of **typed_value** in cases where the type can be losslessly widened without resulting in a significant difference in the reconstructed Variant: + +1) All integer and decimal types in Variant are conceptually a single “number” type. The engine may shred any number into the **typed_value** of any other number, provided that no information about the value is lost. For example, the integer 123 may be shredded as Decimal<9, 2>, but 1.23 may not be shredded as any integer type. + +2) To ensure that behavior remains unchanged before and after shredding, we will aim to have all Spark expressions that operate on Variant be agnostic to the specific numeric type. For example, `cast(val as string)` should produce the string “123” if `val` is any integer or decimal type that is exactly equal to 123. Note that this is unlike the normal Spark behavior for `decimal` types, which would produce “123.00” for `Decimal<9,2>`. + +3) One exception to the above is `schema_of_variant`, which will still report the underlying physical type. This means that `schema_of_variant` may report different numeric types before and after shredding. + +4) Other than integer and decimal, we will not allow casting between types. For example, we will not write the string “123” to an integer **typed_value** column, even though `variant_get(“123”, “$”, “integer”)` would produce the integer 123. Similarly, double and float types are considered distinct from other numeric types, and we would not write them to a numeric **typed_value** column. + +# Reconstructing a Variant + +It is possible to recover a full Variant value using a recursive algorithm, where the initial call is to `ConstructVariant` with the top-level fields, which are assumed to be null if they are not present in the schema. + +``` +# Constructs a Variant from `untyped_value`, `object`, `array` and `typed_value`. +# Only one of object, array and typed_value may be non-null. +def ConstructVariant(untyped_value, object, array, typed_value): + if object is null and array is null and typed_value is null: return untyped_value + elif object is not null: + return ConstructObject(untyped_value, object) + elif array is not null: + return ConstructArray(array) + else: + # Leaf in the tree. + assert(untyped_value is null or untyped_value is VariantNull) + return coalesce(untyped_value, cast(typed_value as Variant)) + +# Construct an object from an `object` group, and a (possibly null) Variant untyped_value +def ConstructObject(untyped_value, object) + # If untyped_value is present and is not an Object, then the result is ambiguous. + assert(untyped_value is null or is_object(untyped_value)) + all_keys = Union(untyped_value.keys, object.fields) + return VariantObject(all_keys.map { key -> + if object[field] is null: (key, untyped_value[field]) + else: (key, ConstructVariant(null, object[field], null, null)) + } + +def ConstructArray(array) + newVariantArray = VariantArray() + for i in range(array.size): + # Any of these may be missing from the schema, in which case they are null. + newVariantArray.append(ConstructVariant(array[i].untyped_value, array[i].object, array[i].array, array[i].typed_value) +``` + +# Nested Parquet Example + +This section describes a more deeply nested example, using a top-level array as the shredding type. + +Below is a sample of JSON that would be fully shredded in this example. It contains an array of objects, containing an “a” field shredded as an array, and a “b” field shredded as an integer. + +``` +[ + { + "a": [1, 2, 3], + "b": 100 + }, + { + "a": [4, 5, 6], + "b": 200 + } +] +``` + + +The corresponding Parquet schema with “a” and “b” as leaf types is: + +``` +optional group variant_col { + optional binary untyped_value; + optional group array (LIST) { + repeated group list { + optional group element { + optional binary untyped_value; + optional group object { + optional group a { + optional binary untyped_value; + optional group array (LIST) { + repeated group list { + optional group element { + optional int64 typed_value; + optional binary untyped_value; + } + } + } + } + optional group b { + optional int64 typed_value; + optional binary untyped_value; + } + } + } + } + } +} +``` + +In the above example schema, if “a” is an array containing a mix of integer and non-integer values, the engine will shred individual elements appropriately into either **typed_value** or **untyped_value**. +If the top-level Variant is not an array (for example, an object), the engine cannot shred the value and it will store it in the top-level **untyped_value**. +Similarly, if "a" is not an array, it will be stored in the **untyped_value** under "a". + +Consider the following example: + +``` +[ + { + "a": [1, 2, 3], + "b": 100, + “c”: “unexpected” + }, + { + "a": [4, 5, 6], + "b": 200 + }, + “not an object” +] +``` + +The second array element can be fully shredded, but the first and third cannot be. The contents of `variant_col.array[*].untyped_value` would be as follows: + +``` +[ + { “c”: “unexpected” }, + NULL, + “not an object” +] +``` + +# Backward and forward compatibility + +Shredding is an optional features of Variant, and readers must continue to be able to read a group containing only a `value` and `metadata` column. + +We will follow the convention defined in https://github.com/delta-io/delta/blob/master/protocol_rfcs/variant-type.md#variant-data-in-parquet, and ignore any fields in the same group as typed_value/untyped_value that start with `_` (underscore). +This is intended to allow future backwards-compatible extensions. In particular, the field names `_metadata_key_paths` and any name starting with `_spark` are reserved, and should not be used by other implementations. +Any extra field names that do not start with an underscore should be assumed to be backwards incompatible, and readers should fail when reading such a schema. + +Engines without shredding support are not expected to be able to read Parquet files that use shredding. Since different files may contain conflicting schemas (e.g. a `typed_value` column with incompatible types in two files), it may not be possible to infer or specify a single schema that would allow all Parquet files for a table to be read.