From 73a2ebb901e5eabb590876eb54b8549d4c5ab2a3 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 30 Dec 2024 09:46:34 +0800 Subject: [PATCH 01/60] [SPARK-50696][PYTHON] Optimize Py4J call for DDL parse method ### What changes were proposed in this pull request? Optimize the DDL parse method in Python ### Why are the changes needed? to reduce the Py4J calls ### 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 #49320 from zhengruifeng/py_opt_ddl. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/types.py | 31 ++----------------- .../spark/sql/api/python/PythonSQLUtils.scala | 23 ++++++++++++++ 2 files changed, 26 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 303c0c3c8bd07..93ac6655b886a 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1916,34 +1916,9 @@ def _parse_datatype_string(s: str) -> DataType: from py4j.java_gateway import JVMView sc = get_active_spark_context() - - def from_ddl_schema(type_str: str) -> DataType: - return _parse_datatype_json_string( - cast(JVMView, sc._jvm) - .org.apache.spark.sql.types.StructType.fromDDL(type_str) - .json() - ) - - def from_ddl_datatype(type_str: str) -> DataType: - return _parse_datatype_json_string( - cast(JVMView, sc._jvm) - .org.apache.spark.sql.api.python.PythonSQLUtils.parseDataType(type_str) - .json() - ) - - try: - # DDL format, "fieldname datatype, fieldname datatype". - return from_ddl_schema(s) - except Exception as e: - try: - # For backwards compatibility, "integer", "struct" and etc. - return from_ddl_datatype(s) - except BaseException: - try: - # For backwards compatibility, "fieldname: datatype, fieldname: datatype" case. - return from_ddl_datatype("struct<%s>" % s.strip()) - except BaseException: - raise e + return _parse_datatype_json_string( + cast(JVMView, sc._jvm).org.apache.spark.sql.api.python.PythonSQLUtils.ddlToJson(s) + ) def _parse_datatype_json_string(json_string: str) -> DataType: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index e33fe38b160af..49fe494903cdc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -148,6 +148,29 @@ private[sql] object PythonSQLUtils extends Logging { DataType.fromJson(json).asInstanceOf[StructType].toDDL } + def ddlToJson(ddl: String): String = { + val dataType = try { + // DDL format, "fieldname datatype, fieldname datatype". + StructType.fromDDL(ddl) + } catch { + case e: Throwable => + try { + // For backwards compatibility, "integer", "struct" and etc. + parseDataType(ddl) + } catch { + case _: Throwable => + try { + // For backwards compatibility, "fieldname: datatype, fieldname: datatype" case. + parseDataType(s"struct<${ddl.trim}>") + } catch { + case _: Throwable => + throw e + } + } + } + dataType.json + } + def unresolvedNamedLambdaVariable(name: String): Column = Column(internal.UnresolvedNamedLambdaVariable.apply(name)) From 7d5aaaad61f2e2f843a982bb565ceb1578812eef Mon Sep 17 00:00:00 2001 From: Kent Yao Date: Mon, 30 Dec 2024 10:22:41 +0800 Subject: [PATCH 02/60] [SPARK-50688][SQL] Eliminate ambiguity for rowTag missing in xml write path ### What changes were proposed in this pull request? This PR eliminates ambiguity for rowTag missing in the XML write path, the `rowTag` is a must for both read and write. ### Why are the changes needed? bugfix ### Does this PR introduce _any_ user-facing change? yes, error msg changing ### How was this patch tested? new tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49317 from yaooqinn/SPARK-50688. Authored-by: Kent Yao Signed-off-by: Kent Yao --- .../main/resources/error/error-conditions.json | 2 +- docs/sql-data-sources-xml.md | 4 ++-- .../sql/execution/datasources/xml/XmlSuite.scala | 16 +++++++++++++++- 3 files changed, 18 insertions(+), 4 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 26b3de7f50890..733b094f744e2 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -5935,7 +5935,7 @@ }, "XML_ROW_TAG_MISSING" : { "message" : [ - " option is required for reading files in XML format." + " option is required for reading/writing files in XML format." ], "sqlState" : "42KDF" }, diff --git a/docs/sql-data-sources-xml.md b/docs/sql-data-sources-xml.md index 6168f570a81a9..949e6239e52ba 100644 --- a/docs/sql-data-sources-xml.md +++ b/docs/sql-data-sources-xml.md @@ -19,7 +19,7 @@ license: | limitations under the License. --- -Spark SQL provides `spark.read().xml("file_1_path","file_2_path")` to read a file or directory of files in XML format into a Spark DataFrame, and `dataframe.write().xml("path")` to write to a xml file. When reading a XML file, the `rowTag` option must be specified to indicate the XML element that maps to a `DataFrame row`. The option() function can be used to customize the behavior of reading or writing, such as controlling behavior of the XML attributes, XSD validation, compression, and so on. +Spark SQL provides `spark.read().xml("file_1_path","file_2_path")` to read a file or directory of files in XML format into a Spark DataFrame, and `dataframe.write().xml("path")` to write to a xml file. The `rowTag` option must be specified to indicate the XML element that maps to a `DataFrame row`. The option() function can be used to customize the behavior of reading or writing, such as controlling behavior of the XML attributes, XSD validation, compression, and so on.
@@ -61,7 +61,7 @@ Data source options of XML can be set via: <books><book></book>...</books> the appropriate value would be book. This is a required option for both read and write. - read + read/write diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala index fe910c21cb0c6..560292b263ba3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/xml/XmlSuite.scala @@ -1930,7 +1930,7 @@ class XmlSuite } checkXmlOptionErrorMessage(Map.empty, - "[XML_ROW_TAG_MISSING] `rowTag` option is required for reading files in XML format.", + "[XML_ROW_TAG_MISSING] `rowTag` option is required for reading/writing files in XML format.", QueryCompilationErrors.xmlRowTagRequiredError(XmlOptions.ROW_TAG).getCause) checkXmlOptionErrorMessage(Map("rowTag" -> ""), "'rowTag' option should not be an empty string.") @@ -1951,6 +1951,20 @@ class XmlSuite .xml(spark.createDataset(Seq(xmlString))) } + test("SPARK-50688: rowTag requirement for write") { + withTempDir { dir => + dir.delete() + val e = intercept[AnalysisException] { + spark.range(1).write.xml(dir.getCanonicalPath) + } + checkError( + exception = e, + condition = "XML_ROW_TAG_MISSING", + parameters = Map("rowTag" -> "`rowTag`") + ) + } + } + test("Primitive field casting") { val ts = Seq("2002-05-30 21:46:54", "2002-05-30T21:46:54", "2002-05-30T21:46:54.1234", "2002-05-30T21:46:54Z", "2002-05-30T21:46:54.1234Z", "2002-05-30T21:46:54-06:00", From 97ee25a7af967d083dfb4b7cf58e38aeb8edcfe5 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Mon, 30 Dec 2024 20:39:57 +0800 Subject: [PATCH 03/60] [SPARK-50691][SQL][FOLLOWUP] Use UnsafeProjection for LocalRelation rows instead of ComparableLocalRelation ### What changes were proposed in this pull request? Use `UnsafeProjection` for `LocalRelation` rows instead of `ComparableLocalRelation`. ### Why are the changes needed? `UnsafeRow.equals` compares the whole byte sequence under it, so it's a convenient way to compare all kind of row values, including `ArrayBasedMapData`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Existing test. ### Was this patch authored or co-authored using generative AI tooling? copilot.nvim. Closes #49322 from vladimirg-db/vladimirg-db/update-comparable-local-relation-to-cover-unsafe-row. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql/catalyst/plans/NormalizePlan.scala | 42 ++++--------------- .../BooleanSimplificationSuite.scala | 4 +- .../optimizer/LimitPushdownSuite.scala | 10 +++-- 3 files changed, 18 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala index ee68e433fbeab..38cf2730e9acd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/NormalizePlan.scala @@ -21,7 +21,6 @@ import org.apache.spark.sql.catalyst.analysis.GetViewColumnByNameAndOrdinal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.connector.read.streaming.SparkDataStream object NormalizePlan extends PredicateHelper { def apply(plan: LogicalPlan): LogicalPlan = @@ -105,8 +104,15 @@ object NormalizePlan extends PredicateHelper { case Project(projectList, child) => Project(normalizeProjectList(projectList), child) case c: KeepAnalyzedQuery => c.storeAnalyzedQuery() - case localRelation: LocalRelation => - ComparableLocalRelation.fromLocalRelation(localRelation) + case localRelation: LocalRelation if !localRelation.data.isEmpty => + /** + * A substitute for the [[LocalRelation.data]]. [[GenericInternalRow]] is incomparable for + * maps, because [[ArrayBasedMapData]] doesn't define [[equals]]. + */ + val unsafeProjection = UnsafeProjection.create(localRelation.schema) + localRelation.copy(data = localRelation.data.map { row => + unsafeProjection(row) + }) } } @@ -137,33 +143,3 @@ object NormalizePlan extends PredicateHelper { case _ => condition // Don't reorder. } } - -/** - * A substitute for the [[LocalRelation]] that has comparable `data` field. [[LocalRelation]]'s - * `data` is incomparable for maps, because [[ArrayBasedMapData]] doesn't define [[equals]]. - */ -case class ComparableLocalRelation( - override val output: Seq[Attribute], - data: Seq[Seq[Expression]], - override val isStreaming: Boolean, - stream: Option[SparkDataStream]) extends LeafNode - -object ComparableLocalRelation { - def fromLocalRelation(localRelation: LocalRelation): ComparableLocalRelation = { - val dataTypes = localRelation.output.map(_.dataType) - ComparableLocalRelation( - output = localRelation.output, - data = localRelation.data.map { row => - if (row != null) { - row.toSeq(dataTypes).zip(dataTypes).map { - case (value, dataType) => Literal(value, dataType) - } - } else { - Seq.empty - } - }, - isStreaming = localRelation.isStreaming, - stream = localRelation.stream - ) - } -} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index fc2697d55f6d0..4cc2ee99284a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -46,7 +46,7 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper { $"e".boolean, $"f".boolean, $"g".boolean, $"h".boolean) val testRelationWithData = LocalRelation.fromExternalRows( - testRelation.output, Seq(Row(1, 2, 3, "abc")) + testRelation.output, Seq(Row(1, 2, 3, "abc", true, true, true, true)) ) val testNotNullableRelation = LocalRelation($"a".int.notNull, $"b".int.notNull, $"c".int.notNull, @@ -54,7 +54,7 @@ class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper { $"h".boolean.notNull) val testNotNullableRelationWithData = LocalRelation.fromExternalRows( - testNotNullableRelation.output, Seq(Row(1, 2, 3, "abc")) + testNotNullableRelation.output, Seq(Row(1, 2, 3, "abc", true, true, true, true)) ) private def checkCondition(input: Expression, expected: LogicalPlan): Unit = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala index 02631c4cf61c9..2dcab5cfd29c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.Add +import org.apache.spark.sql.catalyst.expressions.{Add, GenericInternalRow} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -189,7 +189,9 @@ class LimitPushdownSuite extends PlanTest { } test("full outer join where neither side is limited and left side has larger statistics") { - val xBig = testRelation.copy(data = Seq.fill(10)(null)).subquery("x") + val nulls = new GenericInternalRow( + Seq.fill(testRelation.output.length)(null).toArray.asInstanceOf[Array[Any]]) + val xBig = testRelation.copy(data = Seq.fill(10)(nulls)).subquery("x") assert(xBig.stats.sizeInBytes > y.stats.sizeInBytes) Seq(Some("x.a".attr === "y.b".attr), None).foreach { condition => val originalQuery = xBig.join(y, FullOuter, condition).limit(1).analyze @@ -204,7 +206,9 @@ class LimitPushdownSuite extends PlanTest { } test("full outer join where neither side is limited and right side has larger statistics") { - val yBig = testRelation.copy(data = Seq.fill(10)(null)).subquery("y") + val nulls = new GenericInternalRow( + Seq.fill(testRelation.output.length)(null).toArray.asInstanceOf[Array[Any]]) + val yBig = testRelation.copy(data = Seq.fill(10)(nulls)).subquery("y") assert(x.stats.sizeInBytes < yBig.stats.sizeInBytes) Seq(Some("x.a".attr === "y.b".attr), None).foreach { condition => val originalQuery = x.join(yBig, FullOuter, condition).limit(1).analyze From 43a9b88991b22757c1b5ae40d3fc7efcdb893d82 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 30 Dec 2024 11:50:28 -0800 Subject: [PATCH 04/60] [SPARK-50693][CONNECT] The inputs for TypedScalaUdf should be analyzed ### What changes were proposed in this pull request? Fixes `SparkConnectPlanner` to analyze the inputs for `TypedScalaUdf`. ### Why are the changes needed? The inputs for `TypedScalaUdf` should be analyzed. For example: ```scala val df = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") df.select("*").filter(r => r.getInt(1) > 5) ``` fails with: ``` org.apache.spark.SparkException: [INTERNAL_ERROR] Invalid call to toAttribute on unresolved object SQLSTATE: XX000 at org.apache.spark.sql.catalyst.analysis.Star.toAttribute(unresolved.scala:438) at org.apache.spark.sql.catalyst.plans.logical.Project.$anonfun$output$1(basicLogicalOperators.scala:74) at scala.collection.immutable.List.map(List.scala:247) at scala.collection.immutable.List.map(List.scala:79) at org.apache.spark.sql.catalyst.plans.logical.Project.output(basicLogicalOperators.scala:74) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformTypedFilter(SparkConnectPlanner.scala:1460) at org.apache.spark.sql.connect.planner.SparkConnectPlanner.transformFilter(SparkConnectPlanner.scala:1437) ... ``` ### Does this PR introduce _any_ user-facing change? The failure will not appear. ### How was this patch tested? Added the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49327 from ueshin/issues/SPARK-50693/typed_scala_udf. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../KeyValueGroupedDatasetE2ETestSuite.scala | 68 +++++++++++++++++++ .../sql/UserDefinedFunctionE2ETestSuite.scala | 8 +++ .../connect/planner/SparkConnectPlanner.scala | 23 ++++--- 3 files changed, 90 insertions(+), 9 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala index 988774d5eec94..6fd664d905408 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/KeyValueGroupedDatasetE2ETestSuite.scala @@ -479,6 +479,25 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi "(c,1,1)") } + test("SPARK-50693: groupby on unresolved plan") { + val ds = Seq(("a", 1, 10), ("a", 2, 20), ("b", 2, 1), ("b", 1, 2), ("c", 1, 1)) + .toDF("key", "seq", "value") + val grouped = ds.select("*").groupBy($"key").as[String, (String, Int, Int)] + val aggregated = grouped + .flatMapSortedGroups($"seq", expr("length(key)"), $"value") { (g, iter) => + Iterator(g, iter.mkString(", ")) + } + + checkDatasetUnorderly( + aggregated, + "a", + "(a,1,10), (a,2,20)", + "b", + "(b,1,2), (b,2,1)", + "c", + "(c,1,1)") + } + test("groupby - keyAs, keys") { val ds = Seq(("a", 1, 10), ("a", 2, 20), ("b", 2, 1), ("b", 1, 2), ("c", 1, 1)) .toDF("key", "seq", "value") @@ -597,6 +616,16 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi ("c", 1L)) } + test("SPARK-50693: RowEncoder in udf on unresolved plan") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") + + checkDatasetUnorderly( + ds.select("*").groupByKey(k => k.getAs[String](0)).agg(sum("c2").as[Long]), + ("a", 30L), + ("b", 3L), + ("c", 1L)) + } + test("mapGroups with row encoder") { val df = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") @@ -611,6 +640,21 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi 1) } + test("SPARK-50693: mapGroups with row encoder on unresolved plan") { + val df = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") + + checkDataset( + df.select("*") + .groupByKey(r => r.getAs[String]("c1")) + .mapGroups((_, it) => + it.map(r => { + r.getAs[Int]("c2") + }).sum), + 30, + 3, + 1) + } + test("coGroup with row encoder") { val df1 = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") val df2 = Seq(("x", 10), ("x", 20), ("y", 1), ("y", 2), ("a", 1)).toDF("c1", "c2") @@ -632,6 +676,30 @@ class KeyValueGroupedDatasetE2ETestSuite extends QueryTest with RemoteSparkSessi 3) } + test("SPARK-50693: coGroup with row encoder on unresolved plan") { + val df1 = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") + val df2 = Seq(("x", 10), ("x", 20), ("y", 1), ("y", 2), ("a", 1)).toDF("c1", "c2") + + Seq((df1.select("*"), df2), (df1, df2.select("*")), (df1.select("*"), df2.select("*"))) + .foreach { case (df1, df2) => + val ds1: KeyValueGroupedDataset[String, Row] = + df1.groupByKey(r => r.getAs[String]("c1")) + val ds2: KeyValueGroupedDataset[String, Row] = + df2.groupByKey(r => r.getAs[String]("c1")) + checkDataset( + ds1.cogroup(ds2)((_, it, it2) => { + val sum1 = it.map(r => r.getAs[Int]("c2")).sum + val sum2 = it2.map(r => r.getAs[Int]("c2")).sum + Iterator(sum1 + sum2) + }), + 31, + 3, + 1, + 30, + 3) + } + } + test("serialize as null") { val kvgds = session.range(10).groupByKey(_ % 2) val bytes = SparkSerDeUtils.serialize(kvgds) 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 ca754c7b542f7..8415444c10aac 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 @@ -301,6 +301,14 @@ class UserDefinedFunctionE2ETestSuite extends QueryTest with RemoteSparkSession checkDataset(df.filter(r => r.getInt(1) > 5), Row("a", 10), Row("a", 20)) } + test("SPARK-50693: Filter with row input encoder on unresolved plan") { + val session: SparkSession = spark + import session.implicits._ + val df = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDF("c1", "c2") + + checkDataset(df.select("*").filter(r => r.getInt(1) > 5), Row("a", 10), Row("a", 20)) + } + test("mapPartitions with row input encoder") { val session: SparkSession = spark import session.implicits._ diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index d6ade1ac91264..8bb5e54c36cc6 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -875,18 +875,20 @@ class SparkConnectPlanner( logicalPlan: LogicalPlan, groupingExprs: java.util.List[proto.Expression], sortOrder: Seq[SortOrder]): UntypedKeyValueGroupedDataset = { + val analyzed = session.sessionState.executePlan(logicalPlan).analyzed + assertPlan(groupingExprs.size() >= 1) val dummyFunc = TypedScalaUdf(groupingExprs.get(0), None) val groupExprs = groupingExprs.asScala.toSeq.drop(1).map(expr => transformExpression(expr)) val (qe, aliasedGroupings) = - RelationalGroupedDataset.handleGroupingExpression(logicalPlan, session, groupExprs) + RelationalGroupedDataset.handleGroupingExpression(analyzed, session, groupExprs) UntypedKeyValueGroupedDataset( dummyFunc.outEnc, dummyFunc.inEnc, qe.analyzed, - logicalPlan.output, + analyzed.output, aliasedGroupings, sortOrder) } @@ -895,20 +897,22 @@ class SparkConnectPlanner( logicalPlan: LogicalPlan, groupingExprs: java.util.List[proto.Expression], sortOrder: Seq[SortOrder]): UntypedKeyValueGroupedDataset = { + val analyzed = session.sessionState.executePlan(logicalPlan).analyzed + assertPlan(groupingExprs.size() == 1) - val groupFunc = TypedScalaUdf(groupingExprs.get(0), Some(logicalPlan.output)) + val groupFunc = TypedScalaUdf(groupingExprs.get(0), Some(analyzed.output)) val vEnc = groupFunc.inEnc val kEnc = groupFunc.outEnc - val withGroupingKey = AppendColumns(groupFunc.function, vEnc, kEnc, logicalPlan) + val withGroupingKey = AppendColumns(groupFunc.function, vEnc, kEnc, analyzed) // The input logical plan of KeyValueGroupedDataset need to be executed and analyzed - val analyzed = session.sessionState.executePlan(withGroupingKey).analyzed + val withGroupingKeyAnalyzed = session.sessionState.executePlan(withGroupingKey).analyzed UntypedKeyValueGroupedDataset( kEnc, vEnc, - analyzed, - logicalPlan.output, + withGroupingKeyAnalyzed, + analyzed.output, withGroupingKey.newColumns, sortOrder) } @@ -1457,8 +1461,9 @@ class SparkConnectPlanner( private def transformTypedFilter( fun: proto.CommonInlineUserDefinedFunction, child: LogicalPlan): TypedFilter = { - val udf = TypedScalaUdf(fun, Some(child.output)) - TypedFilter(udf.function, child)(udf.inEnc) + val analyzed = session.sessionState.executePlan(child).analyzed + val udf = TypedScalaUdf(fun, Some(analyzed.output)) + TypedFilter(udf.function, analyzed)(udf.inEnc) } private def transformProject(rel: proto.Project): LogicalPlan = { From 4c39d6fa648a754d0b6585839e2803bc1e2c8cc1 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 31 Dec 2024 09:11:42 +0900 Subject: [PATCH 05/60] [SPARK-50699][PYTHON] Parse and generate DDL string with a specified session ### What changes were proposed in this pull request? Parse and generate DDL string with a specified session ### Why are the changes needed? In `_parse_datatype_string` and `toDDL`, a `SparkSession` or `SparkContext` is always needed. In most cases, the session is already present, so we can avoid creating or fetching the activate session. ### 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 #49331 from zhengruifeng/py_session_ddl_json. Authored-by: Ruifeng Zheng Signed-off-by: Hyukjin Kwon --- python/pyspark/core/context.py | 11 +++++++++++ python/pyspark/pandas/frame.py | 6 ++---- python/pyspark/sql/connect/dataframe.py | 4 ++-- python/pyspark/sql/connect/group.py | 10 +++++----- python/pyspark/sql/connect/session.py | 10 ++++++++++ python/pyspark/sql/pandas/group_ops.py | 10 +++++----- python/pyspark/sql/session.py | 9 +++++++-- python/pyspark/sql/types.py | 24 ++++-------------------- 8 files changed, 46 insertions(+), 38 deletions(-) diff --git a/python/pyspark/core/context.py b/python/pyspark/core/context.py index 9ed4699c4b5ba..42a368555ae9e 100644 --- a/python/pyspark/core/context.py +++ b/python/pyspark/core/context.py @@ -75,6 +75,7 @@ if TYPE_CHECKING: from pyspark.accumulators import AccumulatorParam + from pyspark.sql.types import DataType, StructType __all__ = ["SparkContext"] @@ -2623,6 +2624,16 @@ def _assert_on_driver() -> None: messageParameters={}, ) + def _to_ddl(self, struct: "StructType") -> str: + assert self._jvm is not None + return self._jvm.PythonSQLUtils.jsonToDDL(struct.json()) + + def _parse_ddl(self, ddl: str) -> "DataType": + from pyspark.sql.types import _parse_datatype_json_string + + assert self._jvm is not None + return _parse_datatype_json_string(self._jvm.PythonSQLUtils.ddlToJson(ddl)) + def _test() -> None: import doctest diff --git a/python/pyspark/pandas/frame.py b/python/pyspark/pandas/frame.py index 35b96543b9ebb..86820573344ea 100644 --- a/python/pyspark/pandas/frame.py +++ b/python/pyspark/pandas/frame.py @@ -7292,8 +7292,6 @@ def select_dtypes( 4 1 True 1.0 5 2 False 2.0 """ - from pyspark.sql.types import _parse_datatype_string - include_list: List[str] if not is_list_like(include): include_list = [cast(str, include)] if include is not None else [] @@ -7320,14 +7318,14 @@ def select_dtypes( include_spark_type = [] for inc in include_list: try: - include_spark_type.append(_parse_datatype_string(inc)) + include_spark_type.append(self._internal.spark_frame._session._parse_ddl(inc)) except BaseException: pass exclude_spark_type = [] for exc in exclude_list: try: - exclude_spark_type.append(_parse_datatype_string(exc)) + exclude_spark_type.append(self._internal.spark_frame._session._parse_ddl(exc)) except BaseException: pass diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 33956c867669c..3d8f0eced34b2 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -54,7 +54,7 @@ from pyspark import _NoValue from pyspark._globals import _NoValueType from pyspark.util import is_remote_only -from pyspark.sql.types import Row, StructType, _create_row, _parse_datatype_string +from pyspark.sql.types import Row, StructType, _create_row from pyspark.sql.dataframe import ( DataFrame as ParentDataFrame, DataFrameNaFunctions as ParentDataFrameNaFunctions, @@ -2037,7 +2037,7 @@ def _map_partitions( _validate_pandas_udf(func, evalType) if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = cast(StructType, self._session._parse_ddl(schema)) udf_obj = UserDefinedFunction( func, returnType=schema, diff --git a/python/pyspark/sql/connect/group.py b/python/pyspark/sql/connect/group.py index 006af8756e639..11adc8850fec1 100644 --- a/python/pyspark/sql/connect/group.py +++ b/python/pyspark/sql/connect/group.py @@ -35,7 +35,7 @@ from pyspark.sql.group import GroupedData as PySparkGroupedData from pyspark.sql.pandas.group_ops import PandasCogroupedOps as PySparkPandasCogroupedOps from pyspark.sql.pandas.functions import _validate_pandas_udf # type: ignore[attr-defined] -from pyspark.sql.types import NumericType, StructType, _parse_datatype_string +from pyspark.sql.types import NumericType, StructType import pyspark.sql.connect.plan as plan from pyspark.sql.column import Column @@ -295,7 +295,7 @@ def applyInPandas( _validate_pandas_udf(func, PythonEvalType.SQL_GROUPED_MAP_PANDAS_UDF) if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = cast(StructType, self._df._session._parse_ddl(schema)) udf_obj = UserDefinedFunction( func, returnType=schema, @@ -369,7 +369,7 @@ def applyInArrow( _validate_pandas_udf(func, PythonEvalType.SQL_GROUPED_MAP_ARROW_UDF) if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = cast(StructType, self._df._session._parse_ddl(schema)) udf_obj = UserDefinedFunction( func, returnType=schema, @@ -414,7 +414,7 @@ def applyInPandas( _validate_pandas_udf(func, PythonEvalType.SQL_COGROUPED_MAP_PANDAS_UDF) if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = cast(StructType, self._gd1._df._session._parse_ddl(schema)) udf_obj = UserDefinedFunction( func, returnType=schema, @@ -445,7 +445,7 @@ def applyInArrow( _validate_pandas_udf(func, PythonEvalType.SQL_COGROUPED_MAP_ARROW_UDF) if isinstance(schema, str): - schema = cast(StructType, _parse_datatype_string(schema)) + schema = cast(StructType, self._gd1._df._session._parse_ddl(schema)) udf_obj = UserDefinedFunction( func, returnType=schema, diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 925eaaeabf605..3f1663d06850a 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -1111,6 +1111,16 @@ def creator(old_session_id: str) -> "SparkSession": return creator, (self._session_id,) + def _to_ddl(self, struct: StructType) -> str: + ddl = self._client._analyze(method="json_to_ddl", json_string=struct.json()).ddl_string + assert ddl is not None + return ddl + + def _parse_ddl(self, ddl: str) -> DataType: + dt = self._client._analyze(method="ddl_parse", ddl_string=ddl).parsed + assert dt is not None + return dt + SparkSession.__doc__ = PySparkSession.__doc__ diff --git a/python/pyspark/sql/pandas/group_ops.py b/python/pyspark/sql/pandas/group_ops.py index bd12b41b34361..343a68bf010bf 100644 --- a/python/pyspark/sql/pandas/group_ops.py +++ b/python/pyspark/sql/pandas/group_ops.py @@ -36,7 +36,7 @@ ) from pyspark.sql.streaming.stateful_processor import StatefulProcessor, StatefulProcessorHandle from pyspark.sql.streaming.stateful_processor_util import TransformWithStateInPandasFuncMode -from pyspark.sql.types import StructType, _parse_datatype_string +from pyspark.sql.types import StructType if TYPE_CHECKING: from pyspark.sql.pandas._typing import ( @@ -348,9 +348,9 @@ def applyInPandasWithState( ] if isinstance(outputStructType, str): - outputStructType = cast(StructType, _parse_datatype_string(outputStructType)) + outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) if isinstance(stateStructType, str): - stateStructType = cast(StructType, _parse_datatype_string(stateStructType)) + stateStructType = cast(StructType, self._df._session._parse_ddl(stateStructType)) udf = pandas_udf( func, # type: ignore[call-overload] @@ -502,7 +502,7 @@ def transformWithStateInPandas( if initialState is not None: assert isinstance(initialState, GroupedData) if isinstance(outputStructType, str): - outputStructType = cast(StructType, _parse_datatype_string(outputStructType)) + outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) def handle_pre_init( statefulProcessorApiClient: StatefulProcessorApiClient, @@ -681,7 +681,7 @@ def transformWithStateWithInitStateUDF( return result if isinstance(outputStructType, str): - outputStructType = cast(StructType, _parse_datatype_string(outputStructType)) + outputStructType = cast(StructType, self._df._session._parse_ddl(outputStructType)) df = self._df diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 00fa60442b410..f3a1639fddafa 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -58,7 +58,6 @@ _has_nulltype, _merge_type, _create_converter, - _parse_datatype_string, _from_numpy_type, ) from pyspark.errors.exceptions.captured import install_exception_handler @@ -1501,7 +1500,7 @@ def createDataFrame( # type: ignore[misc] ) if isinstance(schema, str): - schema = cast(Union[AtomicType, StructType, str], _parse_datatype_string(schema)) + schema = cast(Union[AtomicType, StructType, str], self._parse_ddl(schema)) elif isinstance(schema, (list, tuple)): # Must re-encode any unicode strings to be consistent with StructField names schema = [x.encode("utf-8") if not isinstance(x, str) else x for x in schema] @@ -2338,6 +2337,12 @@ def clearTags(self) -> None: """ self._jsparkSession.clearTags() + def _to_ddl(self, struct: StructType) -> str: + return self._sc._to_ddl(struct) + + def _parse_ddl(self, ddl: str) -> DataType: + return self._sc._parse_ddl(ddl) + def _test() -> None: import os diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 93ac6655b886a..f40a8bf62b290 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1563,16 +1563,9 @@ def toDDL(self) -> str: session = SparkSession.getActiveSession() assert session is not None - return session._client._analyze( # type: ignore[return-value] - method="json_to_ddl", json_string=self.json() - ).ddl_string - + return session._to_ddl(self) else: - from py4j.java_gateway import JVMView - - sc = get_active_spark_context() - assert sc._jvm is not None - return cast(JVMView, sc._jvm).PythonSQLUtils.jsonToDDL(self.json()) + return get_active_spark_context()._to_ddl(self) class VariantType(AtomicType): @@ -1907,18 +1900,9 @@ def _parse_datatype_string(s: str) -> DataType: if is_remote(): from pyspark.sql.connect.session import SparkSession - return cast( - DataType, - SparkSession.active()._client._analyze(method="ddl_parse", ddl_string=s).parsed, - ) - + return SparkSession.active()._parse_ddl(s) else: - from py4j.java_gateway import JVMView - - sc = get_active_spark_context() - return _parse_datatype_json_string( - cast(JVMView, sc._jvm).org.apache.spark.sql.api.python.PythonSQLUtils.ddlToJson(s) - ) + return get_active_spark_context()._parse_ddl(s) def _parse_datatype_json_string(json_string: str) -> DataType: From fd8230b84975f47b4ccf4308856078831d9df365 Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Tue, 31 Dec 2024 09:14:18 +0900 Subject: [PATCH 06/60] [SPARK-49229][CONNECT] Deduplicate Scala UDF handling in the SparkConnectPlanner ### What changes were proposed in this pull request? This PR removes some duplicate codes from `transformScalaFunction` and `transformScalaUDF` methods of `SparkConnectPlanner`. ### Why are the changes needed? Keep the code tidy. ### 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 #49324 from xupefei/udf-handling-deduplicate. Authored-by: Paddy Xu Signed-off-by: Hyukjin Kwon --- .../connect/planner/SparkConnectPlanner.scala | 44 ++++++++++--------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 8bb5e54c36cc6..628b758dd4e2a 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -77,7 +77,7 @@ import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.execution.streaming.GroupStateImpl.groupStateTimeoutFromString import org.apache.spark.sql.execution.streaming.StreamingQueryWrapper import org.apache.spark.sql.expressions.{Aggregator, ReduceAggregator, SparkUserDefinedFunction, UserDefinedAggregator, UserDefinedFunction} -import org.apache.spark.sql.internal.{CatalogImpl, MergeIntoWriterImpl, TypedAggUtils} +import org.apache.spark.sql.internal.{CatalogImpl, MergeIntoWriterImpl, TypedAggUtils, UserDefinedFunctionUtils} import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode, StreamingQuery, StreamingQueryListener, StreamingQueryProgress, Trigger} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -1727,34 +1727,36 @@ class SparkConnectPlanner( } /** - * Translates a Scala user-defined function from proto to the Catalyst expression. + * Translates a Scala user-defined function or aggregator from proto to the corresponding + * Catalyst expression. * * @param fun - * Proto representation of the Scala user-defined function. + * Proto representation of the Scala user-defined function or aggregator. * @return - * ScalaUDF. + * An expression, either a ScalaUDF or a ScalaAggregator. */ private def transformScalaUDF(fun: proto.CommonInlineUserDefinedFunction): Expression = { - val udf = fun.getScalarScalaUdf - val udfPacket = unpackUdf(fun) - if (udf.getAggregate) { - ScalaAggregator( - transformScalaFunction(fun).asInstanceOf[UserDefinedAggregator[Any, Any, Any]], - fun.getArgumentsList.asScala.map(transformExpression).toSeq) - .toAggregateExpression() - } else { - ScalaUDF( - function = udfPacket.function, - dataType = transformDataType(udf.getOutputType), - children = fun.getArgumentsList.asScala.map(transformExpression).toSeq, - inputEncoders = udfPacket.inputEncoders.map(e => Try(ExpressionEncoder(e)).toOption), - outputEncoder = Option(ExpressionEncoder(udfPacket.outputEncoder)), - udfName = Option(fun.getFunctionName), - nullable = udf.getNullable, - udfDeterministic = fun.getDeterministic) + val children = fun.getArgumentsList.asScala.map(transformExpression).toSeq + transformScalaFunction(fun) match { + case udf: SparkUserDefinedFunction => + UserDefinedFunctionUtils.toScalaUDF(udf, children) + case uda: UserDefinedAggregator[_, _, _] => + ScalaAggregator(uda, children).toAggregateExpression() + case other => + throw InvalidPlanInput( + s"Unsupported UserDefinedFunction implementation: ${other.getClass}") } } + /** + * Translates a Scala user-defined function or aggregator. from proto to a UserDefinedFunction. + * + * @param fun + * Proto representation of the Scala user-defined function or aggregator. + * @return + * A concrete UserDefinedFunction implementation, either a SparkUserDefinedFunction or a + * UserDefinedAggregator. + */ private def transformScalaFunction( fun: proto.CommonInlineUserDefinedFunction): UserDefinedFunction = { val udf = fun.getScalarScalaUdf From c4145db09ffeb0d0f9714a517ff9bacb018483e3 Mon Sep 17 00:00:00 2001 From: Mihailo Timotic Date: Tue, 31 Dec 2024 09:18:51 +0900 Subject: [PATCH 07/60] [SPARK-50689][SQL][FOLLOWUP] Enforce deterministic ordering in LCA aggregate lists ### What changes were proposed in this pull request? Using `Set` to produce aggregate lists may result in those aggregates being non-deterministic. Instead we switch to using `LinkedHashSet`. ### Why are the changes needed? It's better for the analyzer to produce stable query plans, regardless of the java/scala versions. ### 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 #49334 from mihailotim-db/mihailotim-db/lca_followup. Authored-by: Mihailo Timotic Signed-off-by: Hyukjin Kwon --- .../analysis/ResolveLateralColumnAliasReference.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala index 677d852ebad57..cb26820a0c79d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveLateralColumnAliasReference.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import java.util.LinkedHashSet +import scala.jdk.CollectionConverters._ + import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.WindowExpression.hasWindowExpression import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression @@ -224,7 +226,7 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] { if (!aggregateExpressions.forall(eligibleToLiftUp)) { agg } else { - val newAggExprs = collection.mutable.Set.empty[NamedExpression] + val newAggExprs = new LinkedHashSet[NamedExpression] val expressionMap = collection.mutable.LinkedHashMap.empty[Expression, NamedExpression] // Extract the expressions to keep in the Aggregate. Return the transformed expression // fully substituted with the attribute reference to the extracted expressions. @@ -251,11 +253,11 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] { } } val ne = expressionMap.getOrElseUpdate(aggExpr.canonicalized, assignAlias(aggExpr)) - newAggExprs += ne + newAggExprs.add(ne) ne.toAttribute case e if groupingExpressions.exists(_.semanticEquals(e)) => val ne = expressionMap.getOrElseUpdate(e.canonicalized, assignAlias(e)) - newAggExprs += ne + newAggExprs.add(ne) ne.toAttribute case e => e.mapChildren(extractExpressions) } @@ -264,7 +266,7 @@ object ResolveLateralColumnAliasReference extends Rule[LogicalPlan] { extractExpressions(_).asInstanceOf[NamedExpression]) Project( projectList = projectExprs, - child = agg.copy(aggregateExpressions = newAggExprs.toSeq) + child = agg.copy(aggregateExpressions = newAggExprs.asScala.toSeq) ) } From 6099de7f5995b438335a5a545d5cf62dd3d68f0d Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Tue, 31 Dec 2024 14:11:57 +0900 Subject: [PATCH 08/60] [SPARK-50697][SQL] Enable tail-recursion wherever possible ### What changes were proposed in this pull request? This pr adds `scala.annotation.tailrec` inspected by IDE (IntelliJ), these are new cases after Spark 4.0. ### Why are the changes needed? To improve performance. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GItHub Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #49330 from LuciferYang/tailrec-master. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Hyukjin Kwon --- .../spark/sql/execution/InsertSortForLimitAndOffset.scala | 4 ++++ .../spark/sql/scripting/SqlScriptingExecutionNode.scala | 1 + 2 files changed, 5 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala index 6c7a9206a8e39..fa63e04d91b00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/InsertSortForLimitAndOffset.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import scala.annotation.tailrec + import org.apache.spark.sql.catalyst.expressions.SortOrder import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.rules.Rule @@ -49,6 +51,7 @@ object InsertSortForLimitAndOffset extends Rule[SparkPlan] { } object SinglePartitionShuffleWithGlobalOrdering { + @tailrec def unapply(plan: SparkPlan): Option[Seq[SortOrder]] = plan match { case ShuffleExchangeExec(SinglePartition, SparkPlanWithGlobalOrdering(ordering), _, _) => Some(ordering) @@ -61,6 +64,7 @@ object InsertSortForLimitAndOffset extends Rule[SparkPlan] { // Note: this is not implementing a generalized notion of "global order preservation", but just // tackles the regular ORDER BY semantics with optional LIMIT (top-K). object SparkPlanWithGlobalOrdering { + @tailrec def unapply(plan: SparkPlan): Option[Seq[SortOrder]] = plan match { case p: SortExec if p.global => Some(p.sortOrder) case p: LocalLimitExec => unapply(p.child) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala index 2d50d37e2cb83..ee47491b803bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/scripting/SqlScriptingExecutionNode.scala @@ -790,6 +790,7 @@ class ForStatementExec( case ForState.VariableCleanup => dropVariablesExec.getTreeIterator.hasNext }) + @scala.annotation.tailrec override def next(): CompoundStatementExec = state match { case ForState.VariableAssignment => From 5ef556bc64e4922081c93d1c10c8c2c8551f5e82 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Tue, 31 Dec 2024 16:43:27 +0900 Subject: [PATCH 09/60] [SPARK-50706][PYTHON][TESTS] Skip test_value_state_ttl_expiration in Coverage build ### What changes were proposed in this pull request? This PR proposes to skip `test_value_state_ttl_expiration` in Coverage build for now. ### Why are the changes needed? To make the build passing for now. It fails when the Coverage is on (https://github.com/apache/spark/actions/runs/12544995465/job/34978553717): ``` ====================================================================== ERROR [12.848s]: test_value_state_ttl_expiration (pyspark.sql.tests.pandas.test_pandas_transform_with_state.TransformWithStateInPandasTests.test_value_state_ttl_expiration) ---------------------------------------------------------------------- Traceback (most recent call last): File "/__w/spark/spark/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py", line 403, in test_value_state_ttl_expiration q.processAllAvailable() File "/__w/spark/spark/python/pyspark/sql/streaming/query.py", line 351, in processAllAvailable return self._jsq.processAllAvailable() ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/__w/spark/spark/python/lib/py4j-0.10.9.8-src.zip/py4j/java_gateway.py", line 1355, in __call__ return_value = get_return_value( ^^^^^^^^^^^^^^^^^ File "/__w/spark/spark/python/pyspark/errors/exceptions/captured.py", line 253, in deco raise converted from None pyspark.errors.exceptions.captured.StreamingQueryException: [STREAM_FAILED] Query [id = 623e9008-52cb-4b9d-9343-432e7bd855bb, runId = cc06b909-37fd-4acd-98ff-8809b9df92c7] terminated with exception: [FOREACH_BATCH_USER_FUNCTION_ERROR] An error occurred in the user provided function in foreach batch sink. Reason: An exception was raised by the Python Proxy. Return Message: Traceback (most recent call last): File "/__w/spark/spark/python/lib/py4j-0.10.9.8-src.zip/py4j/clientserver.py", line 641, in _call_proxy return_value = getattr(self.pool[obj_id], method)(*params) ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ File "/__w/spark/spark/python/pyspark/sql/utils.py", line 157, in call raise e File "/__w/spark/spark/python/pyspark/sql/utils.py", line 154, in call self.func(DataFrame(jdf, wrapped_session_jdf), batch_id) File "/__w/spark/spark/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py", line 334, in check_results assertDataFrameEqual( File "/__w/spark/spark/python/pyspark/testing/utils.py", line 1074, in assertDataFrameEqual assert_rows_equal(actual_list, expected_list, maxErrors=maxErrors, showOnlyDiff=showOnlyDiff) File "/__w/spark/spark/python/pyspark/testing/utils.py", line 1030, in assert_rows_equal raise PySparkAssertionError( pyspark.errors.exceptions.base.PySparkAssertionError: [DIFFERENT_ROWS] Results do not match: ( 75.00000 % ) *** actual *** Row(id='count-0', count=2) Row(id='count-1', count=2) ! Row(id='ttl-count-0', count=1) ! Row(id='ttl-count-1', count=1) ! Row(id='ttl-list-state-count-0', count=1) ! Row(id='ttl-list-state-count-1', count=1) ! Row(id='ttl-map-state-count-0', count=1) ! Row(id='ttl-map-state-count-1', count=1) *** expected *** Row(id='count-0', count=2) Row(id='count-1', count=2) ! Row(id='ttl-count-0', count=2) ! Row(id='ttl-count-1', count=2) ! Row(id='ttl-list-state-count-0', count=3) ! Row(id='ttl-list-state-count-1', count=3) ! Row(id='ttl-map-state-count-0', count=2) ! Row(id='ttl-map-state-count-1', count=2) ``` ### Does this PR introduce _any_ user-facing change? No, test-only. ### How was this patch tested? Will monitor the Coverage build. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49337 from HyukjinKwon/SPARK-50706. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../sql/tests/pandas/test_pandas_transform_with_state.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 183e93a871bd8..7a00f2f43d149 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -922,6 +922,9 @@ def test_transform_with_state_in_pandas_batch_query_initial_state(self): # This test covers mapState with TTL, an empty state variable # and additional test against initial state python runner + @unittest.skipIf( + "COVERAGE_PROCESS_START" in os.environ, "Flaky with coverage enabled, skipping for now." + ) def test_transform_with_map_state_metadata(self): checkpoint_path = tempfile.mktemp() From 8a09817bf94892bf0e946e414f1d54d1ebad5f35 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 31 Dec 2024 15:49:11 +0800 Subject: [PATCH 10/60] [SPARK-50701][PYTHON] Make plotting require the minimum plotly version ### What changes were proposed in this pull request? Make plotting require the minimum plotly version ### Why are the changes needed? the `minimum_plotly_version = "4.8"` didn't take effect in existing implementation ### 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 #49333 from zhengruifeng/py_plot_minimum_version. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/utils.py | 22 +++++++++++++++++++--- 1 file changed, 19 insertions(+), 3 deletions(-) diff --git a/python/pyspark/sql/utils.py b/python/pyspark/sql/utils.py index 3cacc5b9d021a..fbe0b274b496c 100644 --- a/python/pyspark/sql/utils.py +++ b/python/pyspark/sql/utils.py @@ -118,18 +118,34 @@ def require_test_compiled() -> None: def require_minimum_plotly_version() -> None: """Raise ImportError if plotly is not installed""" + from pyspark.loose_version import LooseVersion + minimum_plotly_version = "4.8" try: - import plotly # noqa: F401 + import plotly + + have_plotly = True except ImportError as error: + have_plotly = False + raised_error = error + if not have_plotly: raise PySparkImportError( errorClass="PACKAGE_NOT_INSTALLED", messageParameters={ - "package_name": "plotly", + "package_name": "Plotly", "minimum_version": str(minimum_plotly_version), }, - ) from error + ) from raised_error + if LooseVersion(plotly.__version__) < LooseVersion(minimum_plotly_version): + raise PySparkImportError( + errorClass="UNSUPPORTED_PACKAGE_VERSION", + messageParameters={ + "package_name": "Plotly", + "minimum_version": str(minimum_plotly_version), + "current_version": str(plotly.__version__), + }, + ) class ForeachBatchFunction: From e1fb18d7a3d41c01eae46b79a08670d878bbbbd6 Mon Sep 17 00:00:00 2001 From: andrej-gobeljic_data Date: Tue, 31 Dec 2024 16:27:38 +0800 Subject: [PATCH 11/60] [SPARK-50692] Add RPAD pushdown support ### What changes were proposed in this pull request? This PR makes RPAD and LPAD functions translatable across connectors. Individual connectors can now support them via their dialects. ### Why are the changes needed? When trying to compare CHAR(...) and a literal, Spark injects a RPAD function to make the CHAR parameter the same length. This behaviour is per ANSI standard. The problem is that RPAD is not only an unsupported, but an untranslatable function. This prevents CHAR + literal comparison pushdown. Solution is to support RPAD pushdowns and implement it when needed. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? This PR only introduces the ability to support the RPAD pushdown. Individual connectors will have their own tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49325 from andrej-db/SPARK-50692-RPAD. Authored-by: andrej-gobeljic_data Signed-off-by: Wenchen Fan --- .../apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java | 2 +- .../apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java index bd2dec9e27be0..49afcd5ebcd50 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/connector/util/V2ExpressionSQLBuilder.java @@ -118,7 +118,7 @@ yield visitBinaryArithmetic( "COT", "ASIN", "ASINH", "ACOS", "ACOSH", "ATAN", "ATANH", "ATAN2", "CBRT", "DEGREES", "RADIANS", "SIGN", "WIDTH_BUCKET", "SUBSTRING", "UPPER", "LOWER", "TRANSLATE", "DATE_ADD", "DATE_DIFF", "TRUNC", "AES_ENCRYPT", "AES_DECRYPT", "SHA1", "SHA2", "MD5", - "CRC32", "BIT_LENGTH", "CHAR_LENGTH", "CONCAT" -> + "CRC32", "BIT_LENGTH", "CHAR_LENGTH", "CONCAT", "RPAD", "LPAD" -> visitSQLFunction(name, expressionsToStringArray(e.children())); case "CASE_WHEN" -> visitCaseWhen(expressionsToStringArray(e.children())); case "TRIM" -> visitTrim("BOTH", expressionsToStringArray(e.children())); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala index b0ce2bb4293e1..23ae5ee7b9be4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/util/V2ExpressionBuilder.scala @@ -305,6 +305,8 @@ class V2ExpressionBuilder(e: Expression, isPredicate: Boolean = false) extends L case _: Md5 => generateExpressionWithName("MD5", expr, isPredicate) case _: Sha1 => generateExpressionWithName("SHA1", expr, isPredicate) case _: Sha2 => generateExpressionWithName("SHA2", expr, isPredicate) + case _: StringLPad => generateExpressionWithName("LPAD", expr, isPredicate) + case _: StringRPad => generateExpressionWithName("RPAD", expr, isPredicate) // TODO supports other expressions case ApplyFunctionExpression(function, children) => val childrenExpressions = children.flatMap(generateExpression(_)) From 5334494b00fc77cc26d9fe43677ce348c0ff979f Mon Sep 17 00:00:00 2001 From: "chenliang.lu" Date: Tue, 31 Dec 2024 20:20:29 +0800 Subject: [PATCH 12/60] [SPARK-50648][CORE] Cleanup zombie tasks in non-running stages when the job is cancelled ### What changes were proposed in this pull request? This is a problem that Spark always had. See the following section for the scenario when the problem occurs. When cancel a job, some tasks may be still running. The reason is that when `DAGScheduler#handleTaskCompletion` encounters FetchFailed, `markStageAsFinished` will be called to remove the stage in `DAGScheduler#runningStages` (see https://github.com/apache/spark/blob/7cd5c4a1d1eb56fa92c10696bdbd8450d357b128/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2059) and don't `killAllTaskAttempts`. But `DAGScheduler#cancelRunningIndependentStages` only find `runningStages`, this will leave zombie shuffle tasks, occupying cluster resources. ### Why are the changes needed? Assume a job is stage1-> stage2, when FetchFailed occurs during the stage 2, the stage1 and stage2 will resubmit (stage2 may still have some tasks running even if stage2 is resubmitted , this is as expected, because these tasks may eventually succeed and avoid retry) But during the execution of the stage1-retry , if the SQL is canceled, the tasks in stage1 and stage1-retry can all be killed, but the tasks previously running in stage2 are still running and can't be killed. These tasks can greatly affect cluster stability and occupy resources. ### Does this PR introduce _any_ user-facing change? No ### Was this patch authored or co-authored using generative AI tooling? No Closes #49270 from yabola/zombie-task-when-shuffle-retry. Authored-by: chenliang.lu Signed-off-by: Yi Wu --- .../apache/spark/scheduler/DAGScheduler.scala | 3 +- .../spark/scheduler/DAGSchedulerSuite.scala | 52 +++++++++++++++++++ 2 files changed, 54 insertions(+), 1 deletion(-) 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 4f7338f74e298..e06b7d86e1db0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -2937,7 +2937,8 @@ private[spark] class DAGScheduler( } else { // This stage is only used by the job, so finish the stage if it is running. val stage = stageIdToStage(stageId) - if (runningStages.contains(stage)) { + // Stages with failedAttemptIds may have tasks that are running + if (runningStages.contains(stage) || stage.failedAttemptIds.nonEmpty) { try { // killAllTaskAttempts will fail if a SchedulerBackend does not implement killTask taskScheduler.killAllTaskAttempts(stageId, shouldInterruptTaskThread(job), reason) if (legacyAbortStageAfterKillTasks) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 243d33fe55a79..3e507df706ba5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -185,6 +185,8 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti private var firstInit: Boolean = _ /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() + /** Track running tasks, the key is the task's stageId , the value is the task's partitionId */ + var runningTaskInfos = new HashMap[Int, HashSet[Int]]() /** Stages for which the DAGScheduler has called TaskScheduler.killAllTaskAttempts(). */ val cancelledStages = new HashSet[Int]() @@ -206,12 +208,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti // normally done by TaskSetManager taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet + runningTaskInfos.put(taskSet.stageId, new HashSet[Int]() ++ taskSet.tasks.map(_.partitionId)) } override def killTaskAttempt( taskId: Long, interruptThread: Boolean, reason: String): Boolean = false override def killAllTaskAttempts( stageId: Int, interruptThread: Boolean, reason: String): Unit = { cancelledStages += stageId + runningTaskInfos.remove(stageId) } override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = { taskSets.filter(_.stageId == stageId).lastOption.foreach { ts => @@ -393,6 +397,14 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti handleShuffleMergeFinalized(shuffleMapStage, shuffleMapStage.shuffleDep.shuffleMergeId) } } + + override private[scheduler] def handleTaskCompletion(event: CompletionEvent): Unit = { + super.handleTaskCompletion(event) + runningTaskInfos.get(event.task.stageId).foreach{ partitions => + partitions -= event.task.partitionId + if (partitions.isEmpty) runningTaskInfos.remove(event.task.stageId) + } + } } override def beforeEach(): Unit = { @@ -2252,6 +2264,46 @@ class DAGSchedulerSuite extends SparkFunSuite with TempLocalSparkContext with Ti assert(scheduler.activeJobs.isEmpty) } + test("SPARK-50648: when job is cancelled during shuffle retry in parent stage, " + + "should kill all running tasks") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + completeShuffleMapStageSuccessfully(0, 0, 2) + sc.listenerBus.waitUntilEmpty() + + val info = new TaskInfo( + 3, index = 1, attemptNumber = 1, + partitionId = taskSets(1).tasks(0).partitionId, 0L, "", "", TaskLocality.ANY, true) + // result task 0.0 fetch failed, but result task 1.0 is still running + runEvent(makeCompletionEvent(taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleDep.shuffleId, 0L, 0, 1, "ignored"), + null, + Seq.empty, + Array.empty, + info)) + sc.listenerBus.waitUntilEmpty() + + Thread.sleep(DAGScheduler.RESUBMIT_TIMEOUT * 2) + // map stage is running by resubmitted, result stage is waiting + // map tasks and the origin result task 1.0 are running + assert(scheduler.runningStages.size == 1, "Map stage should be running") + val mapStage = scheduler.runningStages.head + assert(mapStage.id === 0) + assert(mapStage.latestInfo.failureReason.isEmpty) + assert(scheduler.waitingStages.size == 1, "Result stage should be waiting") + assert(runningTaskInfos.size == 2) + assert(runningTaskInfos(taskSets(1).stageId).size == 1, + "origin result task 1.0 should be running") + + scheduler.doCancelAllJobs() + // all tasks should be killed + assert(runningTaskInfos.isEmpty) + assert(scheduler.runningStages.isEmpty) + assert(scheduler.waitingStages.isEmpty) + } + test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { val acc = new LongAccumulator { override def add(v: java.lang.Long): Unit = throw new DAGSchedulerSuiteDummyException From 1c79b54ac9346fd25a5b1732675a308425e190e3 Mon Sep 17 00:00:00 2001 From: George314159 Date: Wed, 1 Jan 2025 09:20:08 +0800 Subject: [PATCH 13/60] [SPARK-49491][SQL] Replace AnyRefMap with HashMap ### What changes were proposed in this pull request? Replace AnyRefMap with HashMap. ### Why are the changes needed? HashMap has better performance in Scala 2.13: https://issues.apache.org/jira/browse/SPARK-49491. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing Unit Tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #48128 from George314159/SPARK-49491. Authored-by: George314159 Signed-off-by: Yuming Wang --- .../sql/catalyst/util/RebaseDateTime.scala | 20 +- .../DateTimeRebaseBenchmark-jdk21-results.txt | 206 +++++++++--------- .../DateTimeRebaseBenchmark-results.txt | 206 +++++++++--------- 3 files changed, 216 insertions(+), 216 deletions(-) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala index 9c043320dc812..8dff1ceccfcfe 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/util/RebaseDateTime.scala @@ -22,7 +22,7 @@ import java.time.temporal.ChronoField import java.util.{Calendar, TimeZone} import java.util.Calendar.{DAY_OF_MONTH, DST_OFFSET, ERA, HOUR_OF_DAY, MINUTE, MONTH, SECOND, YEAR, ZONE_OFFSET} -import scala.collection.mutable.AnyRefMap +import scala.collection.mutable.HashMap import com.fasterxml.jackson.databind.ObjectMapper import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule} @@ -285,12 +285,12 @@ object RebaseDateTime { } // Loads rebasing info from an JSON file. JSON records in the files should conform to - // `JsonRebaseRecord`. AnyRefMap is used here instead of Scala's immutable map because - // it is 2 times faster in DateTimeRebaseBenchmark. - private[sql] def loadRebaseRecords(fileName: String): AnyRefMap[String, RebaseInfo] = { + // `JsonRebaseRecord`. Mutable HashMap is used here instead of AnyRefMap due to SPARK-49491. + private[sql] def loadRebaseRecords(fileName: String): HashMap[String, RebaseInfo] = { val file = SparkClassUtils.getSparkClassLoader.getResource(fileName) val jsonRebaseRecords = mapper.readValue[Seq[JsonRebaseRecord]](file) - val anyRefMap = new AnyRefMap[String, RebaseInfo]((3 * jsonRebaseRecords.size) / 2) + val hashMap = new HashMap[String, RebaseInfo] + hashMap.sizeHint(jsonRebaseRecords.size) jsonRebaseRecords.foreach { jsonRecord => val rebaseInfo = RebaseInfo(jsonRecord.switches, jsonRecord.diffs) var i = 0 @@ -299,9 +299,9 @@ object RebaseDateTime { rebaseInfo.diffs(i) = rebaseInfo.diffs(i) * MICROS_PER_SECOND i += 1 } - anyRefMap.update(jsonRecord.tz, rebaseInfo) + hashMap.update(jsonRecord.tz, rebaseInfo) } - anyRefMap + hashMap } /** @@ -313,7 +313,7 @@ object RebaseDateTime { */ private val gregJulianRebaseMap = loadRebaseRecords("gregorian-julian-rebase-micros.json") - private def getLastSwitchTs(rebaseMap: AnyRefMap[String, RebaseInfo]): Long = { + private def getLastSwitchTs(rebaseMap: HashMap[String, RebaseInfo]): Long = { val latestTs = rebaseMap.values.map(_.switches.last).max require( rebaseMap.values.forall(_.diffs.last == 0), @@ -404,7 +404,7 @@ object RebaseDateTime { if (micros >= lastSwitchGregorianTs) { micros } else { - val rebaseRecord = gregJulianRebaseMap.getOrNull(timeZoneId) + val rebaseRecord = gregJulianRebaseMap.get(timeZoneId).orNull if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { rebaseGregorianToJulianMicros(TimeZone.getTimeZone(timeZoneId), micros) } else { @@ -526,7 +526,7 @@ object RebaseDateTime { if (micros >= lastSwitchJulianTs) { micros } else { - val rebaseRecord = julianGregRebaseMap.getOrNull(timeZoneId) + val rebaseRecord = julianGregRebaseMap.get(timeZoneId).orNull if (rebaseRecord == null || micros < rebaseRecord.switches(0)) { rebaseJulianToGregorianMicros(TimeZone.getTimeZone(timeZoneId), micros) } else { diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt index c15fb78f2f165..4b3fc3f4d4a1f 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-jdk21-results.txt @@ -2,153 +2,153 @@ Rebasing dates/timestamps in Parquet datasource ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 11193 11193 0 8.9 111.9 1.0X -before 1582, noop 7070 7070 0 14.1 70.7 1.6X -after 1582, rebase EXCEPTION 19836 19836 0 5.0 198.4 0.6X -after 1582, rebase LEGACY 19368 19368 0 5.2 193.7 0.6X -after 1582, rebase CORRECTED 19627 19627 0 5.1 196.3 0.6X -before 1582, rebase LEGACY 16301 16301 0 6.1 163.0 0.7X -before 1582, rebase CORRECTED 15612 15612 0 6.4 156.1 0.7X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1582, noop 10901 10901 0 9.2 109.0 1.0X +before 1582, noop 6649 6649 0 15.0 66.5 1.6X +after 1582, rebase EXCEPTION 20020 20020 0 5.0 200.2 0.5X +after 1582, rebase LEGACY 20257 20257 0 4.9 202.6 0.5X +after 1582, rebase CORRECTED 20050 20050 0 5.0 200.5 0.5X +before 1582, rebase LEGACY 16178 16178 0 6.2 161.8 0.7X +before 1582, rebase CORRECTED 16322 16322 0 6.1 163.2 0.7X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase EXCEPTION 11601 11657 95 8.6 116.0 1.0X -after 1582, vec off, rebase LEGACY 11671 11751 72 8.6 116.7 1.0X -after 1582, vec off, rebase CORRECTED 11593 11632 67 8.6 115.9 1.0X -after 1582, vec on, rebase EXCEPTION 2394 2427 33 41.8 23.9 4.8X -after 1582, vec on, rebase LEGACY 2466 2489 38 40.6 24.7 4.7X -after 1582, vec on, rebase CORRECTED 2487 2500 18 40.2 24.9 4.7X -before 1582, vec off, rebase LEGACY 11937 11951 14 8.4 119.4 1.0X -before 1582, vec off, rebase CORRECTED 11542 11600 60 8.7 115.4 1.0X -before 1582, vec on, rebase LEGACY 2708 2729 30 36.9 27.1 4.3X -before 1582, vec on, rebase CORRECTED 2436 2445 8 41.1 24.4 4.8X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1582, vec off, rebase EXCEPTION 11905 11978 106 8.4 119.1 1.0X +after 1582, vec off, rebase LEGACY 12069 12083 17 8.3 120.7 1.0X +after 1582, vec off, rebase CORRECTED 11835 11861 40 8.4 118.4 1.0X +after 1582, vec on, rebase EXCEPTION 2453 2492 38 40.8 24.5 4.9X +after 1582, vec on, rebase LEGACY 2471 2494 22 40.5 24.7 4.8X +after 1582, vec on, rebase CORRECTED 2386 2442 59 41.9 23.9 5.0X +before 1582, vec off, rebase LEGACY 12049 12076 27 8.3 120.5 1.0X +before 1582, vec off, rebase CORRECTED 11948 11999 48 8.4 119.5 1.0X +before 1582, vec on, rebase LEGACY 2787 2810 24 35.9 27.9 4.3X +before 1582, vec on, rebase CORRECTED 2443 2457 14 40.9 24.4 4.9X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2403 2403 0 41.6 24.0 1.0X -before 1900, noop 2443 2443 0 40.9 24.4 1.0X -after 1900, rebase EXCEPTION 12805 12805 0 7.8 128.1 0.2X -after 1900, rebase LEGACY 12529 12529 0 8.0 125.3 0.2X -after 1900, rebase CORRECTED 12474 12474 0 8.0 124.7 0.2X -before 1900, rebase LEGACY 14628 14628 0 6.8 146.3 0.2X -before 1900, rebase CORRECTED 12601 12601 0 7.9 126.0 0.2X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2429 2429 0 41.2 24.3 1.0X +before 1900, noop 2334 2334 0 42.8 23.3 1.0X +after 1900, rebase EXCEPTION 14358 14358 0 7.0 143.6 0.2X +after 1900, rebase LEGACY 14210 14210 0 7.0 142.1 0.2X +after 1900, rebase CORRECTED 14052 14052 0 7.1 140.5 0.2X +before 1900, rebase LEGACY 16255 16255 0 6.2 162.5 0.1X +before 1900, rebase CORRECTED 14010 14010 0 7.1 140.1 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 15545 15658 99 6.4 155.4 1.0X -after 1900, vec off, rebase LEGACY 15927 15945 25 6.3 159.3 1.0X -after 1900, vec off, rebase CORRECTED 15558 15620 54 6.4 155.6 1.0X -after 1900, vec on, rebase EXCEPTION 4050 4074 34 24.7 40.5 3.8X -after 1900, vec on, rebase LEGACY 4024 4059 32 24.9 40.2 3.9X -after 1900, vec on, rebase CORRECTED 4062 4074 17 24.6 40.6 3.8X -before 1900, vec off, rebase LEGACY 18219 18234 22 5.5 182.2 0.9X -before 1900, vec off, rebase CORRECTED 15584 15633 45 6.4 155.8 1.0X -before 1900, vec on, rebase LEGACY 6080 6106 23 16.4 60.8 2.6X -before 1900, vec on, rebase CORRECTED 4045 4057 14 24.7 40.4 3.8X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 16891 16906 13 5.9 168.9 1.0X +after 1900, vec off, rebase LEGACY 16737 16761 21 6.0 167.4 1.0X +after 1900, vec off, rebase CORRECTED 16555 16614 54 6.0 165.5 1.0X +after 1900, vec on, rebase EXCEPTION 4046 4094 54 24.7 40.5 4.2X +after 1900, vec on, rebase LEGACY 4057 4067 11 24.6 40.6 4.2X +after 1900, vec on, rebase CORRECTED 4063 4068 6 24.6 40.6 4.2X +before 1900, vec off, rebase LEGACY 18775 18838 56 5.3 187.7 0.9X +before 1900, vec off, rebase CORRECTED 16681 16713 36 6.0 166.8 1.0X +before 1900, vec on, rebase LEGACY 6105 6129 32 16.4 61.0 2.8X +before 1900, vec on, rebase CORRECTED 4046 4067 27 24.7 40.5 4.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2378 2378 0 42.0 23.8 1.0X -before 1900, noop 2426 2426 0 41.2 24.3 1.0X -after 1900, rebase EXCEPTION 14475 14475 0 6.9 144.8 0.2X -after 1900, rebase LEGACY 13685 13685 0 7.3 136.8 0.2X -after 1900, rebase CORRECTED 13448 13448 0 7.4 134.5 0.2X -before 1900, rebase LEGACY 15085 15085 0 6.6 150.8 0.2X -before 1900, rebase CORRECTED 13668 13668 0 7.3 136.7 0.2X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2415 2415 0 41.4 24.1 1.0X +before 1900, noop 2438 2438 0 41.0 24.4 1.0X +after 1900, rebase EXCEPTION 13703 13703 0 7.3 137.0 0.2X +after 1900, rebase LEGACY 13404 13404 0 7.5 134.0 0.2X +after 1900, rebase CORRECTED 13538 13538 0 7.4 135.4 0.2X +before 1900, rebase LEGACY 15245 15245 0 6.6 152.5 0.2X +before 1900, rebase CORRECTED 13242 13242 0 7.6 132.4 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 14595 14621 23 6.9 146.0 1.0X -after 1900, vec off, rebase LEGACY 14689 14699 12 6.8 146.9 1.0X -after 1900, vec off, rebase CORRECTED 14626 14648 25 6.8 146.3 1.0X -after 1900, vec on, rebase EXCEPTION 3732 3745 14 26.8 37.3 3.9X -after 1900, vec on, rebase LEGACY 3753 3771 29 26.6 37.5 3.9X -after 1900, vec on, rebase CORRECTED 3714 3734 23 26.9 37.1 3.9X -before 1900, vec off, rebase LEGACY 17073 17151 107 5.9 170.7 0.9X -before 1900, vec off, rebase CORRECTED 14575 14613 33 6.9 145.8 1.0X -before 1900, vec on, rebase LEGACY 5581 5602 34 17.9 55.8 2.6X -before 1900, vec on, rebase CORRECTED 3680 3698 30 27.2 36.8 4.0X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 14618 14680 54 6.8 146.2 1.0X +after 1900, vec off, rebase LEGACY 14578 14613 35 6.9 145.8 1.0X +after 1900, vec off, rebase CORRECTED 14621 14691 61 6.8 146.2 1.0X +after 1900, vec on, rebase EXCEPTION 3727 3738 16 26.8 37.3 3.9X +after 1900, vec on, rebase LEGACY 3715 3738 22 26.9 37.1 3.9X +after 1900, vec on, rebase CORRECTED 3705 3727 21 27.0 37.1 3.9X +before 1900, vec off, rebase LEGACY 16907 16943 32 5.9 169.1 0.9X +before 1900, vec off, rebase CORRECTED 14704 14711 6 6.8 147.0 1.0X +before 1900, vec on, rebase LEGACY 5588 5610 30 17.9 55.9 2.6X +before 1900, vec on, rebase CORRECTED 3730 3744 15 26.8 37.3 3.9X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2418 2418 0 41.4 24.2 1.0X -before 1900, noop 2413 2413 0 41.5 24.1 1.0X -after 1900, rebase EXCEPTION 11749 11749 0 8.5 117.5 0.2X -after 1900, rebase LEGACY 11757 11757 0 8.5 117.6 0.2X -after 1900, rebase CORRECTED 12081 12081 0 8.3 120.8 0.2X -before 1900, rebase LEGACY 13503 13503 0 7.4 135.0 0.2X -before 1900, rebase CORRECTED 11649 11649 0 8.6 116.5 0.2X - -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2391 2391 0 41.8 23.9 1.0X +before 1900, noop 2431 2431 0 41.1 24.3 1.0X +after 1900, rebase EXCEPTION 11856 11856 0 8.4 118.6 0.2X +after 1900, rebase LEGACY 11077 11077 0 9.0 110.8 0.2X +after 1900, rebase CORRECTED 11292 11292 0 8.9 112.9 0.2X +before 1900, rebase LEGACY 13101 13101 0 7.6 131.0 0.2X +before 1900, rebase CORRECTED 11333 11333 0 8.8 113.3 0.2X + +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 14322 14376 53 7.0 143.2 1.0X -after 1900, vec off, rebase LEGACY 14453 14478 22 6.9 144.5 1.0X -after 1900, vec off, rebase CORRECTED 14429 14523 94 6.9 144.3 1.0X -after 1900, vec on, rebase EXCEPTION 4876 4914 42 20.5 48.8 2.9X -after 1900, vec on, rebase LEGACY 4418 4465 41 22.6 44.2 3.2X -after 1900, vec on, rebase CORRECTED 4876 4909 50 20.5 48.8 2.9X -before 1900, vec off, rebase LEGACY 17196 17238 45 5.8 172.0 0.8X -before 1900, vec off, rebase CORRECTED 14462 14509 57 6.9 144.6 1.0X -before 1900, vec on, rebase LEGACY 6120 6135 16 16.3 61.2 2.3X -before 1900, vec on, rebase CORRECTED 4887 4929 42 20.5 48.9 2.9X +after 1900, vec off, rebase EXCEPTION 14732 14828 88 6.8 147.3 1.0X +after 1900, vec off, rebase LEGACY 15011 15061 58 6.7 150.1 1.0X +after 1900, vec off, rebase CORRECTED 14924 14978 86 6.7 149.2 1.0X +after 1900, vec on, rebase EXCEPTION 4935 4964 37 20.3 49.4 3.0X +after 1900, vec on, rebase LEGACY 4488 4509 19 22.3 44.9 3.3X +after 1900, vec on, rebase CORRECTED 4896 4934 33 20.4 49.0 3.0X +before 1900, vec off, rebase LEGACY 16917 17029 151 5.9 169.2 0.9X +before 1900, vec off, rebase CORRECTED 14924 14953 35 6.7 149.2 1.0X +before 1900, vec on, rebase LEGACY 6010 6023 12 16.6 60.1 2.5X +before 1900, vec on, rebase CORRECTED 4916 4939 25 20.3 49.2 3.0X ================================================================================================ Rebasing dates/timestamps in ORC datasource ================================================================================================ -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 12614 12614 0 7.9 126.1 1.0X -before 1582, noop 6620 6620 0 15.1 66.2 1.9X -after 1582 17066 17066 0 5.9 170.7 0.7X -before 1582 10573 10573 0 9.5 105.7 1.2X +after 1582, noop 12628 12628 0 7.9 126.3 1.0X +before 1582, noop 6710 6710 0 14.9 67.1 1.9X +after 1582 16956 16956 0 5.9 169.6 0.7X +before 1582 10840 10840 0 9.2 108.4 1.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 8422 8523 131 11.9 84.2 1.0X -after 1582, vec on 2386 2401 20 41.9 23.9 3.5X -before 1582, vec off 8447 8474 42 11.8 84.5 1.0X -before 1582, vec on 2526 2542 24 39.6 25.3 3.3X +after 1582, vec off 8521 8619 138 11.7 85.2 1.0X +after 1582, vec on 2414 2421 10 41.4 24.1 3.5X +before 1582, vec off 8621 8693 89 11.6 86.2 1.0X +before 1582, vec on 2555 2562 8 39.1 25.5 3.3X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2336 2336 0 42.8 23.4 1.0X -before 1900, noop 2309 2309 0 43.3 23.1 1.0X -after 1900 9646 9646 0 10.4 96.5 0.2X -before 1900 12150 12150 0 8.2 121.5 0.2X +after 1900, noop 2290 2290 0 43.7 22.9 1.0X +before 1900, noop 2313 2313 0 43.2 23.1 1.0X +after 1900 10097 10097 0 9.9 101.0 0.2X +before 1900 12223 12223 0 8.2 122.2 0.2X -OpenJDK 64-Bit Server VM 21.0.4+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 21.0.5+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 9781 9845 86 10.2 97.8 1.0X -after 1900, vec on 3778 3792 13 26.5 37.8 2.6X -before 1900, vec off 11757 11781 21 8.5 117.6 0.8X -before 1900, vec on 5490 5511 21 18.2 54.9 1.8X +after 1900, vec off 9908 9927 29 10.1 99.1 1.0X +after 1900, vec on 3868 3881 18 25.9 38.7 2.6X +before 1900, vec off 11663 11671 7 8.6 116.6 0.8X +before 1900, vec on 5396 5408 10 18.5 54.0 1.8X diff --git a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt index 249b478e772a8..980897bc96b9b 100644 --- a/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt +++ b/sql/core/benchmarks/DateTimeRebaseBenchmark-results.txt @@ -2,153 +2,153 @@ Rebasing dates/timestamps in Parquet datasource ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save DATE to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 11170 11170 0 9.0 111.7 1.0X -before 1582, noop 6505 6505 0 15.4 65.0 1.7X -after 1582, rebase EXCEPTION 19873 19873 0 5.0 198.7 0.6X -after 1582, rebase LEGACY 19726 19726 0 5.1 197.3 0.6X -after 1582, rebase CORRECTED 19931 19931 0 5.0 199.3 0.6X -before 1582, rebase LEGACY 15590 15590 0 6.4 155.9 0.7X -before 1582, rebase CORRECTED 15523 15523 0 6.4 155.2 0.7X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1582, noop 13229 13229 0 7.6 132.3 1.0X +before 1582, noop 7736 7736 0 12.9 77.4 1.7X +after 1582, rebase EXCEPTION 21925 21925 0 4.6 219.2 0.6X +after 1582, rebase LEGACY 21714 21714 0 4.6 217.1 0.6X +after 1582, rebase CORRECTED 21864 21864 0 4.6 218.6 0.6X +before 1582, rebase LEGACY 16338 16338 0 6.1 163.4 0.8X +before 1582, rebase CORRECTED 16312 16312 0 6.1 163.1 0.8X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load DATE from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off, rebase EXCEPTION 11571 11596 22 8.6 115.7 1.0X -after 1582, vec off, rebase LEGACY 11892 11909 27 8.4 118.9 1.0X -after 1582, vec off, rebase CORRECTED 11681 11724 47 8.6 116.8 1.0X -after 1582, vec on, rebase EXCEPTION 2516 2530 13 39.7 25.2 4.6X -after 1582, vec on, rebase LEGACY 2555 2563 8 39.1 25.5 4.5X -after 1582, vec on, rebase CORRECTED 2487 2503 22 40.2 24.9 4.7X -before 1582, vec off, rebase LEGACY 11947 11996 69 8.4 119.5 1.0X -before 1582, vec off, rebase CORRECTED 11792 11821 41 8.5 117.9 1.0X -before 1582, vec on, rebase LEGACY 2826 2856 25 35.4 28.3 4.1X -before 1582, vec on, rebase CORRECTED 2465 2489 21 40.6 24.6 4.7X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1582, vec off, rebase EXCEPTION 11106 11144 39 9.0 111.1 1.0X +after 1582, vec off, rebase LEGACY 11476 11510 49 8.7 114.8 1.0X +after 1582, vec off, rebase CORRECTED 11373 11409 62 8.8 113.7 1.0X +after 1582, vec on, rebase EXCEPTION 2522 2548 31 39.7 25.2 4.4X +after 1582, vec on, rebase LEGACY 2628 2642 15 38.1 26.3 4.2X +after 1582, vec on, rebase CORRECTED 2490 2547 61 40.2 24.9 4.5X +before 1582, vec off, rebase LEGACY 11671 11743 88 8.6 116.7 1.0X +before 1582, vec off, rebase CORRECTED 11350 11379 41 8.8 113.5 1.0X +before 1582, vec on, rebase LEGACY 2855 2857 2 35.0 28.6 3.9X +before 1582, vec on, rebase CORRECTED 2559 2570 16 39.1 25.6 4.3X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_INT96 to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2340 2340 0 42.7 23.4 1.0X -before 1900, noop 2284 2284 0 43.8 22.8 1.0X -after 1900, rebase EXCEPTION 13230 13230 0 7.6 132.3 0.2X -after 1900, rebase LEGACY 13238 13238 0 7.6 132.4 0.2X -after 1900, rebase CORRECTED 13264 13264 0 7.5 132.6 0.2X -before 1900, rebase LEGACY 15216 15216 0 6.6 152.2 0.2X -before 1900, rebase CORRECTED 13382 13382 0 7.5 133.8 0.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2239 2239 0 44.7 22.4 1.0X +before 1900, noop 2229 2229 0 44.9 22.3 1.0X +after 1900, rebase EXCEPTION 13150 13150 0 7.6 131.5 0.2X +after 1900, rebase LEGACY 13004 13004 0 7.7 130.0 0.2X +after 1900, rebase CORRECTED 13216 13216 0 7.6 132.2 0.2X +before 1900, rebase LEGACY 14965 14965 0 6.7 149.6 0.1X +before 1900, rebase CORRECTED 13218 13218 0 7.6 132.2 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_INT96 from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 16170 16195 22 6.2 161.7 1.0X -after 1900, vec off, rebase LEGACY 16720 16755 31 6.0 167.2 1.0X -after 1900, vec off, rebase CORRECTED 16152 16213 54 6.2 161.5 1.0X -after 1900, vec on, rebase EXCEPTION 4090 4101 18 24.4 40.9 4.0X -after 1900, vec on, rebase LEGACY 4114 4144 33 24.3 41.1 3.9X -after 1900, vec on, rebase CORRECTED 4158 4191 28 24.0 41.6 3.9X -before 1900, vec off, rebase LEGACY 18554 18584 31 5.4 185.5 0.9X -before 1900, vec off, rebase CORRECTED 16192 16267 84 6.2 161.9 1.0X -before 1900, vec on, rebase LEGACY 6256 6271 22 16.0 62.6 2.6X -before 1900, vec on, rebase CORRECTED 4074 4104 27 24.5 40.7 4.0X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 17738 17763 26 5.6 177.4 1.0X +after 1900, vec off, rebase LEGACY 16932 16966 49 5.9 169.3 1.0X +after 1900, vec off, rebase CORRECTED 16738 16751 12 6.0 167.4 1.1X +after 1900, vec on, rebase EXCEPTION 4046 4072 34 24.7 40.5 4.4X +after 1900, vec on, rebase LEGACY 4081 4090 11 24.5 40.8 4.3X +after 1900, vec on, rebase CORRECTED 4015 4046 32 24.9 40.1 4.4X +before 1900, vec off, rebase LEGACY 19319 19369 43 5.2 193.2 0.9X +before 1900, vec off, rebase CORRECTED 16460 16479 19 6.1 164.6 1.1X +before 1900, vec on, rebase LEGACY 5956 5960 5 16.8 59.6 3.0X +before 1900, vec on, rebase CORRECTED 4036 4086 83 24.8 40.4 4.4X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MICROS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2390 2390 0 41.8 23.9 1.0X -before 1900, noop 2291 2291 0 43.6 22.9 1.0X -after 1900, rebase EXCEPTION 12537 12537 0 8.0 125.4 0.2X -after 1900, rebase LEGACY 12047 12047 0 8.3 120.5 0.2X -after 1900, rebase CORRECTED 12151 12151 0 8.2 121.5 0.2X -before 1900, rebase LEGACY 13960 13960 0 7.2 139.6 0.2X -before 1900, rebase CORRECTED 11985 11985 0 8.3 119.9 0.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2262 2262 0 44.2 22.6 1.0X +before 1900, noop 2253 2253 0 44.4 22.5 1.0X +after 1900, rebase EXCEPTION 11901 11901 0 8.4 119.0 0.2X +after 1900, rebase LEGACY 11564 11564 0 8.6 115.6 0.2X +after 1900, rebase CORRECTED 11804 11804 0 8.5 118.0 0.2X +before 1900, rebase LEGACY 13264 13264 0 7.5 132.6 0.2X +before 1900, rebase CORRECTED 11816 11816 0 8.5 118.2 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MICROS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 14977 15018 36 6.7 149.8 1.0X -after 1900, vec off, rebase LEGACY 14924 14960 33 6.7 149.2 1.0X -after 1900, vec off, rebase CORRECTED 14965 14994 26 6.7 149.7 1.0X -after 1900, vec on, rebase EXCEPTION 3810 3819 8 26.2 38.1 3.9X -after 1900, vec on, rebase LEGACY 3829 3835 8 26.1 38.3 3.9X -after 1900, vec on, rebase CORRECTED 3785 3837 47 26.4 37.9 4.0X -before 1900, vec off, rebase LEGACY 17323 17343 19 5.8 173.2 0.9X -before 1900, vec off, rebase CORRECTED 14933 14962 26 6.7 149.3 1.0X -before 1900, vec on, rebase LEGACY 5763 5783 17 17.4 57.6 2.6X -before 1900, vec on, rebase CORRECTED 3798 3817 32 26.3 38.0 3.9X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1900, vec off, rebase EXCEPTION 15095 15151 58 6.6 150.9 1.0X +after 1900, vec off, rebase LEGACY 15243 15292 47 6.6 152.4 1.0X +after 1900, vec off, rebase CORRECTED 15200 15211 11 6.6 152.0 1.0X +after 1900, vec on, rebase EXCEPTION 3758 3772 14 26.6 37.6 4.0X +after 1900, vec on, rebase LEGACY 3775 3811 39 26.5 37.8 4.0X +after 1900, vec on, rebase CORRECTED 3784 3796 16 26.4 37.8 4.0X +before 1900, vec off, rebase LEGACY 17548 17583 41 5.7 175.5 0.9X +before 1900, vec off, rebase CORRECTED 15155 15166 10 6.6 151.6 1.0X +before 1900, vec on, rebase LEGACY 5640 5665 43 17.7 56.4 2.7X +before 1900, vec on, rebase CORRECTED 3803 3856 64 26.3 38.0 4.0X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP_MILLIS to parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2309 2309 0 43.3 23.1 1.0X -before 1900, noop 2358 2358 0 42.4 23.6 1.0X -after 1900, rebase EXCEPTION 11266 11266 0 8.9 112.7 0.2X -after 1900, rebase LEGACY 11582 11582 0 8.6 115.8 0.2X -after 1900, rebase CORRECTED 11555 11555 0 8.7 115.5 0.2X -before 1900, rebase LEGACY 13600 13600 0 7.4 136.0 0.2X -before 1900, rebase CORRECTED 12113 12113 0 8.3 121.1 0.2X - -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +after 1900, noop 2253 2253 0 44.4 22.5 1.0X +before 1900, noop 2253 2253 0 44.4 22.5 1.0X +after 1900, rebase EXCEPTION 11471 11471 0 8.7 114.7 0.2X +after 1900, rebase LEGACY 11173 11173 0 8.9 111.7 0.2X +after 1900, rebase CORRECTED 11497 11497 0 8.7 115.0 0.2X +before 1900, rebase LEGACY 13368 13368 0 7.5 133.7 0.2X +before 1900, rebase CORRECTED 11570 11570 0 8.6 115.7 0.2X + +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP_MILLIS from parquet: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off, rebase EXCEPTION 15076 15125 46 6.6 150.8 1.0X -after 1900, vec off, rebase LEGACY 15480 15491 16 6.5 154.8 1.0X -after 1900, vec off, rebase CORRECTED 15171 15189 17 6.6 151.7 1.0X -after 1900, vec on, rebase EXCEPTION 3976 4001 28 25.2 39.8 3.8X -after 1900, vec on, rebase LEGACY 4582 4609 46 21.8 45.8 3.3X -after 1900, vec on, rebase CORRECTED 3934 3953 29 25.4 39.3 3.8X -before 1900, vec off, rebase LEGACY 17602 17644 37 5.7 176.0 0.9X -before 1900, vec off, rebase CORRECTED 15201 15238 34 6.6 152.0 1.0X -before 1900, vec on, rebase LEGACY 6306 6311 6 15.9 63.1 2.4X -before 1900, vec on, rebase CORRECTED 3926 3961 50 25.5 39.3 3.8X +after 1900, vec off, rebase EXCEPTION 15315 15363 82 6.5 153.1 1.0X +after 1900, vec off, rebase LEGACY 15462 15506 70 6.5 154.6 1.0X +after 1900, vec off, rebase CORRECTED 15394 15426 38 6.5 153.9 1.0X +after 1900, vec on, rebase EXCEPTION 3910 3941 28 25.6 39.1 3.9X +after 1900, vec on, rebase LEGACY 4503 4516 22 22.2 45.0 3.4X +after 1900, vec on, rebase CORRECTED 3915 3919 5 25.5 39.1 3.9X +before 1900, vec off, rebase LEGACY 17595 17636 46 5.7 176.0 0.9X +before 1900, vec off, rebase CORRECTED 15344 15354 17 6.5 153.4 1.0X +before 1900, vec on, rebase LEGACY 6188 6202 23 16.2 61.9 2.5X +before 1900, vec on, rebase CORRECTED 3890 3899 8 25.7 38.9 3.9X ================================================================================================ Rebasing dates/timestamps in ORC datasource ================================================================================================ -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save DATE to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, noop 11208 11208 0 8.9 112.1 1.0X -before 1582, noop 6567 6567 0 15.2 65.7 1.7X -after 1582 15130 15130 0 6.6 151.3 0.7X -before 1582 10992 10992 0 9.1 109.9 1.0X +after 1582, noop 13342 13342 0 7.5 133.4 1.0X +before 1582, noop 7875 7875 0 12.7 78.7 1.7X +after 1582 16884 16884 0 5.9 168.8 0.8X +before 1582 11948 11948 0 8.4 119.5 1.1X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load DATE from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1582, vec off 8770 8777 8 11.4 87.7 1.0X -after 1582, vec on 2445 2478 31 40.9 24.5 3.6X -before 1582, vec off 8820 8896 106 11.3 88.2 1.0X -before 1582, vec on 2580 2615 37 38.8 25.8 3.4X +after 1582, vec off 8890 8914 29 11.2 88.9 1.0X +after 1582, vec on 2459 2499 64 40.7 24.6 3.6X +before 1582, vec off 8453 8462 8 11.8 84.5 1.1X +before 1582, vec on 2618 2636 16 38.2 26.2 3.4X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Save TIMESTAMP to ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, noop 2221 2221 0 45.0 22.2 1.0X -before 1900, noop 2218 2218 0 45.1 22.2 1.0X -after 1900 9916 9916 0 10.1 99.2 0.2X -before 1900 12130 12130 0 8.2 121.3 0.2X +after 1900, noop 2181 2181 0 45.9 21.8 1.0X +before 1900, noop 2161 2161 0 46.3 21.6 1.0X +after 1900 10953 10953 0 9.1 109.5 0.2X +before 1900 12293 12293 0 8.1 122.9 0.2X -OpenJDK 64-Bit Server VM 17.0.12+7-LTS on Linux 6.5.0-1025-azure +OpenJDK 64-Bit Server VM 17.0.13+11-LTS on Linux 6.5.0-1025-azure AMD EPYC 7763 64-Core Processor Load TIMESTAMP from ORC: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -after 1900, vec off 10569 11038 802 9.5 105.7 1.0X -after 1900, vec on 4361 4415 55 22.9 43.6 2.4X -before 1900, vec off 12223 12227 7 8.2 122.2 0.9X -before 1900, vec on 6103 6136 30 16.4 61.0 1.7X +after 1900, vec off 10398 10414 25 9.6 104.0 1.0X +after 1900, vec on 4165 4173 7 24.0 41.7 2.5X +before 1900, vec off 12138 12211 112 8.2 121.4 0.9X +before 1900, vec on 5781 5784 4 17.3 57.8 1.8X From 3f333a0820a991a7642632a49e430843840b75ee Mon Sep 17 00:00:00 2001 From: huanliwang-db Date: Thu, 2 Jan 2025 12:01:47 +0900 Subject: [PATCH 14/60] [SPARK-50642][CONNECT][SS] Fix the state schema for FlatMapGroupsWithState in spark connect when there is no initial state In spark connect, when there is no initial state, we derived the state schema from the input: create the initialDs from the original input: https://github.com/apache/spark/blob/master/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala#L679-L689 derived the state expression encoder from this `initialDs` which is incorrect: https://github.com/apache/spark/blob/master/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala#L729 Our unit tests fail to cover this case because it doesn't do the state update: https://github.com/apache/spark/blob/master/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateStreamingSuite.scala#L55-L59 after changing the `stateFunc` to the following ``` val stateFunc = (key: String, values: Iterator[ClickEvent], state: GroupState[ClickState]) => { if (state.exists) throw new IllegalArgumentException("state.exists should be false") val newState = ClickState(key, values.size) state.update(newState) Iterator(newState) } ``` the test is actually failing with ``` Cause: org.apache.spark.SparkException: Job aborted due to stage failure: Task 122 in stage 2.0 failed 1 times, most recent failure: Lost task 122.0 in stage 2.0 (TID 12) (192.168.68.84 executor driver): java.lang.ClassCastException: class org.apache.spark.sql.streaming.ClickState cannot be cast to class org.apache.spark.sql.streaming.ClickEvent (org.apache.spark.sql.streaming.ClickState and org.apache.spark.sql.streaming.ClickEvent are in unnamed module of loader 'app') at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificUnsafeProjection.CreateNamedStruct_0$(Unknown Source) ``` ### What changes were proposed in this pull request? * introduce a new `state_schema` proto field * pass the state agnostic encoder to the serialized udf * pass the state schema to query proto for spark connect * rebuild the state expression encoder based on the state agnostic encoder and state schema. ### Why are the changes needed? fix the broken behavior for flatMapGroupsWithState on spark connect ### Does this PR introduce _any_ user-facing change? yes ### How was this patch tested? modified the existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? no Closes #49260 from huanliwang-db/huanliwang-db/fmgws-client. Lead-authored-by: huanliwang-db Co-authored-by: Huanli Wang Signed-off-by: Hyukjin Kwon --- .../spark/sql/KeyValueGroupedDataset.scala | 21 +++++++- ...FlatMapGroupsWithStateStreamingSuite.scala | 16 +++++-- .../sql/connect/proto/relations_pb2.py | 48 +++++++++---------- .../sql/connect/proto/relations_pb2.pyi | 17 +++++++ .../protobuf/spark/connect/relations.proto | 3 ++ .../connect/planner/SparkConnectPlanner.scala | 31 +++++++++--- 6 files changed, 99 insertions(+), 37 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 63b5f27c4745e..d5505d2222c4f 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -27,7 +27,7 @@ import org.apache.spark.connect.proto import org.apache.spark.sql.catalyst.encoders.AgnosticEncoder import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.{agnosticEncoderFor, ProductEncoder} import org.apache.spark.sql.connect.ConnectConversions._ -import org.apache.spark.sql.connect.common.UdfUtils +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, UdfUtils} import org.apache.spark.sql.expressions.SparkUserDefinedFunction import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.ColumnNodeToProtoConverter.toExpr @@ -502,6 +502,7 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( } val outputEncoder = agnosticEncoderFor[U] + val stateEncoder = agnosticEncoderFor[S] val nf = UDFAdaptors.flatMapGroupsWithStateWithMappedValues(func, valueMapFunc) sparkSession.newDataset[U](outputEncoder) { builder => @@ -509,11 +510,12 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( groupMapBuilder .setInput(plan.getRoot) .addAllGroupingExpressions(groupingExprs) - .setFunc(getUdf(nf, outputEncoder)(ivEncoder)) + .setFunc(getUdf(nf, outputEncoder, stateEncoder)(ivEncoder)) .setIsMapGroupsWithState(isMapGroupWithState) .setOutputMode(if (outputMode.isEmpty) OutputMode.Update.toString else outputMode.get.toString) .setTimeoutConf(timeoutConf.toString) + .setStateSchema(DataTypeProtoConverter.toConnectProtoType(stateEncoder.schema)) if (initialStateImpl != null) { groupMapBuilder @@ -533,6 +535,21 @@ private class KeyValueGroupedDatasetImpl[K, V, IK, IV]( udf.apply(inputEncoders.map(_ => col("*")): _*).expr.getCommonInlineUserDefinedFunction } + private def getUdf[U: Encoder, S: Encoder]( + nf: AnyRef, + outputEncoder: AgnosticEncoder[U], + stateEncoder: AgnosticEncoder[S])( + inEncoders: AgnosticEncoder[_]*): proto.CommonInlineUserDefinedFunction = { + // Apply keyAs changes by setting kEncoder + // Add the state encoder to the inputEncoders. + val inputEncoders = kEncoder +: stateEncoder +: inEncoders + val udf = SparkUserDefinedFunction( + function = nf, + inputEncoders = inputEncoders, + outputEncoder = outputEncoder) + udf.apply(inputEncoders.map(_ => col("*")): _*).expr.getCommonInlineUserDefinedFunction + } + /** * We cannot deserialize a connect [[KeyValueGroupedDataset]] because of a class clash on the * server side. We null out the instance for now. diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateStreamingSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateStreamingSuite.scala index dc74463f1a25b..9bd6614028cbf 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateStreamingSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/streaming/FlatMapGroupsWithStateStreamingSuite.scala @@ -55,7 +55,9 @@ class FlatMapGroupsWithStateStreamingSuite extends QueryTest with RemoteSparkSes val stateFunc = (key: String, values: Iterator[ClickEvent], state: GroupState[ClickState]) => { if (state.exists) throw new IllegalArgumentException("state.exists should be false") - Iterator(ClickState(key, values.size)) + val newState = ClickState(key, values.size) + state.update(newState) + Iterator(newState) } spark.sql("DROP TABLE IF EXISTS my_sink") @@ -96,7 +98,9 @@ class FlatMapGroupsWithStateStreamingSuite extends QueryTest with RemoteSparkSes val stateFunc = (key: String, values: Iterator[ClickEvent], state: GroupState[ClickState]) => { val currState = state.getOption.getOrElse(ClickState(key, 0)) - Iterator(ClickState(key, currState.count + values.size)) + val newState = ClickState(key, currState.count + values.size) + state.update(newState) + Iterator(newState) } val initialState = flatMapGroupsWithStateInitialStateData .toDS() @@ -141,7 +145,9 @@ class FlatMapGroupsWithStateStreamingSuite extends QueryTest with RemoteSparkSes val stateFunc = (key: String, values: Iterator[ClickEvent], state: GroupState[ClickState]) => { if (state.exists) throw new IllegalArgumentException("state.exists should be false") - ClickState(key, values.size) + val newState = ClickState(key, values.size) + state.update(newState) + newState } spark.sql("DROP TABLE IF EXISTS my_sink") @@ -183,7 +189,9 @@ class FlatMapGroupsWithStateStreamingSuite extends QueryTest with RemoteSparkSes val stateFunc = (key: String, values: Iterator[ClickEvent], state: GroupState[ClickState]) => { val currState = state.getOption.getOrElse(ClickState(key, 0)) - ClickState(key, currState.count + values.size) + val newState = ClickState(key, currState.count + values.size) + state.update(newState) + newState } val initialState = flatMapGroupsWithStateInitialStateData .toDS() diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 506b266f60145..b7248d4b17080 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -42,7 +42,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xdd\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\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"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\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\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\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\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xfb\x04\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_conf"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\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<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\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\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/relations.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1fspark/connect/expressions.proto\x1a\x19spark/connect/types.proto\x1a\x1bspark/connect/catalog.proto\x1a\x1aspark/connect/common.proto"\xdd\x1c\n\x08Relation\x12\x35\n\x06\x63ommon\x18\x01 \x01(\x0b\x32\x1d.spark.connect.RelationCommonR\x06\x63ommon\x12)\n\x04read\x18\x02 \x01(\x0b\x32\x13.spark.connect.ReadH\x00R\x04read\x12\x32\n\x07project\x18\x03 \x01(\x0b\x32\x16.spark.connect.ProjectH\x00R\x07project\x12/\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x15.spark.connect.FilterH\x00R\x06\x66ilter\x12)\n\x04join\x18\x05 \x01(\x0b\x32\x13.spark.connect.JoinH\x00R\x04join\x12\x34\n\x06set_op\x18\x06 \x01(\x0b\x32\x1b.spark.connect.SetOperationH\x00R\x05setOp\x12)\n\x04sort\x18\x07 \x01(\x0b\x32\x13.spark.connect.SortH\x00R\x04sort\x12,\n\x05limit\x18\x08 \x01(\x0b\x32\x14.spark.connect.LimitH\x00R\x05limit\x12\x38\n\taggregate\x18\t \x01(\x0b\x32\x18.spark.connect.AggregateH\x00R\taggregate\x12&\n\x03sql\x18\n \x01(\x0b\x32\x12.spark.connect.SQLH\x00R\x03sql\x12\x45\n\x0elocal_relation\x18\x0b \x01(\x0b\x32\x1c.spark.connect.LocalRelationH\x00R\rlocalRelation\x12/\n\x06sample\x18\x0c \x01(\x0b\x32\x15.spark.connect.SampleH\x00R\x06sample\x12/\n\x06offset\x18\r \x01(\x0b\x32\x15.spark.connect.OffsetH\x00R\x06offset\x12>\n\x0b\x64\x65\x64uplicate\x18\x0e \x01(\x0b\x32\x1a.spark.connect.DeduplicateH\x00R\x0b\x64\x65\x64uplicate\x12,\n\x05range\x18\x0f \x01(\x0b\x32\x14.spark.connect.RangeH\x00R\x05range\x12\x45\n\x0esubquery_alias\x18\x10 \x01(\x0b\x32\x1c.spark.connect.SubqueryAliasH\x00R\rsubqueryAlias\x12>\n\x0brepartition\x18\x11 \x01(\x0b\x32\x1a.spark.connect.RepartitionH\x00R\x0brepartition\x12*\n\x05to_df\x18\x12 \x01(\x0b\x32\x13.spark.connect.ToDFH\x00R\x04toDf\x12U\n\x14with_columns_renamed\x18\x13 \x01(\x0b\x32!.spark.connect.WithColumnsRenamedH\x00R\x12withColumnsRenamed\x12<\n\x0bshow_string\x18\x14 \x01(\x0b\x32\x19.spark.connect.ShowStringH\x00R\nshowString\x12)\n\x04\x64rop\x18\x15 \x01(\x0b\x32\x13.spark.connect.DropH\x00R\x04\x64rop\x12)\n\x04tail\x18\x16 \x01(\x0b\x32\x13.spark.connect.TailH\x00R\x04tail\x12?\n\x0cwith_columns\x18\x17 \x01(\x0b\x32\x1a.spark.connect.WithColumnsH\x00R\x0bwithColumns\x12)\n\x04hint\x18\x18 \x01(\x0b\x32\x13.spark.connect.HintH\x00R\x04hint\x12\x32\n\x07unpivot\x18\x19 \x01(\x0b\x32\x16.spark.connect.UnpivotH\x00R\x07unpivot\x12\x36\n\tto_schema\x18\x1a \x01(\x0b\x32\x17.spark.connect.ToSchemaH\x00R\x08toSchema\x12\x64\n\x19repartition_by_expression\x18\x1b \x01(\x0b\x32&.spark.connect.RepartitionByExpressionH\x00R\x17repartitionByExpression\x12\x45\n\x0emap_partitions\x18\x1c \x01(\x0b\x32\x1c.spark.connect.MapPartitionsH\x00R\rmapPartitions\x12H\n\x0f\x63ollect_metrics\x18\x1d \x01(\x0b\x32\x1d.spark.connect.CollectMetricsH\x00R\x0e\x63ollectMetrics\x12,\n\x05parse\x18\x1e \x01(\x0b\x32\x14.spark.connect.ParseH\x00R\x05parse\x12\x36\n\tgroup_map\x18\x1f \x01(\x0b\x32\x17.spark.connect.GroupMapH\x00R\x08groupMap\x12=\n\x0c\x63o_group_map\x18 \x01(\x0b\x32\x19.spark.connect.CoGroupMapH\x00R\ncoGroupMap\x12\x45\n\x0ewith_watermark\x18! \x01(\x0b\x32\x1c.spark.connect.WithWatermarkH\x00R\rwithWatermark\x12\x63\n\x1a\x61pply_in_pandas_with_state\x18" \x01(\x0b\x32%.spark.connect.ApplyInPandasWithStateH\x00R\x16\x61pplyInPandasWithState\x12<\n\x0bhtml_string\x18# \x01(\x0b\x32\x19.spark.connect.HtmlStringH\x00R\nhtmlString\x12X\n\x15\x63\x61\x63hed_local_relation\x18$ \x01(\x0b\x32".spark.connect.CachedLocalRelationH\x00R\x13\x63\x61\x63hedLocalRelation\x12[\n\x16\x63\x61\x63hed_remote_relation\x18% \x01(\x0b\x32#.spark.connect.CachedRemoteRelationH\x00R\x14\x63\x61\x63hedRemoteRelation\x12\x8e\x01\n)common_inline_user_defined_table_function\x18& \x01(\x0b\x32\x33.spark.connect.CommonInlineUserDefinedTableFunctionH\x00R$commonInlineUserDefinedTableFunction\x12\x37\n\nas_of_join\x18\' \x01(\x0b\x32\x17.spark.connect.AsOfJoinH\x00R\x08\x61sOfJoin\x12\x85\x01\n&common_inline_user_defined_data_source\x18( \x01(\x0b\x32\x30.spark.connect.CommonInlineUserDefinedDataSourceH\x00R!commonInlineUserDefinedDataSource\x12\x45\n\x0ewith_relations\x18) \x01(\x0b\x32\x1c.spark.connect.WithRelationsH\x00R\rwithRelations\x12\x38\n\ttranspose\x18* \x01(\x0b\x32\x18.spark.connect.TransposeH\x00R\ttranspose\x12w\n unresolved_table_valued_function\x18+ \x01(\x0b\x32,.spark.connect.UnresolvedTableValuedFunctionH\x00R\x1dunresolvedTableValuedFunction\x12?\n\x0clateral_join\x18, \x01(\x0b\x32\x1a.spark.connect.LateralJoinH\x00R\x0blateralJoin\x12\x30\n\x07\x66ill_na\x18Z \x01(\x0b\x32\x15.spark.connect.NAFillH\x00R\x06\x66illNa\x12\x30\n\x07\x64rop_na\x18[ \x01(\x0b\x32\x15.spark.connect.NADropH\x00R\x06\x64ropNa\x12\x34\n\x07replace\x18\\ \x01(\x0b\x32\x18.spark.connect.NAReplaceH\x00R\x07replace\x12\x36\n\x07summary\x18\x64 \x01(\x0b\x32\x1a.spark.connect.StatSummaryH\x00R\x07summary\x12\x39\n\x08\x63rosstab\x18\x65 \x01(\x0b\x32\x1b.spark.connect.StatCrosstabH\x00R\x08\x63rosstab\x12\x39\n\x08\x64\x65scribe\x18\x66 \x01(\x0b\x32\x1b.spark.connect.StatDescribeH\x00R\x08\x64\x65scribe\x12*\n\x03\x63ov\x18g \x01(\x0b\x32\x16.spark.connect.StatCovH\x00R\x03\x63ov\x12-\n\x04\x63orr\x18h \x01(\x0b\x32\x17.spark.connect.StatCorrH\x00R\x04\x63orr\x12L\n\x0f\x61pprox_quantile\x18i \x01(\x0b\x32!.spark.connect.StatApproxQuantileH\x00R\x0e\x61pproxQuantile\x12=\n\nfreq_items\x18j \x01(\x0b\x32\x1c.spark.connect.StatFreqItemsH\x00R\tfreqItems\x12:\n\tsample_by\x18k \x01(\x0b\x32\x1b.spark.connect.StatSampleByH\x00R\x08sampleBy\x12\x33\n\x07\x63\x61talog\x18\xc8\x01 \x01(\x0b\x32\x16.spark.connect.CatalogH\x00R\x07\x63\x61talog\x12\x35\n\textension\x18\xe6\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x33\n\x07unknown\x18\xe7\x07 \x01(\x0b\x32\x16.spark.connect.UnknownH\x00R\x07unknownB\n\n\x08rel_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\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"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\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\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\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\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\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<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\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\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -208,29 +208,29 @@ _globals["_MAPPARTITIONS"]._serialized_start = 12857 _globals["_MAPPARTITIONS"]._serialized_end = 13089 _globals["_GROUPMAP"]._serialized_start = 13092 - _globals["_GROUPMAP"]._serialized_end = 13727 - _globals["_COGROUPMAP"]._serialized_start = 13730 - _globals["_COGROUPMAP"]._serialized_end = 14256 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14259 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14616 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14619 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14863 - _globals["_PYTHONUDTF"]._serialized_start = 14866 - _globals["_PYTHONUDTF"]._serialized_end = 15043 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15046 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15197 - _globals["_PYTHONDATASOURCE"]._serialized_start = 15199 - _globals["_PYTHONDATASOURCE"]._serialized_end = 15274 - _globals["_COLLECTMETRICS"]._serialized_start = 15277 - _globals["_COLLECTMETRICS"]._serialized_end = 15413 - _globals["_PARSE"]._serialized_start = 15416 - _globals["_PARSE"]._serialized_end = 15804 + _globals["_GROUPMAP"]._serialized_end = 13809 + _globals["_COGROUPMAP"]._serialized_start = 13812 + _globals["_COGROUPMAP"]._serialized_end = 14338 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14341 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14698 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14701 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14945 + _globals["_PYTHONUDTF"]._serialized_start = 14948 + _globals["_PYTHONUDTF"]._serialized_end = 15125 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15128 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15279 + _globals["_PYTHONDATASOURCE"]._serialized_start = 15281 + _globals["_PYTHONDATASOURCE"]._serialized_end = 15356 + _globals["_COLLECTMETRICS"]._serialized_start = 15359 + _globals["_COLLECTMETRICS"]._serialized_end = 15495 + _globals["_PARSE"]._serialized_start = 15498 + _globals["_PARSE"]._serialized_end = 15886 _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4941 _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4999 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15705 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15793 - _globals["_ASOFJOIN"]._serialized_start = 15807 - _globals["_ASOFJOIN"]._serialized_end = 16282 - _globals["_LATERALJOIN"]._serialized_start = 16285 - _globals["_LATERALJOIN"]._serialized_end = 16515 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15787 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15875 + _globals["_ASOFJOIN"]._serialized_start = 15889 + _globals["_ASOFJOIN"]._serialized_end = 16364 + _globals["_LATERALJOIN"]._serialized_start = 16367 + _globals["_LATERALJOIN"]._serialized_end = 16597 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/relations_pb2.pyi b/python/pyspark/sql/connect/proto/relations_pb2.pyi index e5a6bff9e430a..371d735b9e871 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -3409,6 +3409,7 @@ class GroupMap(google.protobuf.message.Message): IS_MAP_GROUPS_WITH_STATE_FIELD_NUMBER: builtins.int OUTPUT_MODE_FIELD_NUMBER: builtins.int TIMEOUT_CONF_FIELD_NUMBER: builtins.int + STATE_SCHEMA_FIELD_NUMBER: builtins.int @property def input(self) -> global___Relation: """(Required) Input relation for Group Map API: apply, applyInPandas.""" @@ -3447,6 +3448,9 @@ class GroupMap(google.protobuf.message.Message): """(Optional) The output mode of the function.""" timeout_conf: builtins.str """(Optional) Timeout configuration for groups that do not receive data for a while.""" + @property + def state_schema(self) -> pyspark.sql.connect.proto.types_pb2.DataType: + """(Optional) The schema for the grouped state.""" def __init__( self, *, @@ -3469,6 +3473,7 @@ class GroupMap(google.protobuf.message.Message): is_map_groups_with_state: builtins.bool | None = ..., output_mode: builtins.str | None = ..., timeout_conf: builtins.str | None = ..., + state_schema: pyspark.sql.connect.proto.types_pb2.DataType | None = ..., ) -> None: ... def HasField( self, @@ -3477,6 +3482,8 @@ class GroupMap(google.protobuf.message.Message): b"_is_map_groups_with_state", "_output_mode", b"_output_mode", + "_state_schema", + b"_state_schema", "_timeout_conf", b"_timeout_conf", "func", @@ -3489,6 +3496,8 @@ class GroupMap(google.protobuf.message.Message): b"is_map_groups_with_state", "output_mode", b"output_mode", + "state_schema", + b"state_schema", "timeout_conf", b"timeout_conf", ], @@ -3500,6 +3509,8 @@ class GroupMap(google.protobuf.message.Message): b"_is_map_groups_with_state", "_output_mode", b"_output_mode", + "_state_schema", + b"_state_schema", "_timeout_conf", b"_timeout_conf", "func", @@ -3518,6 +3529,8 @@ class GroupMap(google.protobuf.message.Message): b"output_mode", "sorting_expressions", b"sorting_expressions", + "state_schema", + b"state_schema", "timeout_conf", b"timeout_conf", ], @@ -3534,6 +3547,10 @@ class GroupMap(google.protobuf.message.Message): self, oneof_group: typing_extensions.Literal["_output_mode", b"_output_mode"] ) -> typing_extensions.Literal["output_mode"] | None: ... @typing.overload + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_state_schema", b"_state_schema"] + ) -> typing_extensions.Literal["state_schema"] | None: ... + @typing.overload def WhichOneof( self, oneof_group: typing_extensions.Literal["_timeout_conf", b"_timeout_conf"] ) -> typing_extensions.Literal["timeout_conf"] | None: ... diff --git a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto index 7a86db2799149..5ab9f64149f51 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -975,6 +975,9 @@ message GroupMap { // (Optional) Timeout configuration for groups that do not receive data for a while. optional string timeout_conf = 9; + + // (Optional) The schema for the grouped state. + optional DataType state_schema = 10; } message CoGroupMap { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 628b758dd4e2a..f4be1d17b0e95 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -667,7 +667,8 @@ class SparkConnectPlanner( private def transformTypedGroupMap( rel: proto.GroupMap, commonUdf: proto.CommonInlineUserDefinedFunction): LogicalPlan = { - val udf = TypedScalaUdf(commonUdf) + val unpackedUdf = unpackUdf(commonUdf) + val udf = TypedScalaUdf(unpackedUdf, None) val ds = UntypedKeyValueGroupedDataset( rel.getInput, rel.getGroupingExpressionsList, @@ -697,6 +698,18 @@ class SparkConnectPlanner( InternalOutputModes(rel.getOutputMode) } + val stateSchema = DataTypeProtoConverter.toCatalystType(rel.getStateSchema) match { + case s: StructType => s + case other => + throw InvalidPlanInput( + s"Invalid state schema dataType $other for flatMapGroupsWithState") + } + val stateEncoder = TypedScalaUdf.encoderFor( + // the state agnostic encoder is the second element in the input encoders. + unpackedUdf.inputEncoders.tail.head, + "state", + Some(DataTypeUtils.toAttributes(stateSchema))) + val flatMapGroupsWithState = if (hasInitialState) { new FlatMapGroupsWithState( udf.function @@ -706,7 +719,7 @@ class SparkConnectPlanner( ds.groupingAttributes, ds.dataAttributes, udf.outputObjAttr, - initialDs.vEncoder.asInstanceOf[ExpressionEncoder[Any]], + stateEncoder.asInstanceOf[ExpressionEncoder[Any]], outputMode, rel.getIsMapGroupsWithState, timeoutConf, @@ -725,7 +738,7 @@ class SparkConnectPlanner( ds.groupingAttributes, ds.dataAttributes, udf.outputObjAttr, - initialDs.vEncoder.asInstanceOf[ExpressionEncoder[Any]], + stateEncoder.asInstanceOf[ExpressionEncoder[Any]], outputMode, rel.getIsMapGroupsWithState, timeoutConf, @@ -947,10 +960,7 @@ class SparkConnectPlanner( } } - def apply( - commonUdf: proto.CommonInlineUserDefinedFunction, - inputAttrs: Option[Seq[Attribute]] = None): TypedScalaUdf = { - val udf = unpackUdf(commonUdf) + def apply(udf: UdfPacket, inputAttrs: Option[Seq[Attribute]]): TypedScalaUdf = { // There might be more than one inputs, but we only interested in the first one. // Most typed API takes one UDF input. // For the few that takes more than one inputs, e.g. grouping function mapping UDFs, @@ -960,6 +970,13 @@ class SparkConnectPlanner( TypedScalaUdf(udf.function, udf.outputEncoder, inEnc, inputAttrs) } + def apply( + commonUdf: proto.CommonInlineUserDefinedFunction, + inputAttrs: Option[Seq[Attribute]] = None): TypedScalaUdf = { + val udf = unpackUdf(commonUdf) + apply(udf, inputAttrs) + } + def encoderFor( encoder: AgnosticEncoder[_], errorType: String, From 721a41794e113c4372598ceb5e8176f73c911764 Mon Sep 17 00:00:00 2001 From: Evan Wu Date: Thu, 2 Jan 2025 12:12:23 +0800 Subject: [PATCH 15/60] [SPARK-50702][PYTHON] Refine the docstring of regexp_count, regexp_extract and regexp_extract_all ### What changes were proposed in this pull request? Refine the docstring for `regexp_count`, `regexp_extract` and `regexp_extract_all`. ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? new doctests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49338 from drexler-sky/docstring. Authored-by: Evan Wu Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 124 +++++++++++++++++++----- 1 file changed, 98 insertions(+), 26 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 7b14598a0ef4e..5d557c7277a30 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15329,9 +15329,9 @@ def regexp_count(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. Returns @@ -15341,13 +15341,35 @@ def regexp_count(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("1a 2b 14m", r"\d+")], ["str", "regexp"]) - >>> df.select(regexp_count('str', lit(r'\d+')).alias('d')).collect() - [Row(d=3)] - >>> df.select(regexp_count('str', lit(r'mmm')).alias('d')).collect() - [Row(d=0)] - >>> df.select(regexp_count("str", col("regexp")).alias('d')).collect() - [Row(d=3)] + >>> df.select('*', sf.regexp_count('str', sf.lit(r'\d+'))).show() + +---------+------+----------------------+ + | str|regexp|regexp_count(str, \d+)| + +---------+------+----------------------+ + |1a 2b 14m| \d+| 3| + +---------+------+----------------------+ + + >>> df.select('*', sf.regexp_count('str', sf.lit(r'mmm'))).show() + +---------+------+----------------------+ + | str|regexp|regexp_count(str, mmm)| + +---------+------+----------------------+ + |1a 2b 14m| \d+| 0| + +---------+------+----------------------+ + + >>> df.select('*', sf.regexp_count("str", sf.col("regexp"))).show() + +---------+------+-------------------------+ + | str|regexp|regexp_count(str, regexp)| + +---------+------+-------------------------+ + |1a 2b 14m| \d+| 3| + +---------+------+-------------------------+ + + >>> df.select('*', sf.regexp_count(sf.col('str'), "regexp")).show() + +---------+------+-------------------------+ + | str|regexp|regexp_count(str, regexp)| + +---------+------+-------------------------+ + |1a 2b 14m| \d+| 3| + +---------+------+-------------------------+ """ return _invoke_function_over_columns("regexp_count", str, regexp) @@ -15364,7 +15386,7 @@ def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. pattern : str regex pattern to apply. @@ -15376,17 +15398,36 @@ def regexp_extract(str: "ColumnOrName", pattern: str, idx: int) -> Column: :class:`~pyspark.sql.Column` matched value specified by `idx` group id. + See Also + -------- + :meth:`pyspark.sql.functions.regexp_extract_all` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([('100-200',)], ['str']) - >>> df.select(regexp_extract('str', r'(\d+)-(\d+)', 1).alias('d')).collect() - [Row(d='100')] + >>> df.select('*', sf.regexp_extract('str', r'(\d+)-(\d+)', 1)).show() + +-------+-----------------------------------+ + | str|regexp_extract(str, (\d+)-(\d+), 1)| + +-------+-----------------------------------+ + |100-200| 100| + +-------+-----------------------------------+ + >>> df = spark.createDataFrame([('foo',)], ['str']) - >>> df.select(regexp_extract('str', r'(\d+)', 1).alias('d')).collect() - [Row(d='')] + >>> df.select('*', sf.regexp_extract('str', r'(\d+)', 1)).show() + +---+-----------------------------+ + |str|regexp_extract(str, (\d+), 1)| + +---+-----------------------------+ + |foo| | + +---+-----------------------------+ + >>> df = spark.createDataFrame([('aaaac',)], ['str']) - >>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect() - [Row(d='')] + >>> df.select('*', sf.regexp_extract(sf.col('str'), '(a+)(b)?(c)', 2)).show() + +-----+-----------------------------------+ + | str|regexp_extract(str, (a+)(b)?(c), 2)| + +-----+-----------------------------------+ + |aaaac| | + +-----+-----------------------------------+ """ from pyspark.sql.classic.column import _to_java_column @@ -15406,11 +15447,11 @@ def regexp_extract_all( Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. - idx : int, optional + idx : :class:`~pyspark.sql.Column` or int, optional matched group id. Returns @@ -15418,17 +15459,48 @@ def regexp_extract_all( :class:`~pyspark.sql.Column` all strings in the `str` that match a Java regex and corresponding to the regex group index. + See Also + -------- + :meth:`pyspark.sql.functions.regexp_extract` + Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("100-200, 300-400", r"(\d+)-(\d+)")], ["str", "regexp"]) - >>> df.select(regexp_extract_all('str', lit(r'(\d+)-(\d+)')).alias('d')).collect() - [Row(d=['100', '300'])] - >>> df.select(regexp_extract_all('str', lit(r'(\d+)-(\d+)'), 1).alias('d')).collect() - [Row(d=['100', '300'])] - >>> df.select(regexp_extract_all('str', lit(r'(\d+)-(\d+)'), 2).alias('d')).collect() - [Row(d=['200', '400'])] - >>> df.select(regexp_extract_all('str', col("regexp")).alias('d')).collect() - [Row(d=['100', '300'])] + >>> df.select('*', sf.regexp_extract_all('str', sf.lit(r'(\d+)-(\d+)'))).show() + +----------------+-----------+---------------------------------------+ + | str| regexp|regexp_extract_all(str, (\d+)-(\d+), 1)| + +----------------+-----------+---------------------------------------+ + |100-200, 300-400|(\d+)-(\d+)| [100, 300]| + +----------------+-----------+---------------------------------------+ + + >>> df.select('*', sf.regexp_extract_all('str', sf.lit(r'(\d+)-(\d+)'), sf.lit(1))).show() + +----------------+-----------+---------------------------------------+ + | str| regexp|regexp_extract_all(str, (\d+)-(\d+), 1)| + +----------------+-----------+---------------------------------------+ + |100-200, 300-400|(\d+)-(\d+)| [100, 300]| + +----------------+-----------+---------------------------------------+ + + >>> df.select('*', sf.regexp_extract_all('str', sf.lit(r'(\d+)-(\d+)'), 2)).show() + +----------------+-----------+---------------------------------------+ + | str| regexp|regexp_extract_all(str, (\d+)-(\d+), 2)| + +----------------+-----------+---------------------------------------+ + |100-200, 300-400|(\d+)-(\d+)| [200, 400]| + +----------------+-----------+---------------------------------------+ + + >>> df.select('*', sf.regexp_extract_all('str', sf.col("regexp"))).show() + +----------------+-----------+----------------------------------+ + | str| regexp|regexp_extract_all(str, regexp, 1)| + +----------------+-----------+----------------------------------+ + |100-200, 300-400|(\d+)-(\d+)| [100, 300]| + +----------------+-----------+----------------------------------+ + + >>> df.select('*', sf.regexp_extract_all(sf.col('str'), "regexp")).show() + +----------------+-----------+----------------------------------+ + | str| regexp|regexp_extract_all(str, regexp, 1)| + +----------------+-----------+----------------------------------+ + |100-200, 300-400|(\d+)-(\d+)| [100, 300]| + +----------------+-----------+----------------------------------+ """ if idx is None: return _invoke_function_over_columns("regexp_extract_all", str, regexp) From 492fcd83258bcd5a41f60a7b44b0ab6d4c9916b0 Mon Sep 17 00:00:00 2001 From: zml1206 Date: Thu, 2 Jan 2025 12:37:15 +0800 Subject: [PATCH 16/60] [SPARK-50683][SQL] Inline the common expression in With if used once ### What changes were proposed in this pull request? As title. ### Why are the changes needed? Simplify plan and reduce unnecessary project. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? UT. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49310 from zml1206/with. Authored-by: zml1206 Signed-off-by: Wenchen Fan --- .../optimizer/RewriteWithExpression.scala | 21 +++++++++++++------ .../RewriteWithExpressionSuite.scala | 14 +++++++++++-- 2 files changed, 27 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala index 40189a9f61021..5d85e89e1eabe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpression.scala @@ -68,9 +68,15 @@ object RewriteWithExpression extends Rule[LogicalPlan] { private def applyInternal(p: LogicalPlan): LogicalPlan = { val inputPlans = p.children + val commonExprIdSet = p.expressions + .flatMap(_.collect { case r: CommonExpressionRef => r.id }) + .groupBy(identity) + .transform((_, v) => v.size) + .filter(_._2 > 1) + .keySet val commonExprsPerChild = Array.fill(inputPlans.length)(mutable.ListBuffer.empty[(Alias, Long)]) var newPlan: LogicalPlan = p.mapExpressions { expr => - rewriteWithExprAndInputPlans(expr, inputPlans, commonExprsPerChild) + rewriteWithExprAndInputPlans(expr, inputPlans, commonExprsPerChild, commonExprIdSet) } val newChildren = inputPlans.zip(commonExprsPerChild).map { case (inputPlan, commonExprs) => if (commonExprs.isEmpty) { @@ -96,6 +102,7 @@ object RewriteWithExpression extends Rule[LogicalPlan] { e: Expression, inputPlans: Seq[LogicalPlan], commonExprsPerChild: Array[mutable.ListBuffer[(Alias, Long)]], + commonExprIdSet: Set[CommonExpressionId], isNestedWith: Boolean = false): Expression = { if (!e.containsPattern(WITH_EXPRESSION)) return e e match { @@ -103,9 +110,9 @@ object RewriteWithExpression extends Rule[LogicalPlan] { case w: With if !isNestedWith => // Rewrite nested With expressions first val child = rewriteWithExprAndInputPlans( - w.child, inputPlans, commonExprsPerChild, isNestedWith = true) + w.child, inputPlans, commonExprsPerChild, commonExprIdSet, isNestedWith = true) val defs = w.defs.map(rewriteWithExprAndInputPlans( - _, inputPlans, commonExprsPerChild, isNestedWith = true)) + _, inputPlans, commonExprsPerChild, commonExprIdSet, isNestedWith = true)) val refToExpr = mutable.HashMap.empty[CommonExpressionId, Expression] defs.zipWithIndex.foreach { case (CommonExpressionDef(child, id), index) => @@ -114,7 +121,7 @@ object RewriteWithExpression extends Rule[LogicalPlan] { "Cannot rewrite canonicalized Common expression definitions") } - if (CollapseProject.isCheap(child)) { + if (CollapseProject.isCheap(child) || !commonExprIdSet.contains(id)) { refToExpr(id) = child } else { val childPlanIndex = inputPlans.indexWhere( @@ -171,7 +178,8 @@ object RewriteWithExpression extends Rule[LogicalPlan] { case c: ConditionalExpression => val newAlwaysEvaluatedInputs = c.alwaysEvaluatedInputs.map( - rewriteWithExprAndInputPlans(_, inputPlans, commonExprsPerChild, isNestedWith)) + rewriteWithExprAndInputPlans( + _, inputPlans, commonExprsPerChild, commonExprIdSet, isNestedWith)) val newExpr = c.withNewAlwaysEvaluatedInputs(newAlwaysEvaluatedInputs) // Use transformUp to handle nested With. newExpr.transformUpWithPruning(_.containsPattern(WITH_EXPRESSION)) { @@ -185,7 +193,8 @@ object RewriteWithExpression extends Rule[LogicalPlan] { } case other => other.mapChildren( - rewriteWithExprAndInputPlans(_, inputPlans, commonExprsPerChild, isNestedWith) + rewriteWithExprAndInputPlans( + _, inputPlans, commonExprsPerChild, commonExprIdSet, isNestedWith) ) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala index 9f0a7fdaf3152..8918b58ca1b56 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteWithExpressionSuite.scala @@ -140,7 +140,7 @@ class RewriteWithExpressionSuite extends PlanTest { val commonExprDef2 = CommonExpressionDef(a + a, CommonExpressionId(2)) val ref2 = new CommonExpressionRef(commonExprDef2) // The inner main expression references the outer expression - val innerExpr2 = With(ref2 + outerRef, Seq(commonExprDef2)) + val innerExpr2 = With(ref2 + ref2 + outerRef, Seq(commonExprDef2)) val outerExpr2 = With(outerRef + innerExpr2, Seq(outerCommonExprDef)) comparePlans( Optimizer.execute(testRelation.select(outerExpr2.as("col"))), @@ -152,7 +152,8 @@ class RewriteWithExpressionSuite extends PlanTest { .select(star(), (a + a).as("_common_expr_2")) // The final Project contains the final result expression, which references both common // expressions. - .select(($"_common_expr_0" + ($"_common_expr_2" + $"_common_expr_0")).as("col")) + .select(($"_common_expr_0" + + ($"_common_expr_2" + $"_common_expr_2" + $"_common_expr_0")).as("col")) .analyze ) } @@ -490,4 +491,13 @@ class RewriteWithExpressionSuite extends PlanTest { val wrongPlan = testRelation.select(expr1.as("c1"), expr3.as("c3")).analyze intercept[AssertionError](Optimizer.execute(wrongPlan)) } + + test("SPARK-50683: inline the common expression in With if used once") { + val a = testRelation.output.head + val exprDef = CommonExpressionDef(a + a) + val exprRef = new CommonExpressionRef(exprDef) + val expr = With(exprRef + 1, Seq(exprDef)) + val plan = testRelation.select(expr.as("col")) + comparePlans(Optimizer.execute(plan), testRelation.select((a + a + 1).as("col"))) + } } From 5c634845f0849b3d5bf6a98c4ecddff26b71572b Mon Sep 17 00:00:00 2001 From: cashmand Date: Thu, 2 Jan 2025 12:39:07 +0800 Subject: [PATCH 17/60] [SPARK-50614][SQL] Add Variant shredding support for Parquet ### What changes were proposed in this pull request? Adds support for shredding in the Parquet writer code. Currently, the only way to enable shredding is through a SQLConf that provides the schema to use for shredding. This doesn't make sense as a user API, and is added only for testing. The exact API for Spark to determine a shredding schema is still TBD, but likely candidates are to infer it at the task level by inspecting the first few rows of data, or add an API to specify the schema for a given column. Either way, the code in this PR would basically be unchanged, it would just use a different mechanism to provide the schema. ### Why are the changes needed? Needed for Variant shredding support. ### Does this PR introduce _any_ user-facing change? No, the feature is new in Spark 4.0, and is currently disabled, and only usable as a test feature. ### How was this patch tested? Added a unit test suite. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49234 from cashmand/SPARK-50614. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../apache/spark/sql/internal/SQLConf.scala | 16 ++ .../parquet/ParquetSchemaConverter.scala | 8 + .../datasources/parquet/ParquetUtils.scala | 34 ++- .../parquet/ParquetWriteSupport.scala | 65 ++++- .../parquet/SparkShreddingUtils.scala | 23 ++ .../ParquetVariantShreddingSuite.scala | 229 ++++++++++++++++++ 6 files changed, 362 insertions(+), 13 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala 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 5e630577638a0..6a45380b7a99d 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 @@ -4659,6 +4659,22 @@ object SQLConf { .booleanConf .createWithDefault(false) + val VARIANT_WRITE_SHREDDING_ENABLED = + buildConf("spark.sql.variant.writeShredding.enabled") + .internal() + .doc("When true, the Parquet writer is allowed to write shredded variant. ") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + + val VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST = + buildConf("spark.sql.variant.forceShreddingSchemaForTest") + .internal() + .doc("FOR INTERNAL TESTING ONLY. Sets shredding schema for Variant.") + .version("4.0.0") + .stringConf + .createWithDefault("") + val LEGACY_CSV_ENABLE_DATE_TIME_PARSING_FALLBACK = buildConf("spark.sql.legacy.csv.enableDateTimeParsingFallback") .internal() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala index 64c2a3126ca9a..daeb8e88a924b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala @@ -756,6 +756,14 @@ class SparkToParquetSchemaConverter( .addField(convertField(StructField("metadata", BinaryType, nullable = false))) .named(field.name) + case s: StructType if SparkShreddingUtils.isVariantShreddingStruct(s) => + // Variant struct takes a Variant and writes to Parquet as a shredded schema. + val group = Types.buildGroup(repetition) + s.fields.foreach { f => + group.addField(convertField(f)) + } + group.named(field.name) + case StructType(fields) => fields.foldLeft(Types.buildGroup(repetition)) { (builder, field) => builder.addField(convertField(field)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index 3e111252bc6fe..a609a4e0a25f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -45,7 +45,7 @@ import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, Outpu import org.apache.spark.sql.execution.datasources.v2.V2ColumnUtils import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.internal.SQLConf.PARQUET_AGGREGATE_PUSHDOWN_ENABLED -import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, MapType, StructField, StructType, UserDefinedType} +import org.apache.spark.sql.types.{ArrayType, AtomicType, DataType, MapType, StructField, StructType, UserDefinedType, VariantType} import org.apache.spark.util.ArrayImplicits._ object ParquetUtils extends Logging { @@ -420,6 +420,22 @@ object ParquetUtils extends Logging { statistics.getNumNulls; } + // Replaces each VariantType in the schema with the corresponding type in the shredding schema. + // Used for testing, where we force a single shredding schema for all Variant fields. + // Does not touch Variant fields nested in arrays, maps, or UDTs. + private def replaceVariantTypes(schema: StructType, shreddingSchema: StructType): StructType = { + val newFields = schema.fields.zip(shreddingSchema.fields).map { + case (field, shreddingField) => + field.dataType match { + case s: StructType => + field.copy(dataType = replaceVariantTypes(s, shreddingSchema)) + case VariantType => field.copy(dataType = shreddingSchema) + case _ => field + } + } + StructType(newFields) + } + def prepareWrite( sqlConf: SQLConf, job: Job, @@ -454,8 +470,22 @@ object ParquetUtils extends Logging { ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) + val shreddingSchema = if (sqlConf.getConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED) && + !sqlConf.getConf(SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST).isEmpty) { + // Convert the schema to a shredding schema, and replace it anywhere that there is a + // VariantType in the original schema. + val simpleShreddingSchema = DataType.fromDDL( + sqlConf.getConf(SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST) + ) + val oneShreddingSchema = SparkShreddingUtils.variantShreddingSchema(simpleShreddingSchema) + val schemaWithMetadata = SparkShreddingUtils.addWriteShreddingMetadata(oneShreddingSchema) + Some(replaceVariantTypes(dataSchema, schemaWithMetadata)) + } else { + None + } + // This metadata is useful for keeping UDTs like Vector/Matrix. - ParquetWriteSupport.setSchema(dataSchema, conf) + ParquetWriteSupport.setSchema(dataSchema, conf, shreddingSchema) // Sets flags for `ParquetWriteSupport`, which converts Catalyst schema to Parquet // schema and writes actual rows to Parquet files. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 89a1cd5d4375a..9402f5638094d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -38,6 +38,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ +import org.apache.spark.types.variant.Variant /** * A Parquet [[WriteSupport]] implementation that writes Catalyst [[InternalRow]]s as Parquet @@ -59,6 +60,10 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { // Schema of the `InternalRow`s to be written private var schema: StructType = _ + // Schema of the `InternalRow`s to be written, with VariantType replaced with its shredding + // schema, if appropriate. + private var shreddedSchema: StructType = _ + // `ValueWriter`s for all fields of the schema private var rootFieldWriters: Array[ValueWriter] = _ @@ -95,7 +100,16 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { override def init(configuration: Configuration): WriteContext = { val schemaString = configuration.get(ParquetWriteSupport.SPARK_ROW_SCHEMA) + val shreddedSchemaString = configuration.get(ParquetWriteSupport.SPARK_VARIANT_SHREDDING_SCHEMA) this.schema = StructType.fromString(schemaString) + // If shreddingSchemaString is provided, we use that everywhere in the writer, except for + // setting the spark schema in the Parquet metadata. If it isn't provided, it means that there + // are no shredded Variant columns, so it is identical to this.schema. + this.shreddedSchema = if (shreddedSchemaString == null) { + this.schema + } else { + StructType.fromString(shreddedSchemaString) + } this.writeLegacyParquetFormat = { // `SQLConf.PARQUET_WRITE_LEGACY_FORMAT` should always be explicitly set in ParquetRelation assert(configuration.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key) != null) @@ -108,9 +122,9 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { SQLConf.ParquetOutputTimestampType.withName(configuration.get(key)) } - this.rootFieldWriters = schema.map(_.dataType).map(makeWriter).toArray[ValueWriter] + this.rootFieldWriters = shreddedSchema.map(_.dataType).map(makeWriter).toArray[ValueWriter] - val messageType = new SparkToParquetSchemaConverter(configuration).convert(schema) + val messageType = new SparkToParquetSchemaConverter(configuration).convert(shreddedSchema) val metadata = Map( SPARK_VERSION_METADATA_KEY -> SPARK_VERSION_SHORT, ParquetReadSupport.SPARK_METADATA_KEY -> schemaString @@ -132,13 +146,23 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { } } - logDebug( - s"""Initialized Parquet WriteSupport with Catalyst schema: - |${schema.prettyJson} - |and corresponding Parquet message type: - |$messageType - """.stripMargin) - + if (shreddedSchemaString == null) { + logDebug( + s"""Initialized Parquet WriteSupport with Catalyst schema: + |${schema.prettyJson} + |and corresponding Parquet message type: + |$messageType + """.stripMargin) + } else { + logDebug( + s"""Initialized Parquet WriteSupport with Catalyst schema: + |${schema.prettyJson} + |and shredding schema: + |$shreddedSchema.prettyJson} + |and corresponding Parquet message type: + |$messageType + """.stripMargin) + } new WriteContext(messageType, metadata.asJava) } @@ -148,7 +172,7 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { override def write(row: InternalRow): Unit = { consumeMessage { - writeFields(row, schema, rootFieldWriters) + writeFields(row, shreddedSchema, rootFieldWriters) } } @@ -250,6 +274,17 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { } } + case s: StructType if SparkShreddingUtils.isVariantShreddingStruct(s) => + val fieldWriters = s.map(_.dataType).map(makeWriter).toArray[ValueWriter] + val variantShreddingSchema = SparkShreddingUtils.buildVariantSchema(s) + (row: SpecializedGetters, ordinal: Int) => + val v = row.getVariant(ordinal) + val variant = new Variant(v.getValue, v.getMetadata) + val shreddedValues = SparkShreddingUtils.castShredded(variant, variantShreddingSchema) + consumeGroup { + writeFields(shreddedValues, s, fieldWriters) + } + case t: StructType => val fieldWriters = t.map(_.dataType).map(makeWriter).toArray[ValueWriter] (row: SpecializedGetters, ordinal: Int) => @@ -499,11 +534,19 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { object ParquetWriteSupport { val SPARK_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.attributes" + // A version of `SPARK_ROW_SCHEMA`, where one or more Variant attributes have been replace with a + // shredded struct schema. + val SPARK_VARIANT_SHREDDING_SCHEMA: String = + "org.apache.spark.sql.parquet.variant.shredding.attributes" - def setSchema(schema: StructType, configuration: Configuration): Unit = { + def setSchema(schema: StructType, configuration: Configuration, + shreddingSchema: Option[StructType]): Unit = { configuration.set(SPARK_ROW_SCHEMA, schema.json) configuration.setIfUnset( ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) + shreddingSchema.foreach { s => + configuration.set(SPARK_VARIANT_SHREDDING_SCHEMA, s.json) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index 34c167aea363a..c0c4900344154 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -448,6 +448,8 @@ case object SparkShreddingUtils { val TypedValueFieldName = "typed_value"; val MetadataFieldName = "metadata"; + val VARIANT_WRITE_SHREDDING_KEY: String = "__VARIANT_WRITE_SHREDDING_KEY" + def buildVariantSchema(schema: DataType): VariantSchema = { schema match { case s: StructType => buildVariantSchema(s, topLevel = true) @@ -512,6 +514,27 @@ case object SparkShreddingUtils { } } + /** + * Given a schema that represents a valid shredding schema (e.g. constructed by + * SparkShreddingUtils.variantShreddingSchema), add metadata to the top-level fields to mark it + * as a shredding schema for writers. + */ + def addWriteShreddingMetadata(schema: StructType): StructType = { + val newFields = schema.fields.map { f => + f.copy(metadata = new + MetadataBuilder() + .withMetadata(f.metadata) + .putNull(VARIANT_WRITE_SHREDDING_KEY).build()) + } + StructType(newFields) + } + + // Check if the struct is marked with metadata set by addWriteShreddingMetadata - i.e. it + // represents a Variant converted to a shredding schema for writing. + def isVariantShreddingStruct(s: StructType): Boolean = { + s.fields.length > 0 && s.fields.forall(_.metadata.contains(VARIANT_WRITE_SHREDDING_KEY)) + } + /* * Given a Spark schema that represents a valid shredding schema (e.g. constructed by * SparkShreddingUtils.variantShreddingSchema), return the corresponding VariantSchema. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala new file mode 100644 index 0000000000000..8bb5a4b1d0bc5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetVariantShreddingSuite.scala @@ -0,0 +1,229 @@ +/* + * 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.parquet + +import java.io.File + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.unsafe.types.VariantVal + +/** + * Test shredding Variant values in the Parquet reader/writer. + */ +class ParquetVariantShreddingSuite extends QueryTest with ParquetTest with SharedSparkSession { + + private def testWithTempDir(name: String)(block: File => Unit): Unit = test(name) { + withTempDir { dir => + block(dir) + } + } + + testWithTempDir("write shredded variant basic") { dir => + val schema = "a int, b string, c decimal(15, 1)" + val df = spark.sql( + """ + | select case + | when id = 0 then parse_json('{"a": 1, "b": "2", "c": 3.3, "d": 4.4}') + | when id = 1 then parse_json('{"a": [1,2,3], "b": "hello", "c": {"x": 0}}') + | when id = 2 then parse_json('{"A": 1, "c": 1.23}') + | end v from range(3) + |""".stripMargin) + val fullSchema = "v struct, b struct," + + "c struct>>" + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + + + // Verify that we can read the full variant. The exact binary layout can change before and + // after shredding, so just check that the JSON representation matches. + checkAnswer( + spark.read.parquet(dir.getAbsolutePath).selectExpr("to_json(v)"), + df.selectExpr("to_json(v)").collect() + ) + + // Verify that it was shredded to the expected fields. + + val shreddedDf = spark.read.schema(fullSchema).parquet(dir.getAbsolutePath) + // Metadata should be unchanaged. + checkAnswer(shreddedDf.selectExpr("v.metadata"), + df.collect().map(v => Row(v.get(0).asInstanceOf[VariantVal].getMetadata)) + ) + + // Check typed values. + // Second row is not an integer, and third is A, not a + checkAnswer( + shreddedDf.selectExpr("v.typed_value.a.typed_value"), + Seq(Row(1), Row(null), Row(null))) + // b is missing from third row. + checkAnswer( + shreddedDf.selectExpr("v.typed_value.b.typed_value"), + Seq(Row("2"), Row("hello"), Row(null))) + // Second row is an object, third is the wrong scale. (Note: we may eventually allow the + // latter, in which case this test should be updated.) + checkAnswer( + shreddedDf.selectExpr("v.typed_value.c.typed_value"), + Seq(Row(3.3), Row(null), Row(null))) + + // Untyped values are more awkward to check, so for now just check their nullness. We + // can do more thorough checking once the reader is ready. + checkAnswer( + shreddedDf.selectExpr("v.value is null"), + // First row has "d" and third has "A". + Seq(Row(false), Row(true), Row(false))) + checkAnswer( + shreddedDf.selectExpr("v.typed_value.a.value is null"), + // First row is fully shredded, third is missing. + Seq(Row(true), Row(false), Row(true))) + checkAnswer( + shreddedDf.selectExpr("v.typed_value.b.value is null"), + // b is always fully shredded or missing. + Seq(Row(true), Row(true), Row(true))) + checkAnswer( + shreddedDf.selectExpr("v.typed_value.c.value is null"), + Seq(Row(true), Row(false), Row(false))) + // The a/b/c levels are not null, even if the field is missing. + checkAnswer( + shreddedDf.selectExpr( + "v.typed_value.a is null or v.typed_value.b is null or v.typed_value.c is null"), + Seq(Row(false), Row(false), Row(false))) + } + } + + testWithTempDir("write shredded variant array") { dir => + val schema = "array" + val df = spark.sql( + """ + | select case + | when id = 0 then parse_json('[1, "2", 3.5, null, 5]') + | when id = 1 then parse_json('{"a": [1, 2, 3]}') + | when id = 2 then parse_json('1') + | when id = 3 then parse_json('null') + | end v from range(4) + |""".stripMargin) + val fullSchema = "v struct>>" + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + + // Verify that we can read the full variant. + checkAnswer( + spark.read.parquet(dir.getAbsolutePath).selectExpr("to_json(v)"), + df.selectExpr("to_json(v)").collect() + ) + + // Verify that it was shredded to the expected fields. + + val shreddedDf = spark.read.schema(fullSchema).parquet(dir.getAbsolutePath) + // Metadata should be unchanaged. + checkAnswer(shreddedDf.selectExpr("v.metadata"), + df.collect().map(v => Row(v.get(0).asInstanceOf[VariantVal].getMetadata)) + ) + + // Check typed values. + checkAnswer( + shreddedDf.selectExpr("v.typed_value.typed_value"), + Seq(Row(Array(1, null, null, null, 5)), Row(null), Row(null), Row(null))) + + // All the other array elements should have non-null value. + checkAnswer( + shreddedDf.selectExpr("transform(v.typed_value.value, v -> v is null)"), + Seq(Row(Array(true, false, false, false, true)), Row(null), Row(null), Row(null))) + + // The non-arrays should have non-null top-level value. + checkAnswer( + shreddedDf.selectExpr("v.value is null"), + Seq(Row(true), Row(false), Row(false), Row(false))) + } + } + + testWithTempDir("write no shredding schema") { dir => + // Check that we can write and read normally when shredding is enabled if + // we don't provide a shredding schema. + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString) { + val df = spark.sql( + """ + | select parse_json('{"a": ' || id || ', "b": 2}') as v, + | array(parse_json('{"c": 3}'), 123::variant) as a + | from range(1, 3, 1, 1) + |""".stripMargin) + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + checkAnswer( + spark.read.parquet(dir.getAbsolutePath), df.collect() + ) + } + } + + testWithTempDir("arrays and maps ignore shredding schema") { dir => + // Check that we don't try to shred array or map elements, even if a shredding schema + // is specified. + val schema = "a int" + val df = spark.sql( + """ select v, array(v) as arr, map('myKey', v) as m from + | (select parse_json('{"a":' || id || '}') v from range(3)) + |""".stripMargin) + val fullSchema = "v struct>>, " + + "arr array>, " + + "m map>" + withSQLConf(SQLConf.VARIANT_WRITE_SHREDDING_ENABLED.key -> true.toString, + SQLConf.VARIANT_FORCE_SHREDDING_SCHEMA_FOR_TEST.key -> schema) { + df.write.mode("overwrite").parquet(dir.getAbsolutePath) + + // Verify that we can read the full variant. + checkAnswer( + spark.read.parquet(dir.getAbsolutePath).selectExpr("to_json(v)"), + df.selectExpr("to_json(v)").collect() + ) + + // Verify that it was shredded to the expected fields. + + val shreddedDf = spark.read.schema(fullSchema).parquet(dir.getAbsolutePath) + // Metadata should be unchanaged. + checkAnswer(shreddedDf.selectExpr("v.metadata"), + df.selectExpr("v").collect().map(v => Row(v.get(0).asInstanceOf[VariantVal].getMetadata)) + ) + checkAnswer(shreddedDf.selectExpr("arr[0].metadata"), + df.selectExpr("arr[0]").collect().map(v => + Row(v.get(0).asInstanceOf[VariantVal].getMetadata)) + ) + checkAnswer(shreddedDf.selectExpr("m['myKey'].metadata"), + df.selectExpr("m['myKey']").collect().map( + v => Row(v.get(0).asInstanceOf[VariantVal].getMetadata)) + ) + + // v should be fully shredded, but the array and map should not be. + checkAnswer( + shreddedDf.selectExpr( + "v.value is null"), + Seq(Row(true), Row(true), Row(true))) + checkAnswer( + shreddedDf.selectExpr( + "arr[0].value is null"), + Seq(Row(false), Row(false), Row(false))) + checkAnswer( + shreddedDf.selectExpr( + "m['myKey'].value is null"), + Seq(Row(false), Row(false), Row(false))) + } + } +} From 3c0824d92068e92c5672b2d2ea1eba9c5c30cc13 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 2 Jan 2025 16:43:09 +0800 Subject: [PATCH 18/60] [SPARK-50711][PS][DOCS] Upgrade the minimum version of Pandas in PS to 2.2.0 ### What changes were proposed in this pull request? Upgrade the minimum version of Pandas in PS to 2.2.0 But the version used in `require_minimum_pandas_version` is still 2.0.0, because currently this function is used in both PS and non-PS modules. ### Why are the changes needed? Actually, PS with Pandas < 2.2.0 has already been broken in master branch ### Does this PR introduce _any_ user-facing change? No, doc-only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #49344 from zhengruifeng/ps_pd_mini. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- dev/requirements.txt | 2 +- python/docs/source/getting_started/install.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/requirements.txt b/dev/requirements.txt index 33300cc28d3ca..c1ae808c5e291 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -5,7 +5,7 @@ py4j>=0.10.9.7 numpy>=1.21 pyarrow>=11.0.0 six==1.16.0 -pandas>=2.0.0 +pandas>=2.2.0 scipy plotly>=4.8 mlflow>=2.3.1 diff --git a/python/docs/source/getting_started/install.rst b/python/docs/source/getting_started/install.rst index b35588a618acf..515224b52e09a 100644 --- a/python/docs/source/getting_started/install.rst +++ b/python/docs/source/getting_started/install.rst @@ -239,7 +239,7 @@ Installable with ``pip install "pyspark[pandas_on_spark]"``. ========= ================= ================================ Package Supported version Note ========= ================= ================================ -`pandas` >=2.0.0 Required for Pandas API on Spark +`pandas` >=2.2.0 Required for Pandas API on Spark `pyarrow` >=11.0.0 Required for Pandas API on Spark ========= ================= ================================ From c374c00432bd28b98b7623481b7985aa7bc7624a Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Thu, 2 Jan 2025 16:44:21 +0800 Subject: [PATCH 19/60] [SPARK-50712][INFRA][PS][TESTS] Add a daily build for Pandas API on Spark with old dependencies ### What changes were proposed in this pull request? Add a daily build for Pandas API on Spark with old dependencies ### Why are the changes needed? The PS part requires a newer version of Pandas ### Does this PR introduce _any_ user-facing change? No, infra-only ### How was this patch tested? PR builder with ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-ps-minimum", "PYTHON_TO_TEST": "python3.9"}' default: '{"pyspark": "true", "pyspark-pandas": "true"}' ``` https://github.com/zhengruifeng/spark/runs/35054863846 ### Was this patch authored or co-authored using generative AI tooling? Closes #49343 from zhengruifeng/infra_ps_mini. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../workflows/build_infra_images_cache.yml | 13 +++ .github/workflows/build_python_ps_minimum.yml | 47 +++++++++++ .../python-ps-minimum/Dockerfile | 81 +++++++++++++++++++ python/pyspark/pandas/tests/io/test_io.py | 8 +- .../pandas/tests/io/test_series_conversion.py | 2 + 5 files changed, 150 insertions(+), 1 deletion(-) create mode 100644 .github/workflows/build_python_ps_minimum.yml create mode 100644 dev/spark-test-image/python-ps-minimum/Dockerfile diff --git a/.github/workflows/build_infra_images_cache.yml b/.github/workflows/build_infra_images_cache.yml index 565bb8c7d6e68..ac139147beb91 100644 --- a/.github/workflows/build_infra_images_cache.yml +++ b/.github/workflows/build_infra_images_cache.yml @@ -122,6 +122,19 @@ jobs: - name: Image digest (PySpark with old dependencies) if: hashFiles('dev/spark-test-image/python-minimum/Dockerfile') != '' run: echo ${{ steps.docker_build_pyspark_python_minimum.outputs.digest }} + - name: Build and push (PySpark PS with old dependencies) + if: hashFiles('dev/spark-test-image/python-ps-minimum/Dockerfile') != '' + id: docker_build_pyspark_python_ps_minimum + uses: docker/build-push-action@v6 + with: + context: ./dev/spark-test-image/python-ps-minimum/ + push: true + tags: ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-ps-minimum-cache:${{ github.ref_name }}-static + cache-from: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-ps-minimum-cache:${{ github.ref_name }} + cache-to: type=registry,ref=ghcr.io/apache/spark/apache-spark-github-action-image-pyspark-python-ps-minimum-cache:${{ github.ref_name }},mode=max + - name: Image digest (PySpark PS with old dependencies) + if: hashFiles('dev/spark-test-image/python-ps-minimum/Dockerfile') != '' + run: echo ${{ steps.docker_build_pyspark_python_ps_minimum.outputs.digest }} - name: Build and push (PySpark with PyPy 3.10) if: hashFiles('dev/spark-test-image/pypy-310/Dockerfile') != '' id: docker_build_pyspark_pypy_310 diff --git a/.github/workflows/build_python_ps_minimum.yml b/.github/workflows/build_python_ps_minimum.yml new file mode 100644 index 0000000000000..742d578e27418 --- /dev/null +++ b/.github/workflows/build_python_ps_minimum.yml @@ -0,0 +1,47 @@ +# +# 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. +# + +name: "Build / Python-only (master, Python PS with old dependencies)" + +on: + schedule: + - cron: '0 10 * * *' + workflow_dispatch: + +jobs: + run-build: + permissions: + packages: write + name: Run + uses: ./.github/workflows/build_and_test.yml + if: github.repository == 'apache/spark' + with: + java: 17 + branch: master + hadoop: hadoop3 + envs: >- + { + "PYSPARK_IMAGE_TO_TEST": "python-ps-minimum", + "PYTHON_TO_TEST": "python3.9" + } + jobs: >- + { + "pyspark": "true", + "pyspark-pandas": "true" + } diff --git a/dev/spark-test-image/python-ps-minimum/Dockerfile b/dev/spark-test-image/python-ps-minimum/Dockerfile new file mode 100644 index 0000000000000..913da06c551ca --- /dev/null +++ b/dev/spark-test-image/python-ps-minimum/Dockerfile @@ -0,0 +1,81 @@ +# +# 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. +# + +# Image for building and testing Spark branches. Based on Ubuntu 22.04. +# See also in https://hub.docker.com/_/ubuntu +FROM ubuntu:jammy-20240911.1 +LABEL org.opencontainers.image.authors="Apache Spark project " +LABEL org.opencontainers.image.licenses="Apache-2.0" +LABEL org.opencontainers.image.ref.name="Apache Spark Infra Image For Pandas API on Spark with old dependencies" +# Overwrite this label to avoid exposing the underlying Ubuntu OS version label +LABEL org.opencontainers.image.version="" + +ENV FULL_REFRESH_DATE=20250102 + +ENV DEBIAN_FRONTEND=noninteractive +ENV DEBCONF_NONINTERACTIVE_SEEN=true + +RUN apt-get update && apt-get install -y \ + build-essential \ + ca-certificates \ + curl \ + gfortran \ + git \ + gnupg \ + libcurl4-openssl-dev \ + libfontconfig1-dev \ + libfreetype6-dev \ + libfribidi-dev \ + libgit2-dev \ + libharfbuzz-dev \ + libjpeg-dev \ + liblapack-dev \ + libopenblas-dev \ + libpng-dev \ + libpython3-dev \ + libssl-dev \ + libtiff5-dev \ + libxml2-dev \ + openjdk-17-jdk-headless \ + pkg-config \ + qpdf \ + tzdata \ + software-properties-common \ + wget \ + zlib1g-dev + + +# Should keep the installation consistent with https://apache.github.io/spark/api/python/getting_started/install.html + +# Install Python 3.9 +RUN add-apt-repository ppa:deadsnakes/ppa +RUN apt-get update && apt-get install -y \ + python3.9 \ + python3.9-distutils \ + && apt-get autoremove --purge -y \ + && apt-get clean \ + && rm -rf /var/lib/apt/lists/* + + +ARG BASIC_PIP_PKGS="pyarrow==11.0.0 pandas==2.2.0 six==1.16.0 numpy scipy coverage unittest-xml-reporting" +# Python deps for Spark Connect +ARG CONNECT_PIP_PKGS="grpcio==1.67.0 grpcio-status==1.67.0 googleapis-common-protos==1.65.0 graphviz==0.20 protobuf" + +# Install Python 3.9 packages +RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 +RUN python3.9 -m pip install --force $BASIC_PIP_PKGS $CONNECT_PIP_PKGS && \ + python3.9 -m pip cache purge diff --git a/python/pyspark/pandas/tests/io/test_io.py b/python/pyspark/pandas/tests/io/test_io.py index 6fbdc366dd76a..da5817b86b984 100644 --- a/python/pyspark/pandas/tests/io/test_io.py +++ b/python/pyspark/pandas/tests/io/test_io.py @@ -24,7 +24,12 @@ from pyspark import pandas as ps from pyspark.testing.pandasutils import PandasOnSparkTestCase from pyspark.testing.sqlutils import SQLTestUtils -from pyspark.testing.utils import have_tabulate, tabulate_requirement_message +from pyspark.testing.utils import ( + have_jinja2, + jinja2_requirement_message, + have_tabulate, + tabulate_requirement_message, +) # This file contains test cases for 'Serialization / IO / Conversion' @@ -91,6 +96,7 @@ def test_from_dict(self): psdf = ps.DataFrame.from_dict(data, orient="index", columns=["A", "B", "C", "D"]) self.assert_eq(pdf, psdf) + @unittest.skipIf(not have_jinja2, jinja2_requirement_message) def test_style(self): # Currently, the `style` function returns a pandas object `Styler` as it is, # processing only the number of rows declared in `compute.max_rows`. diff --git a/python/pyspark/pandas/tests/io/test_series_conversion.py b/python/pyspark/pandas/tests/io/test_series_conversion.py index 2ae40e92b489a..06d923816633d 100644 --- a/python/pyspark/pandas/tests/io/test_series_conversion.py +++ b/python/pyspark/pandas/tests/io/test_series_conversion.py @@ -23,6 +23,7 @@ from pyspark import pandas as ps from pyspark.testing.pandasutils import PandasOnSparkTestCase from pyspark.testing.sqlutils import SQLTestUtils +from pyspark.testing.utils import have_jinja2, jinja2_requirement_message class SeriesConversionTestsMixin: @@ -48,6 +49,7 @@ def test_to_clipboard(self): psser.to_clipboard(sep=",", index=False), pser.to_clipboard(sep=",", index=False) ) + @unittest.skipIf(not have_jinja2, jinja2_requirement_message) def test_to_latex(self): pser = self.pser psser = self.psser From 9087d84b51278fc8e7429ab5ec27b73254cf60ad Mon Sep 17 00:00:00 2001 From: vicennial Date: Fri, 3 Jan 2025 09:12:57 +0900 Subject: [PATCH 20/60] [SPARK-50710][CONNECT] Add support for optional client reconnection to sessions after release ### What changes were proposed in this pull request? Adds a new boolean `allow_reconnect` field to `ReleaseSessionRequest`. When set to `true` in the request, the server will not place the session in the `closedSessionsCache` of `SparkConnectSessionManager`. The session's clean-up process is unmodified. ### Why are the changes needed? Currently, the connect server will, by default, tombstone all sessions that have either been released explicitly (through a `ReleaseSession` request) or cleaned up due to inactivity/idleness in periodic checks. Tombstoning prevents clients from reconnecting with the same `userId` and `sessionId`. This mechanism ensures that clients do not accidentally end up with a 'fresh' server-side session, which may be disastrous/fatal as all previously held state is lost (e.g., Temporary views, temporary UDFs, modified configs, current catalog, etc.). Consider a client that runs simple non-state dependant queries (e.g `select count from ...`), perhaps sparsely during the lifetime of the application. Such a client may prefer to opt out of tombstoning for the following reasons: - Queries do not depend on any custom server-side state. - Modifying `userId`/`sessionId` on each reconnect may be inconvenient for tracking/observability purposes. - On resource-constrained servers, clients may want to minimize their memory footprint by explicitly releasing their state, especially when they believe their requests are sparsely spread out. Currently, the only way to allow clients to reconnect is to set `spark.connect.session.manager.closedSessionsTombstonesSize` to `0`. However, this is not ideal as it would allow all clients to reconnect, which as previously pointed out, may be dangerous. As an improvement, allowing specific clients to explicitly signal/request the reconnection possibility addresses the needs mentioned earlier. ### Does this PR introduce _any_ user-facing change? Yes. When the client releases a session with `allow_reconnect` set to `true`, a reconnection will lead to the server generation a fresh session and not result in an error like `[INVALID_HANDLE.SESSION_CLOSED] The handle 271dab46-a9a0-4458-ad3a-71442eaa9a21 is invalid. Session was closed. SQLSTATE: HY000` Full example (gRPC based): #### Default/`allow_reconnect` set to `false` Create a session via a `Config` request: ```gRPC { "operation": { "get": { "keys": ["spark.sql.ansi.enabled"] } }, "session_id": "271dab46-a9a0-4458-ad3a-71442eaa9a21", "user_context": { "user_id": "vicennial", "user_name": "Akhil" } } ``` Release session via `ReleaseSession` request: ``` { "session_id": "271dab46-a9a0-4458-ad3a-71442eaa9a21", "user_context": { "user_id": "vicennial", "user_name": "Akhil" }, "allow_reconnect": false } ``` Retry the earlier config request, the error `[INVALID_HANDLE.SESSION_CLOSED] The handle 271dab46-a9a0-4458-ad3a-71442eaa9a21 is invalid. Session was closed. SQLSTATE: HY000` is hit. #### Default/`allow_reconnect` set to `true` Create a session via a `Config` request: ```gRPC { "operation": { "get": { "keys": ["spark.sql.ansi.enabled"] } }, "session_id": "ff1410b9-0a75-4634-820a-b46c14f30896", "user_context": { "user_id": "vicennial", "user_name": "Akhil" } } ``` Release session via `ReleaseSession` request: ``` { "session_id": "ff1410b9-0a75-4634-820a-b46c14f30896", "user_context": { "user_id": "vicennial", "user_name": "Akhil" }, "allow_reconnect": true } ``` Retry the earlier config request, the request goes through and it can be noted the `server_side_session_id` in the response of the last config request is different from the first one as a new server side session was generated. ### How was this patch tested? New unit test + existing tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49342 from vicennial/allowReconnect. Authored-by: vicennial Signed-off-by: Hyukjin Kwon --- python/pyspark/sql/connect/proto/base_pb2.py | 48 +++++++++---------- python/pyspark/sql/connect/proto/base_pb2.pyi | 18 +++++++ .../main/protobuf/spark/connect/base.proto | 14 ++++++ .../SparkConnectReleaseSessionHandler.scala | 3 +- .../service/SparkConnectSessionManager.scala | 30 +++++++----- .../sql/connect/SparkConnectServerTest.scala | 11 +++++ .../service/SparkConnectServiceE2ESuite.scala | 22 +++++++++ 7 files changed, 110 insertions(+), 36 deletions(-) diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 97694c33abeb2..6e946a5bd4ae7 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xab\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x42\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -227,31 +227,31 @@ _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13786 _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13951 _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13954 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14125 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14127 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14235 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14238 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14570 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14573 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16128 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14802 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 14976 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 14979 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15347 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15310 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15347 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15350 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15759 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14166 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14168 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14276 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14279 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14611 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14614 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16169 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14843 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15017 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15020 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15388 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15351 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15388 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15391 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15800 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 15661 + ]._serialized_start = 15702 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 15729 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15762 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16109 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16130 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16220 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16223 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17169 + ]._serialized_end = 15770 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15803 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16150 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16171 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16261 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16264 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17210 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index 253f8a58166a6..fc3a7e804f278 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -3216,6 +3216,7 @@ class ReleaseSessionRequest(google.protobuf.message.Message): SESSION_ID_FIELD_NUMBER: builtins.int USER_CONTEXT_FIELD_NUMBER: builtins.int CLIENT_TYPE_FIELD_NUMBER: builtins.int + ALLOW_RECONNECT_FIELD_NUMBER: builtins.int session_id: builtins.str """(Required) @@ -3234,12 +3235,27 @@ class ReleaseSessionRequest(google.protobuf.message.Message): can be used for language or version specific information and is only intended for logging purposes and will not be interpreted by the server. """ + allow_reconnect: builtins.bool + """Signals the server to allow the client to reconnect to the session after it is released. + + By default, the server tombstones the session upon release, preventing reconnections and + fully cleaning the session state. + + If this flag is set to true, the server may permit the client to reconnect to the session + post-release, even if the session state has been cleaned. This can result in missing state, + such as Temporary Views, Temporary UDFs, or the Current Catalog, in the reconnected session. + + Use this option sparingly and only when the client fully understands the implications of + reconnecting to a released session. The client must ensure that any queries executed do not + rely on the session state prior to its release. + """ def __init__( self, *, session_id: builtins.str = ..., user_context: global___UserContext | None = ..., client_type: builtins.str | None = ..., + allow_reconnect: builtins.bool = ..., ) -> None: ... def HasField( self, @@ -3257,6 +3273,8 @@ class ReleaseSessionRequest(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "_client_type", b"_client_type", + "allow_reconnect", + b"allow_reconnect", "client_type", b"client_type", "session_id", diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index e27049d2114d7..c308c7e21b66d 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -924,6 +924,20 @@ message ReleaseSessionRequest { // can be used for language or version specific information and is only intended for // logging purposes and will not be interpreted by the server. optional string client_type = 3; + + // Signals the server to allow the client to reconnect to the session after it is released. + // + // By default, the server tombstones the session upon release, preventing reconnections and + // fully cleaning the session state. + // + // If this flag is set to true, the server may permit the client to reconnect to the session + // post-release, even if the session state has been cleaned. This can result in missing state, + // such as Temporary Views, Temporary UDFs, or the Current Catalog, in the reconnected session. + // + // Use this option sparingly and only when the client fully understands the implications of + // reconnecting to a released session. The client must ensure that any queries executed do not + // rely on the session state prior to its release. + bool allow_reconnect = 4; } // Next ID: 3 diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseSessionHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseSessionHandler.scala index ec7a7f3bd242c..c36f07fc67f8f 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseSessionHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectReleaseSessionHandler.scala @@ -37,7 +37,8 @@ class SparkConnectReleaseSessionHandler( val maybeSession = SparkConnectService.sessionManager.getIsolatedSessionIfPresent(key) maybeSession.foreach(f => responseBuilder.setServerSideSessionId(f.serverSessionId)) - SparkConnectService.sessionManager.closeSession(key) + val allowReconnect = v.getAllowReconnect + SparkConnectService.sessionManager.closeSession(key, allowReconnect) responseObserver.onNext(responseBuilder.build()) responseObserver.onCompleted() diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala index b0b74a36e187b..c59fd02a829ae 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectSessionManager.scala @@ -134,7 +134,9 @@ class SparkConnectSessionManager extends Logging { } // Removes session from sessionStore and returns it. - private def removeSessionHolder(key: SessionKey): Option[SessionHolder] = { + private def removeSessionHolder( + key: SessionKey, + allowReconnect: Boolean = false): Option[SessionHolder] = { var sessionHolder: Option[SessionHolder] = None // The session holder should remain in the session store until it is added to the closed session @@ -144,9 +146,11 @@ class SparkConnectSessionManager extends Logging { sessionHolder = Option(sessionStore.get(key)) sessionHolder.foreach { s => - // Put into closedSessionsCache to prevent the same session from being recreated by - // getOrCreateIsolatedSession. - closedSessionsCache.put(s.key, s.getSessionHolderInfo) + if (!allowReconnect) { + // Put into closedSessionsCache to prevent the same session from being recreated by + // getOrCreateIsolatedSession when reconnection isn't allowed. + closedSessionsCache.put(s.key, s.getSessionHolderInfo) + } // Then, remove the session holder from the session store. sessionStore.remove(key) @@ -154,17 +158,21 @@ class SparkConnectSessionManager extends Logging { sessionHolder } - // Shut downs the session after removing. - private def shutdownSessionHolder(sessionHolder: SessionHolder): Unit = { + // Shuts down the session after removing. + private def shutdownSessionHolder( + sessionHolder: SessionHolder, + allowReconnect: Boolean = false): Unit = { sessionHolder.close() - // Update in closedSessionsCache: above it wasn't updated with closedTime etc. yet. - closedSessionsCache.put(sessionHolder.key, sessionHolder.getSessionHolderInfo) + if (!allowReconnect) { + // Update in closedSessionsCache: above it wasn't updated with closedTime etc. yet. + closedSessionsCache.put(sessionHolder.key, sessionHolder.getSessionHolderInfo) + } } - def closeSession(key: SessionKey): Unit = { - val sessionHolder = removeSessionHolder(key) + def closeSession(key: SessionKey, allowReconnect: Boolean = false): Unit = { + val sessionHolder = removeSessionHolder(key, allowReconnect) // Rest of the cleanup: the session cannot be accessed anymore by getOrCreateIsolatedSession. - sessionHolder.foreach(shutdownSessionHolder(_)) + sessionHolder.foreach(shutdownSessionHolder(_, allowReconnect)) } private[connect] def shutdown(): Unit = { diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala index b04c42a730785..3c857554dc756 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/SparkConnectServerTest.scala @@ -128,6 +128,17 @@ trait SparkConnectServerTest extends SharedSparkSession { req.build() } + protected def buildReleaseSessionRequest( + sessionId: String = defaultSessionId, + allowReconnect: Boolean = false) = { + proto.ReleaseSessionRequest + .newBuilder() + .setUserContext(userContext) + .setSessionId(sessionId) + .setAllowReconnect(allowReconnect) + .build() + } + protected def buildPlan(query: String) = { proto.Plan.newBuilder().setRoot(dsl.sql(query)).build() } diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala index f86298a8b5b98..f24560259a883 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/service/SparkConnectServiceE2ESuite.scala @@ -245,4 +245,26 @@ class SparkConnectServiceE2ESuite extends SparkConnectServerTest { assert(queryError.getMessage.contains("INVALID_HANDLE.SESSION_CHANGED")) } } + + test("Client is allowed to reconnect to released session if allow_reconnect is set") { + withRawBlockingStub { stub => + val sessionId = UUID.randomUUID.toString() + val iter = + stub.executePlan( + buildExecutePlanRequest( + buildPlan("select * from range(1000000)"), + sessionId = sessionId)) + iter.hasNext // guarantees the request was received by server. + + stub.releaseSession(buildReleaseSessionRequest(sessionId, allowReconnect = true)) + + val iter2 = + stub.executePlan( + buildExecutePlanRequest( + buildPlan("select * from range(1000000)"), + sessionId = sessionId)) + // guarantees the request was received by server. No exception should be thrown on reuse + iter2.hasNext + } + } } From 032e53180d3776fad56e6b3e8279fb447d3c9141 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 3 Jan 2025 09:16:18 +0900 Subject: [PATCH 21/60] [SPARK-50311][TESTS][FOLLOWUP] Fix test name properly ### What changes were proposed in this pull request? This is minor followup for https://github.com/apache/spark/pull/48843 to fix test name properly from Python file ### Why are the changes needed? Use proper test name ### Does this PR introduce _any_ user-facing change? No, it's test only ### How was this patch tested? CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #49199 from itholic/SPARK-50311-followup-2. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- .../pyspark/sql/tests/connect/test_parity_job_cancellation.py | 2 +- python/pyspark/sql/tests/test_job_cancellation.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py index c7f50495af15b..c5184b04d6aa5 100644 --- a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py +++ b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py @@ -57,7 +57,7 @@ def test_interrupt_all(self): if __name__ == "__main__": import unittest - from pyspark.sql.tests.connect.test_parity_serde import * # noqa: F401 + from pyspark.sql.tests.connect.test_parity_job_cancellation import * # noqa: F401 try: import xmlrunner # type: ignore[import] diff --git a/python/pyspark/sql/tests/test_job_cancellation.py b/python/pyspark/sql/tests/test_job_cancellation.py index 6fc404b0a0b3a..a046c9c01811b 100644 --- a/python/pyspark/sql/tests/test_job_cancellation.py +++ b/python/pyspark/sql/tests/test_job_cancellation.py @@ -172,7 +172,7 @@ class JobCancellationTests(JobCancellationTestsMixin, ReusedSQLTestCase): if __name__ == "__main__": - from pyspark.sql.tests.test_session import * # noqa: F401 + from pyspark.sql.tests.test_job_cancellation import * # noqa: F401 try: import xmlrunner From fa0c9954aa4d4281c31aaaf26b94e699699bb469 Mon Sep 17 00:00:00 2001 From: cashmand Date: Fri, 3 Jan 2025 09:10:42 +0800 Subject: [PATCH 22/60] [SPARK-50614][SQL] Fix shredding debug message ### What changes were proposed in this pull request? A debug message added in https://github.com/apache/spark/pull/49234 was missing a brace. As a result, the message was printing the non-pretty string representation of the struct, followed by the string `.prettyJson}`. This PR fixes it. ### Why are the changes needed? Cleaner debug messages. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Manually verified in spark-shell that the message looks wrong without the change, and correct with the change. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49345 from cashmand/fix_debug_msg. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../sql/execution/datasources/parquet/ParquetWriteSupport.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 9402f5638094d..02b432f98d7d8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -158,7 +158,7 @@ class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { s"""Initialized Parquet WriteSupport with Catalyst schema: |${schema.prettyJson} |and shredding schema: - |$shreddedSchema.prettyJson} + |${shreddedSchema.prettyJson} |and corresponding Parquet message type: |$messageType """.stripMargin) From a3a8d1ca668aae61f672c84a8873960edb8ab37b Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 3 Jan 2025 12:40:07 +0800 Subject: [PATCH 23/60] [SPARK-50715][PYTHON][CONNECT] `SparkSession.Builder` sets the configs in batch ### What changes were proposed in this pull request? `SparkSession.Builder` sets the configs in batch ### Why are the changes needed? I notice that there are practice workflows with 500+ configs, existing implementation always set the configs one by one, and thus cause 500+ `Config` RPCs. ### Does this PR introduce _any_ user-facing change? No, internal change ### How was this patch tested? Existing CI ### Was this patch authored or co-authored using generative AI tooling? No Closes #49346 from zhengruifeng/py_connect_set_all. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/connect/conf.py | 14 ++ python/pyspark/sql/connect/proto/base_pb2.py | 162 +++++++++--------- python/pyspark/sql/connect/proto/base_pb2.pyi | 17 +- python/pyspark/sql/connect/session.py | 40 ++--- .../main/protobuf/spark/connect/base.proto | 3 + .../service/SparkConnectConfigHandler.scala | 14 +- 6 files changed, 142 insertions(+), 108 deletions(-) diff --git a/python/pyspark/sql/connect/conf.py b/python/pyspark/sql/connect/conf.py index 1ef72ee3cfa43..84d7ad34fb360 100644 --- a/python/pyspark/sql/connect/conf.py +++ b/python/pyspark/sql/connect/conf.py @@ -49,6 +49,20 @@ def set(self, key: str, value: Union[str, int, bool]) -> None: set.__doc__ = PySparkRuntimeConfig.set.__doc__ + def _set_all(self, configs: Dict[str, Union[str, int, bool]], silent: bool) -> None: + conf_list = [] + for key, value in configs.items(): + if isinstance(value, bool): + value = "true" if value else "false" + elif isinstance(value, int): + value = str(value) + conf_list.append(proto.KeyValue(key=key, value=value)) + op_set = proto.ConfigRequest.Set(pairs=conf_list, silent=silent) + operation = proto.ConfigRequest.Operation(set=op_set) + result = self._client.config(operation) + for warn in result.warnings: + warnings.warn(warn) + def get( self, key: str, default: Union[Optional[str], _NoValueType] = _NoValue ) -> Optional[str]: diff --git a/python/pyspark/sql/connect/proto/base_pb2.py b/python/pyspark/sql/connect/proto/base_pb2.py index 6e946a5bd4ae7..64c549ffe9565 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.py +++ b/python/pyspark/sql/connect/proto/base_pb2.py @@ -43,7 +43,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\x87\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\x34\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x18spark/connect/base.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x1cspark/connect/commands.proto\x1a\x1aspark/connect/common.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/relations.proto\x1a\x19spark/connect/types.proto"t\n\x04Plan\x12-\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x04root\x12\x32\n\x07\x63ommand\x18\x02 \x01(\x0b\x32\x16.spark.connect.CommandH\x00R\x07\x63ommandB\t\n\x07op_type"z\n\x0bUserContext\x12\x17\n\x07user_id\x18\x01 \x01(\tR\x06userId\x12\x1b\n\tuser_name\x18\x02 \x01(\tR\x08userName\x12\x35\n\nextensions\x18\xe7\x07 \x03(\x0b\x32\x14.google.protobuf.AnyR\nextensions"\xf5\x14\n\x12\x41nalyzePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x11 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12\x42\n\x06schema\x18\x04 \x01(\x0b\x32(.spark.connect.AnalyzePlanRequest.SchemaH\x00R\x06schema\x12\x45\n\x07\x65xplain\x18\x05 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.ExplainH\x00R\x07\x65xplain\x12O\n\x0btree_string\x18\x06 \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.TreeStringH\x00R\ntreeString\x12\x46\n\x08is_local\x18\x07 \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.IsLocalH\x00R\x07isLocal\x12R\n\x0cis_streaming\x18\x08 \x01(\x0b\x32-.spark.connect.AnalyzePlanRequest.IsStreamingH\x00R\x0bisStreaming\x12O\n\x0binput_files\x18\t \x01(\x0b\x32,.spark.connect.AnalyzePlanRequest.InputFilesH\x00R\ninputFiles\x12U\n\rspark_version\x18\n \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SparkVersionH\x00R\x0csparkVersion\x12I\n\tddl_parse\x18\x0b \x01(\x0b\x32*.spark.connect.AnalyzePlanRequest.DDLParseH\x00R\x08\x64\x64lParse\x12X\n\x0esame_semantics\x18\x0c \x01(\x0b\x32/.spark.connect.AnalyzePlanRequest.SameSemanticsH\x00R\rsameSemantics\x12U\n\rsemantic_hash\x18\r \x01(\x0b\x32..spark.connect.AnalyzePlanRequest.SemanticHashH\x00R\x0csemanticHash\x12\x45\n\x07persist\x18\x0e \x01(\x0b\x32).spark.connect.AnalyzePlanRequest.PersistH\x00R\x07persist\x12K\n\tunpersist\x18\x0f \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.UnpersistH\x00R\tunpersist\x12_\n\x11get_storage_level\x18\x10 \x01(\x0b\x32\x31.spark.connect.AnalyzePlanRequest.GetStorageLevelH\x00R\x0fgetStorageLevel\x12M\n\x0bjson_to_ddl\x18\x12 \x01(\x0b\x32+.spark.connect.AnalyzePlanRequest.JsonToDDLH\x00R\tjsonToDdl\x1a\x31\n\x06Schema\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\xbb\x02\n\x07\x45xplain\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12X\n\x0c\x65xplain_mode\x18\x02 \x01(\x0e\x32\x35.spark.connect.AnalyzePlanRequest.Explain.ExplainModeR\x0b\x65xplainMode"\xac\x01\n\x0b\x45xplainMode\x12\x1c\n\x18\x45XPLAIN_MODE_UNSPECIFIED\x10\x00\x12\x17\n\x13\x45XPLAIN_MODE_SIMPLE\x10\x01\x12\x19\n\x15\x45XPLAIN_MODE_EXTENDED\x10\x02\x12\x18\n\x14\x45XPLAIN_MODE_CODEGEN\x10\x03\x12\x15\n\x11\x45XPLAIN_MODE_COST\x10\x04\x12\x1a\n\x16\x45XPLAIN_MODE_FORMATTED\x10\x05\x1aZ\n\nTreeString\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12\x19\n\x05level\x18\x02 \x01(\x05H\x00R\x05level\x88\x01\x01\x42\x08\n\x06_level\x1a\x32\n\x07IsLocal\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x36\n\x0bIsStreaming\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x35\n\nInputFiles\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x0e\n\x0cSparkVersion\x1a)\n\x08\x44\x44LParse\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlString\x1ay\n\rSameSemantics\x12\x34\n\x0btarget_plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\ntargetPlan\x12\x32\n\nother_plan\x18\x02 \x01(\x0b\x32\x13.spark.connect.PlanR\totherPlan\x1a\x37\n\x0cSemanticHash\x12\'\n\x04plan\x18\x01 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x1a\x97\x01\n\x07Persist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x45\n\rstorage_level\x18\x02 \x01(\x0b\x32\x1b.spark.connect.StorageLevelH\x00R\x0cstorageLevel\x88\x01\x01\x42\x10\n\x0e_storage_level\x1an\n\tUnpersist\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x12\x1f\n\x08\x62locking\x18\x02 \x01(\x08H\x00R\x08\x62locking\x88\x01\x01\x42\x0b\n\t_blocking\x1a\x46\n\x0fGetStorageLevel\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1a,\n\tJsonToDDL\x12\x1f\n\x0bjson_string\x18\x01 \x01(\tR\njsonStringB\t\n\x07\x61nalyzeB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xca\x0e\n\x13\x41nalyzePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12\x43\n\x06schema\x18\x02 \x01(\x0b\x32).spark.connect.AnalyzePlanResponse.SchemaH\x00R\x06schema\x12\x46\n\x07\x65xplain\x18\x03 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.ExplainH\x00R\x07\x65xplain\x12P\n\x0btree_string\x18\x04 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.TreeStringH\x00R\ntreeString\x12G\n\x08is_local\x18\x05 \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.IsLocalH\x00R\x07isLocal\x12S\n\x0cis_streaming\x18\x06 \x01(\x0b\x32..spark.connect.AnalyzePlanResponse.IsStreamingH\x00R\x0bisStreaming\x12P\n\x0binput_files\x18\x07 \x01(\x0b\x32-.spark.connect.AnalyzePlanResponse.InputFilesH\x00R\ninputFiles\x12V\n\rspark_version\x18\x08 \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SparkVersionH\x00R\x0csparkVersion\x12J\n\tddl_parse\x18\t \x01(\x0b\x32+.spark.connect.AnalyzePlanResponse.DDLParseH\x00R\x08\x64\x64lParse\x12Y\n\x0esame_semantics\x18\n \x01(\x0b\x32\x30.spark.connect.AnalyzePlanResponse.SameSemanticsH\x00R\rsameSemantics\x12V\n\rsemantic_hash\x18\x0b \x01(\x0b\x32/.spark.connect.AnalyzePlanResponse.SemanticHashH\x00R\x0csemanticHash\x12\x46\n\x07persist\x18\x0c \x01(\x0b\x32*.spark.connect.AnalyzePlanResponse.PersistH\x00R\x07persist\x12L\n\tunpersist\x18\r \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.UnpersistH\x00R\tunpersist\x12`\n\x11get_storage_level\x18\x0e \x01(\x0b\x32\x32.spark.connect.AnalyzePlanResponse.GetStorageLevelH\x00R\x0fgetStorageLevel\x12N\n\x0bjson_to_ddl\x18\x10 \x01(\x0b\x32,.spark.connect.AnalyzePlanResponse.JsonToDDLH\x00R\tjsonToDdl\x1a\x39\n\x06Schema\x12/\n\x06schema\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1a\x30\n\x07\x45xplain\x12%\n\x0e\x65xplain_string\x18\x01 \x01(\tR\rexplainString\x1a-\n\nTreeString\x12\x1f\n\x0btree_string\x18\x01 \x01(\tR\ntreeString\x1a$\n\x07IsLocal\x12\x19\n\x08is_local\x18\x01 \x01(\x08R\x07isLocal\x1a\x30\n\x0bIsStreaming\x12!\n\x0cis_streaming\x18\x01 \x01(\x08R\x0bisStreaming\x1a"\n\nInputFiles\x12\x14\n\x05\x66iles\x18\x01 \x03(\tR\x05\x66iles\x1a(\n\x0cSparkVersion\x12\x18\n\x07version\x18\x01 \x01(\tR\x07version\x1a;\n\x08\x44\x44LParse\x12/\n\x06parsed\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06parsed\x1a\'\n\rSameSemantics\x12\x16\n\x06result\x18\x01 \x01(\x08R\x06result\x1a&\n\x0cSemanticHash\x12\x16\n\x06result\x18\x01 \x01(\x05R\x06result\x1a\t\n\x07Persist\x1a\x0b\n\tUnpersist\x1aS\n\x0fGetStorageLevel\x12@\n\rstorage_level\x18\x01 \x01(\x0b\x32\x1b.spark.connect.StorageLevelR\x0cstorageLevel\x1a*\n\tJsonToDDL\x12\x1d\n\nddl_string\x18\x01 \x01(\tR\tddlStringB\x08\n\x06result"\xa3\x05\n\x12\x45xecutePlanRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12&\n\x0coperation_id\x18\x06 \x01(\tH\x01R\x0boperationId\x88\x01\x01\x12\'\n\x04plan\x18\x03 \x01(\x0b\x32\x13.spark.connect.PlanR\x04plan\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12X\n\x0frequest_options\x18\x05 \x03(\x0b\x32/.spark.connect.ExecutePlanRequest.RequestOptionR\x0erequestOptions\x12\x12\n\x04tags\x18\x07 \x03(\tR\x04tags\x1a\xa5\x01\n\rRequestOption\x12K\n\x10reattach_options\x18\x01 \x01(\x0b\x32\x1e.spark.connect.ReattachOptionsH\x00R\x0freattachOptions\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textensionB\x10\n\x0erequest_optionB)\n\'_client_observed_server_side_session_idB\x0f\n\r_operation_idB\x0e\n\x0c_client_type"\xe6\x16\n\x13\x45xecutePlanResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x0f \x01(\tR\x13serverSideSessionId\x12!\n\x0coperation_id\x18\x0c \x01(\tR\x0boperationId\x12\x1f\n\x0bresponse_id\x18\r \x01(\tR\nresponseId\x12P\n\x0b\x61rrow_batch\x18\x02 \x01(\x0b\x32-.spark.connect.ExecutePlanResponse.ArrowBatchH\x00R\narrowBatch\x12\x63\n\x12sql_command_result\x18\x05 \x01(\x0b\x32\x33.spark.connect.ExecutePlanResponse.SqlCommandResultH\x00R\x10sqlCommandResult\x12~\n#write_stream_operation_start_result\x18\x08 \x01(\x0b\x32..spark.connect.WriteStreamOperationStartResultH\x00R\x1fwriteStreamOperationStartResult\x12q\n\x1estreaming_query_command_result\x18\t \x01(\x0b\x32*.spark.connect.StreamingQueryCommandResultH\x00R\x1bstreamingQueryCommandResult\x12k\n\x1cget_resources_command_result\x18\n \x01(\x0b\x32(.spark.connect.GetResourcesCommandResultH\x00R\x19getResourcesCommandResult\x12\x87\x01\n&streaming_query_manager_command_result\x18\x0b \x01(\x0b\x32\x31.spark.connect.StreamingQueryManagerCommandResultH\x00R"streamingQueryManagerCommandResult\x12\x87\x01\n&streaming_query_listener_events_result\x18\x10 \x01(\x0b\x32\x31.spark.connect.StreamingQueryListenerEventsResultH\x00R"streamingQueryListenerEventsResult\x12\\\n\x0fresult_complete\x18\x0e \x01(\x0b\x32\x31.spark.connect.ExecutePlanResponse.ResultCompleteH\x00R\x0eresultComplete\x12\x87\x01\n&create_resource_profile_command_result\x18\x11 \x01(\x0b\x32\x31.spark.connect.CreateResourceProfileCommandResultH\x00R"createResourceProfileCommandResult\x12\x65\n\x12\x65xecution_progress\x18\x12 \x01(\x0b\x32\x34.spark.connect.ExecutePlanResponse.ExecutionProgressH\x00R\x11\x65xecutionProgress\x12\x64\n\x19\x63heckpoint_command_result\x18\x13 \x01(\x0b\x32&.spark.connect.CheckpointCommandResultH\x00R\x17\x63heckpointCommandResult\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x12\x44\n\x07metrics\x18\x04 \x01(\x0b\x32*.spark.connect.ExecutePlanResponse.MetricsR\x07metrics\x12]\n\x10observed_metrics\x18\x06 \x03(\x0b\x32\x32.spark.connect.ExecutePlanResponse.ObservedMetricsR\x0fobservedMetrics\x12/\n\x06schema\x18\x07 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema\x1aG\n\x10SqlCommandResult\x12\x33\n\x08relation\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x08relation\x1av\n\nArrowBatch\x12\x1b\n\trow_count\x18\x01 \x01(\x03R\x08rowCount\x12\x12\n\x04\x64\x61ta\x18\x02 \x01(\x0cR\x04\x64\x61ta\x12&\n\x0cstart_offset\x18\x03 \x01(\x03H\x00R\x0bstartOffset\x88\x01\x01\x42\x0f\n\r_start_offset\x1a\x85\x04\n\x07Metrics\x12Q\n\x07metrics\x18\x01 \x03(\x0b\x32\x37.spark.connect.ExecutePlanResponse.Metrics.MetricObjectR\x07metrics\x1a\xcc\x02\n\x0cMetricObject\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x17\n\x07plan_id\x18\x02 \x01(\x03R\x06planId\x12\x16\n\x06parent\x18\x03 \x01(\x03R\x06parent\x12z\n\x11\x65xecution_metrics\x18\x04 \x03(\x0b\x32M.spark.connect.ExecutePlanResponse.Metrics.MetricObject.ExecutionMetricsEntryR\x10\x65xecutionMetrics\x1a{\n\x15\x45xecutionMetricsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ExecutePlanResponse.Metrics.MetricValueR\x05value:\x02\x38\x01\x1aX\n\x0bMetricValue\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x14\n\x05value\x18\x02 \x01(\x03R\x05value\x12\x1f\n\x0bmetric_type\x18\x03 \x01(\tR\nmetricType\x1a\x8d\x01\n\x0fObservedMetrics\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x12\x12\n\x04keys\x18\x03 \x03(\tR\x04keys\x12\x17\n\x07plan_id\x18\x04 \x01(\x03R\x06planId\x1a\x10\n\x0eResultComplete\x1a\xcd\x02\n\x11\x45xecutionProgress\x12V\n\x06stages\x18\x01 \x03(\x0b\x32>.spark.connect.ExecutePlanResponse.ExecutionProgress.StageInfoR\x06stages\x12,\n\x12num_inflight_tasks\x18\x02 \x01(\x03R\x10numInflightTasks\x1a\xb1\x01\n\tStageInfo\x12\x19\n\x08stage_id\x18\x01 \x01(\x03R\x07stageId\x12\x1b\n\tnum_tasks\x18\x02 \x01(\x03R\x08numTasks\x12.\n\x13num_completed_tasks\x18\x03 \x01(\x03R\x11numCompletedTasks\x12(\n\x10input_bytes_read\x18\x04 \x01(\x03R\x0einputBytesRead\x12\x12\n\x04\x64one\x18\x05 \x01(\x08R\x04\x64oneB\x0f\n\rresponse_type"A\n\x08KeyValue\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x19\n\x05value\x18\x02 \x01(\tH\x00R\x05value\x88\x01\x01\x42\x08\n\x06_value"\xaf\t\n\rConfigRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x08 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x44\n\toperation\x18\x03 \x01(\x0b\x32&.spark.connect.ConfigRequest.OperationR\toperation\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x1a\xf2\x03\n\tOperation\x12\x34\n\x03set\x18\x01 \x01(\x0b\x32 .spark.connect.ConfigRequest.SetH\x00R\x03set\x12\x34\n\x03get\x18\x02 \x01(\x0b\x32 .spark.connect.ConfigRequest.GetH\x00R\x03get\x12W\n\x10get_with_default\x18\x03 \x01(\x0b\x32+.spark.connect.ConfigRequest.GetWithDefaultH\x00R\x0egetWithDefault\x12G\n\nget_option\x18\x04 \x01(\x0b\x32&.spark.connect.ConfigRequest.GetOptionH\x00R\tgetOption\x12>\n\x07get_all\x18\x05 \x01(\x0b\x32#.spark.connect.ConfigRequest.GetAllH\x00R\x06getAll\x12:\n\x05unset\x18\x06 \x01(\x0b\x32".spark.connect.ConfigRequest.UnsetH\x00R\x05unset\x12P\n\ris_modifiable\x18\x07 \x01(\x0b\x32).spark.connect.ConfigRequest.IsModifiableH\x00R\x0cisModifiableB\t\n\x07op_type\x1a\\\n\x03Set\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1b\n\x06silent\x18\x02 \x01(\x08H\x00R\x06silent\x88\x01\x01\x42\t\n\x07_silent\x1a\x19\n\x03Get\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a?\n\x0eGetWithDefault\x12-\n\x05pairs\x18\x01 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x1a\x1f\n\tGetOption\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a\x30\n\x06GetAll\x12\x1b\n\x06prefix\x18\x01 \x01(\tH\x00R\x06prefix\x88\x01\x01\x42\t\n\x07_prefix\x1a\x1b\n\x05Unset\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keys\x1a"\n\x0cIsModifiable\x12\x12\n\x04keys\x18\x01 \x03(\tR\x04keysB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xaf\x01\n\x0e\x43onfigResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x04 \x01(\tR\x13serverSideSessionId\x12-\n\x05pairs\x18\x02 \x03(\x0b\x32\x17.spark.connect.KeyValueR\x05pairs\x12\x1a\n\x08warnings\x18\x03 \x03(\tR\x08warnings"\xea\x07\n\x13\x41\x64\x64\x41rtifactsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12$\n\x0b\x63lient_type\x18\x06 \x01(\tH\x02R\nclientType\x88\x01\x01\x12@\n\x05\x62\x61tch\x18\x03 \x01(\x0b\x32(.spark.connect.AddArtifactsRequest.BatchH\x00R\x05\x62\x61tch\x12Z\n\x0b\x62\x65gin_chunk\x18\x04 \x01(\x0b\x32\x37.spark.connect.AddArtifactsRequest.BeginChunkedArtifactH\x00R\nbeginChunk\x12H\n\x05\x63hunk\x18\x05 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkH\x00R\x05\x63hunk\x1a\x35\n\rArtifactChunk\x12\x12\n\x04\x64\x61ta\x18\x01 \x01(\x0cR\x04\x64\x61ta\x12\x10\n\x03\x63rc\x18\x02 \x01(\x03R\x03\x63rc\x1ao\n\x13SingleChunkArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x44\n\x04\x64\x61ta\x18\x02 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x04\x64\x61ta\x1a]\n\x05\x42\x61tch\x12T\n\tartifacts\x18\x01 \x03(\x0b\x32\x36.spark.connect.AddArtifactsRequest.SingleChunkArtifactR\tartifacts\x1a\xc1\x01\n\x14\x42\x65ginChunkedArtifact\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1f\n\x0btotal_bytes\x18\x02 \x01(\x03R\ntotalBytes\x12\x1d\n\nnum_chunks\x18\x03 \x01(\x03R\tnumChunks\x12U\n\rinitial_chunk\x18\x04 \x01(\x0b\x32\x30.spark.connect.AddArtifactsRequest.ArtifactChunkR\x0cinitialChunkB\t\n\x07payloadB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x02\n\x14\x41\x64\x64\x41rtifactsResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\tartifacts\x18\x01 \x03(\x0b\x32\x33.spark.connect.AddArtifactsResponse.ArtifactSummaryR\tartifacts\x1aQ\n\x0f\x41rtifactSummary\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12*\n\x11is_crc_successful\x18\x02 \x01(\x08R\x0fisCrcSuccessful"\xc6\x02\n\x17\x41rtifactStatusesRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x01R\nclientType\x88\x01\x01\x12\x14\n\x05names\x18\x04 \x03(\tR\x05namesB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xe0\x02\n\x18\x41rtifactStatusesResponse\x12\x1d\n\nsession_id\x18\x02 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12Q\n\x08statuses\x18\x01 \x03(\x0b\x32\x35.spark.connect.ArtifactStatusesResponse.StatusesEntryR\x08statuses\x1as\n\rStatusesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12L\n\x05value\x18\x02 \x01(\x0b\x32\x36.spark.connect.ArtifactStatusesResponse.ArtifactStatusR\x05value:\x02\x38\x01\x1a(\n\x0e\x41rtifactStatus\x12\x16\n\x06\x65xists\x18\x01 \x01(\x08R\x06\x65xists"\xdb\x04\n\x10InterruptRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x02R\nclientType\x88\x01\x01\x12T\n\x0einterrupt_type\x18\x04 \x01(\x0e\x32-.spark.connect.InterruptRequest.InterruptTypeR\rinterruptType\x12%\n\roperation_tag\x18\x05 \x01(\tH\x00R\x0coperationTag\x12#\n\x0coperation_id\x18\x06 \x01(\tH\x00R\x0boperationId"\x80\x01\n\rInterruptType\x12\x1e\n\x1aINTERRUPT_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12INTERRUPT_TYPE_ALL\x10\x01\x12\x16\n\x12INTERRUPT_TYPE_TAG\x10\x02\x12\x1f\n\x1bINTERRUPT_TYPE_OPERATION_ID\x10\x03\x42\x0b\n\tinterruptB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x90\x01\n\x11InterruptResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\'\n\x0finterrupted_ids\x18\x02 \x03(\tR\x0einterruptedIds"5\n\x0fReattachOptions\x12"\n\x0creattachable\x18\x01 \x01(\x08R\x0creattachable"\x96\x03\n\x16ReattachExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x06 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x12-\n\x10last_response_id\x18\x05 \x01(\tH\x02R\x0elastResponseId\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_typeB\x13\n\x11_last_response_id"\xc9\x04\n\x15ReleaseExecuteRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x07 \x01(\tH\x01R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12!\n\x0coperation_id\x18\x03 \x01(\tR\x0boperationId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x02R\nclientType\x88\x01\x01\x12R\n\x0brelease_all\x18\x05 \x01(\x0b\x32/.spark.connect.ReleaseExecuteRequest.ReleaseAllH\x00R\nreleaseAll\x12X\n\rrelease_until\x18\x06 \x01(\x0b\x32\x31.spark.connect.ReleaseExecuteRequest.ReleaseUntilH\x00R\x0creleaseUntil\x1a\x0c\n\nReleaseAll\x1a/\n\x0cReleaseUntil\x12\x1f\n\x0bresponse_id\x18\x01 \x01(\tR\nresponseIdB\t\n\x07releaseB)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\xa5\x01\n\x16ReleaseExecuteResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12&\n\x0coperation_id\x18\x02 \x01(\tH\x00R\x0boperationId\x88\x01\x01\x42\x0f\n\r_operation_id"\xd4\x01\n\x15ReleaseSessionRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12$\n\x0b\x63lient_type\x18\x03 \x01(\tH\x00R\nclientType\x88\x01\x01\x12\'\n\x0f\x61llow_reconnect\x18\x04 \x01(\x08R\x0e\x61llowReconnectB\x0e\n\x0c_client_type"l\n\x16ReleaseSessionResponse\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12\x33\n\x16server_side_session_id\x18\x02 \x01(\tR\x13serverSideSessionId"\xcc\x02\n\x18\x46\x65tchErrorDetailsRequest\x12\x1d\n\nsession_id\x18\x01 \x01(\tR\tsessionId\x12V\n&client_observed_server_side_session_id\x18\x05 \x01(\tH\x00R!clientObservedServerSideSessionId\x88\x01\x01\x12=\n\x0cuser_context\x18\x02 \x01(\x0b\x32\x1a.spark.connect.UserContextR\x0buserContext\x12\x19\n\x08\x65rror_id\x18\x03 \x01(\tR\x07\x65rrorId\x12$\n\x0b\x63lient_type\x18\x04 \x01(\tH\x01R\nclientType\x88\x01\x01\x42)\n\'_client_observed_server_side_session_idB\x0e\n\x0c_client_type"\x93\x0c\n\x19\x46\x65tchErrorDetailsResponse\x12\x33\n\x16server_side_session_id\x18\x03 \x01(\tR\x13serverSideSessionId\x12\x1d\n\nsession_id\x18\x04 \x01(\tR\tsessionId\x12)\n\x0eroot_error_idx\x18\x01 \x01(\x05H\x00R\x0crootErrorIdx\x88\x01\x01\x12\x46\n\x06\x65rrors\x18\x02 \x03(\x0b\x32..spark.connect.FetchErrorDetailsResponse.ErrorR\x06\x65rrors\x1a\xae\x01\n\x11StackTraceElement\x12\'\n\x0f\x64\x65\x63laring_class\x18\x01 \x01(\tR\x0e\x64\x65\x63laringClass\x12\x1f\n\x0bmethod_name\x18\x02 \x01(\tR\nmethodName\x12 \n\tfile_name\x18\x03 \x01(\tH\x00R\x08\x66ileName\x88\x01\x01\x12\x1f\n\x0bline_number\x18\x04 \x01(\x05R\nlineNumberB\x0c\n\n_file_name\x1a\xf0\x02\n\x0cQueryContext\x12\x64\n\x0c\x63ontext_type\x18\n \x01(\x0e\x32\x41.spark.connect.FetchErrorDetailsResponse.QueryContext.ContextTypeR\x0b\x63ontextType\x12\x1f\n\x0bobject_type\x18\x01 \x01(\tR\nobjectType\x12\x1f\n\x0bobject_name\x18\x02 \x01(\tR\nobjectName\x12\x1f\n\x0bstart_index\x18\x03 \x01(\x05R\nstartIndex\x12\x1d\n\nstop_index\x18\x04 \x01(\x05R\tstopIndex\x12\x1a\n\x08\x66ragment\x18\x05 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x06 \x01(\tR\x08\x63\x61llSite\x12\x18\n\x07summary\x18\x07 \x01(\tR\x07summary"%\n\x0b\x43ontextType\x12\x07\n\x03SQL\x10\x00\x12\r\n\tDATAFRAME\x10\x01\x1a\x99\x03\n\x0eSparkThrowable\x12$\n\x0b\x65rror_class\x18\x01 \x01(\tH\x00R\nerrorClass\x88\x01\x01\x12}\n\x12message_parameters\x18\x02 \x03(\x0b\x32N.spark.connect.FetchErrorDetailsResponse.SparkThrowable.MessageParametersEntryR\x11messageParameters\x12\\\n\x0equery_contexts\x18\x03 \x03(\x0b\x32\x35.spark.connect.FetchErrorDetailsResponse.QueryContextR\rqueryContexts\x12 \n\tsql_state\x18\x04 \x01(\tH\x01R\x08sqlState\x88\x01\x01\x1a\x44\n\x16MessageParametersEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\x0e\n\x0c_error_classB\x0c\n\n_sql_state\x1a\xdb\x02\n\x05\x45rror\x12\x30\n\x14\x65rror_type_hierarchy\x18\x01 \x03(\tR\x12\x65rrorTypeHierarchy\x12\x18\n\x07message\x18\x02 \x01(\tR\x07message\x12[\n\x0bstack_trace\x18\x03 \x03(\x0b\x32:.spark.connect.FetchErrorDetailsResponse.StackTraceElementR\nstackTrace\x12 \n\tcause_idx\x18\x04 \x01(\x05H\x00R\x08\x63\x61useIdx\x88\x01\x01\x12\x65\n\x0fspark_throwable\x18\x05 \x01(\x0b\x32\x37.spark.connect.FetchErrorDetailsResponse.SparkThrowableH\x01R\x0esparkThrowable\x88\x01\x01\x42\x0c\n\n_cause_idxB\x12\n\x10_spark_throwableB\x11\n\x0f_root_error_idx"Z\n\x17\x43heckpointCommandResult\x12?\n\x08relation\x18\x01 \x01(\x0b\x32#.spark.connect.CachedRemoteRelationR\x08relation2\xb2\x07\n\x13SparkConnectService\x12X\n\x0b\x45xecutePlan\x12!.spark.connect.ExecutePlanRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12V\n\x0b\x41nalyzePlan\x12!.spark.connect.AnalyzePlanRequest\x1a".spark.connect.AnalyzePlanResponse"\x00\x12G\n\x06\x43onfig\x12\x1c.spark.connect.ConfigRequest\x1a\x1d.spark.connect.ConfigResponse"\x00\x12[\n\x0c\x41\x64\x64\x41rtifacts\x12".spark.connect.AddArtifactsRequest\x1a#.spark.connect.AddArtifactsResponse"\x00(\x01\x12\x63\n\x0e\x41rtifactStatus\x12&.spark.connect.ArtifactStatusesRequest\x1a\'.spark.connect.ArtifactStatusesResponse"\x00\x12P\n\tInterrupt\x12\x1f.spark.connect.InterruptRequest\x1a .spark.connect.InterruptResponse"\x00\x12`\n\x0fReattachExecute\x12%.spark.connect.ReattachExecuteRequest\x1a".spark.connect.ExecutePlanResponse"\x00\x30\x01\x12_\n\x0eReleaseExecute\x12$.spark.connect.ReleaseExecuteRequest\x1a%.spark.connect.ReleaseExecuteResponse"\x00\x12_\n\x0eReleaseSession\x12$.spark.connect.ReleaseSessionRequest\x1a%.spark.connect.ReleaseSessionResponse"\x00\x12h\n\x11\x46\x65tchErrorDetails\x12\'.spark.connect.FetchErrorDetailsRequest\x1a(.spark.connect.FetchErrorDetailsResponse"\x00\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -167,91 +167,91 @@ _globals["_KEYVALUE"]._serialized_start = 8609 _globals["_KEYVALUE"]._serialized_end = 8674 _globals["_CONFIGREQUEST"]._serialized_start = 8677 - _globals["_CONFIGREQUEST"]._serialized_end = 9836 + _globals["_CONFIGREQUEST"]._serialized_end = 9876 _globals["_CONFIGREQUEST_OPERATION"]._serialized_start = 8985 _globals["_CONFIGREQUEST_OPERATION"]._serialized_end = 9483 _globals["_CONFIGREQUEST_SET"]._serialized_start = 9485 - _globals["_CONFIGREQUEST_SET"]._serialized_end = 9537 - _globals["_CONFIGREQUEST_GET"]._serialized_start = 9539 - _globals["_CONFIGREQUEST_GET"]._serialized_end = 9564 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9566 - _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9629 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9631 - _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9662 - _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9664 - _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9712 - _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9714 - _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9741 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9743 - _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9777 - _globals["_CONFIGRESPONSE"]._serialized_start = 9839 - _globals["_CONFIGRESPONSE"]._serialized_end = 10014 - _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10017 - _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11019 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10492 - _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10545 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10547 - _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10658 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10660 - _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10753 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10756 - _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10949 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11022 - _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11294 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11213 - _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11294 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11297 - _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11623 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11626 - _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 11978 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11821 - _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11936 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11938 - _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 11978 - _globals["_INTERRUPTREQUEST"]._serialized_start = 11981 - _globals["_INTERRUPTREQUEST"]._serialized_end = 12584 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12384 - _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12512 - _globals["_INTERRUPTRESPONSE"]._serialized_start = 12587 - _globals["_INTERRUPTRESPONSE"]._serialized_end = 12731 - _globals["_REATTACHOPTIONS"]._serialized_start = 12733 - _globals["_REATTACHOPTIONS"]._serialized_end = 12786 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12789 - _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13195 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13198 - _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13783 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13652 - _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13664 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13666 - _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13713 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13786 - _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13951 - _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13954 - _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14166 - _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14168 - _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14276 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14279 - _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14611 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14614 - _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16169 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14843 - _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15017 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15020 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15388 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15351 - _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15388 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15391 - _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15800 + _globals["_CONFIGREQUEST_SET"]._serialized_end = 9577 + _globals["_CONFIGREQUEST_GET"]._serialized_start = 9579 + _globals["_CONFIGREQUEST_GET"]._serialized_end = 9604 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_start = 9606 + _globals["_CONFIGREQUEST_GETWITHDEFAULT"]._serialized_end = 9669 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_start = 9671 + _globals["_CONFIGREQUEST_GETOPTION"]._serialized_end = 9702 + _globals["_CONFIGREQUEST_GETALL"]._serialized_start = 9704 + _globals["_CONFIGREQUEST_GETALL"]._serialized_end = 9752 + _globals["_CONFIGREQUEST_UNSET"]._serialized_start = 9754 + _globals["_CONFIGREQUEST_UNSET"]._serialized_end = 9781 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_start = 9783 + _globals["_CONFIGREQUEST_ISMODIFIABLE"]._serialized_end = 9817 + _globals["_CONFIGRESPONSE"]._serialized_start = 9879 + _globals["_CONFIGRESPONSE"]._serialized_end = 10054 + _globals["_ADDARTIFACTSREQUEST"]._serialized_start = 10057 + _globals["_ADDARTIFACTSREQUEST"]._serialized_end = 11059 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_start = 10532 + _globals["_ADDARTIFACTSREQUEST_ARTIFACTCHUNK"]._serialized_end = 10585 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_start = 10587 + _globals["_ADDARTIFACTSREQUEST_SINGLECHUNKARTIFACT"]._serialized_end = 10698 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_start = 10700 + _globals["_ADDARTIFACTSREQUEST_BATCH"]._serialized_end = 10793 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_start = 10796 + _globals["_ADDARTIFACTSREQUEST_BEGINCHUNKEDARTIFACT"]._serialized_end = 10989 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_start = 11062 + _globals["_ADDARTIFACTSRESPONSE"]._serialized_end = 11334 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_start = 11253 + _globals["_ADDARTIFACTSRESPONSE_ARTIFACTSUMMARY"]._serialized_end = 11334 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_start = 11337 + _globals["_ARTIFACTSTATUSESREQUEST"]._serialized_end = 11663 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_start = 11666 + _globals["_ARTIFACTSTATUSESRESPONSE"]._serialized_end = 12018 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_start = 11861 + _globals["_ARTIFACTSTATUSESRESPONSE_STATUSESENTRY"]._serialized_end = 11976 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_start = 11978 + _globals["_ARTIFACTSTATUSESRESPONSE_ARTIFACTSTATUS"]._serialized_end = 12018 + _globals["_INTERRUPTREQUEST"]._serialized_start = 12021 + _globals["_INTERRUPTREQUEST"]._serialized_end = 12624 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_start = 12424 + _globals["_INTERRUPTREQUEST_INTERRUPTTYPE"]._serialized_end = 12552 + _globals["_INTERRUPTRESPONSE"]._serialized_start = 12627 + _globals["_INTERRUPTRESPONSE"]._serialized_end = 12771 + _globals["_REATTACHOPTIONS"]._serialized_start = 12773 + _globals["_REATTACHOPTIONS"]._serialized_end = 12826 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_start = 12829 + _globals["_REATTACHEXECUTEREQUEST"]._serialized_end = 13235 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_start = 13238 + _globals["_RELEASEEXECUTEREQUEST"]._serialized_end = 13823 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_start = 13692 + _globals["_RELEASEEXECUTEREQUEST_RELEASEALL"]._serialized_end = 13704 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_start = 13706 + _globals["_RELEASEEXECUTEREQUEST_RELEASEUNTIL"]._serialized_end = 13753 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_start = 13826 + _globals["_RELEASEEXECUTERESPONSE"]._serialized_end = 13991 + _globals["_RELEASESESSIONREQUEST"]._serialized_start = 13994 + _globals["_RELEASESESSIONREQUEST"]._serialized_end = 14206 + _globals["_RELEASESESSIONRESPONSE"]._serialized_start = 14208 + _globals["_RELEASESESSIONRESPONSE"]._serialized_end = 14316 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_start = 14319 + _globals["_FETCHERRORDETAILSREQUEST"]._serialized_end = 14651 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_start = 14654 + _globals["_FETCHERRORDETAILSRESPONSE"]._serialized_end = 16209 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_start = 14883 + _globals["_FETCHERRORDETAILSRESPONSE_STACKTRACEELEMENT"]._serialized_end = 15057 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_start = 15060 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT"]._serialized_end = 15428 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_start = 15391 + _globals["_FETCHERRORDETAILSRESPONSE_QUERYCONTEXT_CONTEXTTYPE"]._serialized_end = 15428 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_start = 15431 + _globals["_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE"]._serialized_end = 15840 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_start = 15702 + ]._serialized_start = 15742 _globals[ "_FETCHERRORDETAILSRESPONSE_SPARKTHROWABLE_MESSAGEPARAMETERSENTRY" - ]._serialized_end = 15770 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15803 - _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16150 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16171 - _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16261 - _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16264 - _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17210 + ]._serialized_end = 15810 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_start = 15843 + _globals["_FETCHERRORDETAILSRESPONSE_ERROR"]._serialized_end = 16190 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_start = 16211 + _globals["_CHECKPOINTCOMMANDRESULT"]._serialized_end = 16301 + _globals["_SPARKCONNECTSERVICE"]._serialized_start = 16304 + _globals["_SPARKCONNECTSERVICE"]._serialized_end = 17250 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/base_pb2.pyi b/python/pyspark/sql/connect/proto/base_pb2.pyi index fc3a7e804f278..adea62ed6b805 100644 --- a/python/pyspark/sql/connect/proto/base_pb2.pyi +++ b/python/pyspark/sql/connect/proto/base_pb2.pyi @@ -1921,17 +1921,32 @@ class ConfigRequest(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor PAIRS_FIELD_NUMBER: builtins.int + SILENT_FIELD_NUMBER: builtins.int @property def pairs( self, ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[global___KeyValue]: """(Required) The config key-value pairs to set.""" + silent: builtins.bool + """(Optional) Whether to ignore failures.""" def __init__( self, *, pairs: collections.abc.Iterable[global___KeyValue] | None = ..., + silent: builtins.bool | None = ..., ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["pairs", b"pairs"]) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["_silent", b"_silent", "silent", b"silent"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "_silent", b"_silent", "pairs", b"pairs", "silent", b"silent" + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_silent", b"_silent"] + ) -> typing_extensions.Literal["silent"] | None: ... class Get(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor diff --git a/python/pyspark/sql/connect/session.py b/python/pyspark/sql/connect/session.py index 3f1663d06850a..59349a17886bb 100644 --- a/python/pyspark/sql/connect/session.py +++ b/python/pyspark/sql/connect/session.py @@ -200,34 +200,26 @@ def _apply_options(self, session: "SparkSession") -> None: for i in range(int(os.environ.get("PYSPARK_REMOTE_INIT_CONF_LEN", "0"))): init_opts = json.loads(os.environ[f"PYSPARK_REMOTE_INIT_CONF_{i}"]) + # The options are applied after session creation, + # so options ["spark.remote", "spark.master"] always take no effect. + invalid_opts = ["spark.remote", "spark.master"] + with self._lock: + opts = {} + + # Only attempts to set Spark SQL configurations. + # If the configurations are static, it might throw an exception so + # simply ignore it for now. for k, v in init_opts.items(): - # the options are applied after session creation, - # so following options always take no effect - if k not in [ - "spark.remote", - "spark.master", - ] and k.startswith("spark.sql."): - # Only attempts to set Spark SQL configurations. - # If the configurations are static, it might throw an exception so - # simply ignore it for now. - try: - session.conf.set(k, v) - except Exception as e: - logger.warn(f"Failed to set configuration {k} due to {e}") + if k not in invalid_opts and k.startswith("spark.sql."): + opts[k] = v - with self._lock: for k, v in self._options.items(): - # the options are applied after session creation, - # so following options always take no effect - if k not in [ - "spark.remote", - "spark.master", - ]: - try: - session.conf.set(k, v) - except Exception as e: - logger.warn(f"Failed to set configuration {k} due to {e}") + if k not in invalid_opts: + opts[k] = v + + if len(opts) > 0: + session.conf._set_all(configs=opts, silent=True) def create(self) -> "SparkSession": has_channel_builder = self._channel_builder is not None diff --git a/sql/connect/common/src/main/protobuf/spark/connect/base.proto b/sql/connect/common/src/main/protobuf/spark/connect/base.proto index c308c7e21b66d..74413509ebc14 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/base.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/base.proto @@ -525,6 +525,9 @@ message ConfigRequest { message Set { // (Required) The config key-value pairs to set. repeated KeyValue pairs = 1; + + // (Optional) Whether to ignore failures. + optional bool silent = 2; } message Get { diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala index c5e484e022bc4..06bc24b6ccae6 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectConfigHandler.scala @@ -73,11 +73,21 @@ class SparkConnectConfigHandler(responseObserver: StreamObserver[proto.ConfigRes private def handleSet( operation: proto.ConfigRequest.Set, conf: RuntimeConfig): proto.ConfigResponse.Builder = { + val silent = operation.hasSilent && operation.getSilent val builder = proto.ConfigResponse.newBuilder() operation.getPairsList.asScala.iterator.foreach { pair => val (key, value) = SparkConnectConfigHandler.toKeyValue(pair) - conf.set(key, value.orNull) - getWarning(key).foreach(builder.addWarnings) + try { + conf.set(key, value.orNull) + getWarning(key).foreach(builder.addWarnings) + } catch { + case e: Throwable => + if (silent) { + builder.addWarnings(s"Failed to set $key to $value due to ${e.getMessage}") + } else { + throw e + } + } } builder } From 2acc969a489134d00b8179c3a1ae2f5fa2dc0417 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Fri, 3 Jan 2025 16:53:22 +0900 Subject: [PATCH 24/60] [SPARK-50717][PS][DOCS] Update doc and add examples for `from_pandas` ### What changes were proposed in this pull request? This PR proposes to update doc and add examples for `from_pandas` ### Why are the changes needed? Improve documentation ### Does this PR introduce _any_ user-facing change? No API changes ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #49349 from itholic/SPARK-50717. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- python/pyspark/pandas/namespace.py | 38 ++++++++++++++++++++++++++---- 1 file changed, 34 insertions(+), 4 deletions(-) diff --git a/python/pyspark/pandas/namespace.py b/python/pyspark/pandas/namespace.py index c77cdf51a2f6d..d31bc1f48d112 100644 --- a/python/pyspark/pandas/namespace.py +++ b/python/pyspark/pandas/namespace.py @@ -138,14 +138,44 @@ def from_pandas(pobj: Union[pd.DataFrame, pd.Series, pd.Index]) -> Union[Series, Parameters ---------- - pobj : pandas.DataFrame or pandas.Series - pandas DataFrame or Series to read. + pobj : pandas.DataFrame, pandas.Series or pandas.Index + pandas DataFrame, Series or Index to read. Returns ------- - Series or DataFrame - If a pandas Series is passed in, this function returns a pandas-on-Spark Series. + DataFrame, Series or Index If a pandas DataFrame is passed in, this function returns a pandas-on-Spark DataFrame. + If a pandas Series is passed in, this function returns a pandas-on-Spark Series. + If a pandas Index is passed in, this function returns a pandas-on-Spark Index. + + Examples + -------- + >>> import pandas as pd + >>> import pyspark.pandas as ps + + Convert a pandas DataFrame: + >>> pdf = pd.DataFrame({'a': [1, 2, 3]}) + >>> psdf = ps.from_pandas(pdf) + >>> psdf + a + 0 1 + 1 2 + 2 3 + + Convert a pandas Series: + >>> pser = pd.Series([1, 2, 3]) + >>> psser = ps.from_pandas(pser) + >>> psser + 0 1 + 1 2 + 2 3 + dtype: int64 + + Convert a pandas Index: + >>> pidx = pd.Index([1, 2, 3]) + >>> psidx = ps.from_pandas(pidx) + >>> psidx + Index([1, 2, 3], dtype='int64') """ if isinstance(pobj, pd.Series): return Series(pobj) From b210f422b0078d535eddc696ebba8d92f67b81fb Mon Sep 17 00:00:00 2001 From: Paddy Xu Date: Fri, 3 Jan 2025 17:15:31 +0900 Subject: [PATCH 25/60] [SPARK-49087][SQL][CONNECT] Distinguish UnresolvedFunction calling internal functions ### What changes were proposed in this pull request? This PR introduces a new Protobuf field `is_internal` to `UnresolvedFunction` message. This field is used to carry the info on whether an `UnresolvedFunction` is calling an internal function (using the `Column#internalFn` API), so that the Connect server could act appropriately: - If `true`, then (inside Catalyst) mark the function as internal. - If `false`, then mark the function as external (public or user-defined). - (Current behaviour) If not set, then look it up in the internal registry: mark as "internal" if found, "external" otherwise. TODO: [SPARK-50658](https://issues.apache.org/jira/browse/SPARK-50658): apply the same change to the Spark Connect Python client. ### Why are the changes needed? Tidy up the code and make our first-party Connect client adhere to the best practices. ### Does this PR introduce _any_ user-facing change? Nope. ### How was this patch tested? New test is added. ### Was this patch authored or co-authored using generative AI tooling? Copilot helped generate protobuf comments and repeated test codes. Closes #49274 from xupefei/internal-fn. Lead-authored-by: Paddy Xu Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../sql/internal/columnNodeSupport.scala | 11 +- .../ColumnNodeToProtoConverterSuite.scala | 28 ++++-- .../sql/connect/proto/expressions_pb2.py | 94 +++++++++--------- .../sql/connect/proto/expressions_pb2.pyi | 20 ++++ .../protobuf/spark/connect/expressions.proto | 5 + .../query-tests/queries/column_add.json | 3 +- .../query-tests/queries/column_add.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_and.json | 9 +- .../query-tests/queries/column_and.proto.bin | Bin 212 -> 218 bytes .../query-tests/queries/column_between.json | 9 +- .../queries/column_between.proto.bin | Bin 207 -> 213 bytes .../queries/column_bitwiseAND.json | 3 +- .../queries/column_bitwiseAND.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_bitwiseOR.json | 3 +- .../queries/column_bitwiseOR.proto.bin | Bin 176 -> 178 bytes .../queries/column_bitwiseXOR.json | 3 +- .../queries/column_bitwiseXOR.proto.bin | Bin 176 -> 178 bytes .../query-tests/queries/column_contains.json | 3 +- .../queries/column_contains.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/column_divide.json | 3 +- .../queries/column_divide.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_endsWith.json | 3 +- .../queries/column_endsWith.proto.bin | Bin 190 -> 192 bytes .../queries/column_eqNullSafe.json | 3 +- .../queries/column_eqNullSafe.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/column_equals.json | 3 +- .../queries/column_equals.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_geq.json | 3 +- .../query-tests/queries/column_geq.proto.bin | Bin 178 -> 180 bytes .../query-tests/queries/column_gt.json | 3 +- .../query-tests/queries/column_gt.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_ilike.json | 3 +- .../queries/column_ilike.proto.bin | Bin 184 -> 186 bytes .../query-tests/queries/column_isNaN.json | 3 +- .../queries/column_isNaN.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/column_isNotNull.json | 3 +- .../queries/column_isNotNull.proto.bin | Bin 178 -> 180 bytes .../query-tests/queries/column_isNull.json | 3 +- .../queries/column_isNull.proto.bin | Bin 175 -> 177 bytes .../query-tests/queries/column_isin.json | 3 +- .../query-tests/queries/column_isin.proto.bin | Bin 202 -> 204 bytes .../query-tests/queries/column_leq.json | 3 +- .../query-tests/queries/column_leq.proto.bin | Bin 178 -> 180 bytes .../query-tests/queries/column_like.json | 3 +- .../query-tests/queries/column_like.proto.bin | Bin 184 -> 186 bytes .../query-tests/queries/column_lt.json | 3 +- .../query-tests/queries/column_lt.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_modulo.json | 3 +- .../queries/column_modulo.proto.bin | Bin 176 -> 178 bytes .../query-tests/queries/column_multiply.json | 3 +- .../queries/column_multiply.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/column_not.json | 3 +- .../query-tests/queries/column_not.proto.bin | Bin 169 -> 171 bytes .../queries/column_not_equals.json | 6 +- .../queries/column_not_equals.proto.bin | Bin 184 -> 188 bytes .../query-tests/queries/column_or.json | 9 +- .../query-tests/queries/column_or.proto.bin | Bin 211 -> 217 bytes .../query-tests/queries/column_rlike.json | 3 +- .../queries/column_rlike.proto.bin | Bin 187 -> 189 bytes .../queries/column_startsWith.json | 3 +- .../queries/column_startsWith.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/column_substr.json | 3 +- .../queries/column_substr.proto.bin | Bin 187 -> 189 bytes .../query-tests/queries/column_subtract.json | 3 +- .../queries/column_subtract.proto.bin | Bin 177 -> 179 bytes .../queries/column_unary_minus.json | 3 +- .../queries/column_unary_minus.proto.bin | Bin 176 -> 178 bytes .../queries/column_when_otherwise.json | 9 +- .../queries/column_when_otherwise.proto.bin | Bin 237 -> 243 bytes .../query-tests/queries/cube_column.json | 3 +- .../query-tests/queries/cube_column.proto.bin | Bin 92 -> 94 bytes .../query-tests/queries/cube_string.json | 3 +- .../query-tests/queries/cube_string.proto.bin | Bin 96 -> 98 bytes .../resources/query-tests/queries/filter.json | 3 +- .../query-tests/queries/filter.proto.bin | Bin 69 -> 71 bytes .../queries/from_avro_with_options.json | 6 +- .../queries/from_avro_with_options.proto.bin | Bin 173 -> 178 bytes .../queries/from_avro_without_options.json | 3 +- .../from_avro_without_options.proto.bin | Bin 112 -> 114 bytes ...rotobuf_messageClassName_descFilePath.json | 3 +- ...uf_messageClassName_descFilePath.proto.bin | Bin 361 -> 363 bytes ...messageClassName_descFilePath_options.json | 6 +- ...geClassName_descFilePath_options.proto.bin | Bin 409 -> 413 bytes .../query-tests/queries/function_abs.json | 3 +- .../queries/function_abs.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_acos.json | 3 +- .../queries/function_acos.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_acosh.json | 3 +- .../queries/function_acosh.proto.bin | Bin 174 -> 176 bytes .../queries/function_add_months.json | 3 +- .../queries/function_add_months.proto.bin | Bin 130 -> 132 bytes .../queries/function_aes_decrypt.json | 3 +- .../queries/function_aes_decrypt.proto.bin | Bin 187 -> 189 bytes .../function_aes_decrypt_with_mode.json | 3 +- .../function_aes_decrypt_with_mode.proto.bin | Bin 194 -> 196 bytes ...unction_aes_decrypt_with_mode_padding.json | 3 +- ...on_aes_decrypt_with_mode_padding.proto.bin | Bin 201 -> 203 bytes ...ion_aes_decrypt_with_mode_padding_aad.json | 3 +- ...es_decrypt_with_mode_padding_aad.proto.bin | Bin 208 -> 210 bytes .../queries/function_aes_encrypt.json | 3 +- .../queries/function_aes_encrypt.proto.bin | Bin 187 -> 189 bytes .../function_aes_encrypt_with_mode.json | 3 +- .../function_aes_encrypt_with_mode.proto.bin | Bin 194 -> 196 bytes ...unction_aes_encrypt_with_mode_padding.json | 3 +- ...on_aes_encrypt_with_mode_padding.proto.bin | Bin 201 -> 203 bytes ...tion_aes_encrypt_with_mode_padding_iv.json | 3 +- ...aes_encrypt_with_mode_padding_iv.proto.bin | Bin 210 -> 212 bytes ..._aes_encrypt_with_mode_padding_iv_aad.json | 3 +- ...encrypt_with_mode_padding_iv_aad.proto.bin | Bin 217 -> 219 bytes .../queries/function_aggregate.json | 6 +- .../queries/function_aggregate.proto.bin | Bin 247 -> 251 bytes ...function_aggregate_with_finish_lambda.json | 9 +- ...ion_aggregate_with_finish_lambda.proto.bin | Bin 260 -> 266 bytes .../query-tests/queries/function_any.json | 3 +- .../queries/function_any.proto.bin | Bin 65 -> 67 bytes .../function_any_value_with_ignore_nulls.json | 3 +- ...tion_any_value_with_ignore_nulls.proto.bin | Bin 184 -> 186 bytes ...function_any_value_with_respect_nulls.json | 3 +- ...ion_any_value_with_respect_nulls.proto.bin | Bin 178 -> 180 bytes .../function_approx_count_distinct.json | 3 +- .../function_approx_count_distinct.proto.bin | Bin 190 -> 192 bytes .../function_approx_count_distinct_rsd.json | 3 +- ...nction_approx_count_distinct_rsd.proto.bin | Bin 203 -> 205 bytes .../queries/function_approx_percentile.json | 3 +- .../function_approx_percentile.proto.bin | Bin 205 -> 207 bytes .../query-tests/queries/function_array.json | 3 +- .../queries/function_array.proto.bin | Bin 181 -> 183 bytes .../queries/function_array_agg.json | 3 +- .../queries/function_array_agg.proto.bin | Bin 178 -> 180 bytes .../queries/function_array_append.json | 3 +- .../queries/function_array_append.proto.bin | Bin 187 -> 189 bytes .../queries/function_array_compact.json | 3 +- .../queries/function_array_compact.proto.bin | Bin 182 -> 184 bytes .../queries/function_array_contains.json | 3 +- .../queries/function_array_contains.proto.bin | Bin 189 -> 191 bytes .../queries/function_array_distinct.json | 3 +- .../queries/function_array_distinct.proto.bin | Bin 183 -> 185 bytes .../queries/function_array_except.json | 6 +- .../queries/function_array_except.proto.bin | Bin 210 -> 214 bytes .../queries/function_array_insert.json | 3 +- .../queries/function_array_insert.proto.bin | Bin 193 -> 195 bytes .../queries/function_array_intersect.json | 6 +- .../function_array_intersect.proto.bin | Bin 207 -> 211 bytes .../queries/function_array_join.json | 3 +- .../queries/function_array_join.proto.bin | Bin 186 -> 188 bytes ...tion_array_join_with_null_replacement.json | 3 +- ...array_join_with_null_replacement.proto.bin | Bin 196 -> 198 bytes .../queries/function_array_max.json | 3 +- .../queries/function_array_max.proto.bin | Bin 178 -> 180 bytes .../queries/function_array_min.json | 3 +- .../queries/function_array_min.proto.bin | Bin 178 -> 180 bytes .../queries/function_array_position.json | 3 +- .../queries/function_array_position.proto.bin | Bin 189 -> 191 bytes .../queries/function_array_prepend.json | 3 +- .../queries/function_array_prepend.proto.bin | Bin 188 -> 190 bytes .../queries/function_array_remove.json | 3 +- .../queries/function_array_remove.proto.bin | Bin 188 -> 190 bytes .../queries/function_array_repeat.json | 3 +- .../queries/function_array_repeat.proto.bin | Bin 187 -> 189 bytes .../queries/function_array_size.json | 3 +- .../queries/function_array_size.proto.bin | Bin 179 -> 181 bytes .../queries/function_array_sort.json | 3 +- .../queries/function_array_sort.proto.bin | Bin 179 -> 181 bytes .../function_array_sort_with_comparator.json | 6 +- ...ction_array_sort_with_comparator.proto.bin | Bin 222 -> 226 bytes .../queries/function_array_union.json | 6 +- .../queries/function_array_union.proto.bin | Bin 209 -> 213 bytes .../queries/function_arrays_overlap.json | 6 +- .../queries/function_arrays_overlap.proto.bin | Bin 206 -> 210 bytes .../queries/function_arrays_zip.json | 6 +- .../queries/function_arrays_zip.proto.bin | Bin 205 -> 209 bytes .../query-tests/queries/function_ascii.json | 3 +- .../queries/function_ascii.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_asin.json | 3 +- .../queries/function_asin.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_asinh.json | 3 +- .../queries/function_asinh.proto.bin | Bin 174 -> 176 bytes .../function_assert_true_with_message.json | 6 +- ...unction_assert_true_with_message.proto.bin | Bin 212 -> 216 bytes .../query-tests/queries/function_atan.json | 3 +- .../queries/function_atan.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_atan2.json | 3 +- .../queries/function_atan2.proto.bin | Bin 189 -> 191 bytes .../query-tests/queries/function_atanh.json | 3 +- .../queries/function_atanh.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_avg.json | 3 +- .../queries/function_avg.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_base64.json | 3 +- .../queries/function_base64.proto.bin | Bin 183 -> 185 bytes .../query-tests/queries/function_bin.json | 3 +- .../queries/function_bin.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_bit_and.json | 3 +- .../queries/function_bit_and.proto.bin | Bin 176 -> 178 bytes .../queries/function_bit_count.json | 3 +- .../queries/function_bit_count.proto.bin | Bin 178 -> 180 bytes .../query-tests/queries/function_bit_get.json | 3 +- .../queries/function_bit_get.proto.bin | Bin 182 -> 184 bytes .../queries/function_bit_length.json | 3 +- .../queries/function_bit_length.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/function_bit_or.json | 3 +- .../queries/function_bit_or.proto.bin | Bin 175 -> 177 bytes .../query-tests/queries/function_bit_xor.json | 3 +- .../queries/function_bit_xor.proto.bin | Bin 176 -> 178 bytes .../queries/function_bitmap_bit_position.json | 3 +- .../function_bitmap_bit_position.proto.bin | Bin 189 -> 191 bytes .../function_bitmap_bucket_number.json | 3 +- .../function_bitmap_bucket_number.proto.bin | Bin 189 -> 191 bytes .../function_bitmap_construct_agg.json | 3 +- .../function_bitmap_construct_agg.proto.bin | Bin 190 -> 192 bytes .../queries/function_bitmap_count.json | 3 +- .../queries/function_bitmap_count.proto.bin | Bin 75 -> 77 bytes .../queries/function_bitmap_or_agg.json | 3 +- .../queries/function_bitmap_or_agg.proto.bin | Bin 76 -> 78 bytes .../queries/function_bitwise_not.json | 3 +- .../queries/function_bitwise_not.proto.bin | Bin 170 -> 172 bytes .../queries/function_bool_and.json | 3 +- .../queries/function_bool_and.proto.bin | Bin 70 -> 72 bytes .../query-tests/queries/function_bool_or.json | 3 +- .../queries/function_bool_or.proto.bin | Bin 69 -> 71 bytes .../query-tests/queries/function_bround.json | 3 +- .../queries/function_bround.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_btrim.json | 3 +- .../queries/function_btrim.proto.bin | Bin 174 -> 176 bytes ...tion_btrim_with_specified_trim_string.json | 3 +- ...btrim_with_specified_trim_string.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_bucket.json | 3 +- .../queries/function_bucket.proto.bin | Bin 181 -> 183 bytes .../queries/function_call_function.json | 3 +- .../queries/function_call_function.proto.bin | Bin 176 -> 178 bytes .../queries/function_cardinality.json | 3 +- .../queries/function_cardinality.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_ceil.json | 3 +- .../queries/function_ceil.proto.bin | Bin 173 -> 175 bytes .../queries/function_ceil_scale.json | 3 +- .../queries/function_ceil_scale.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/function_ceiling.json | 3 +- .../queries/function_ceiling.proto.bin | Bin 176 -> 178 bytes .../queries/function_ceiling_scale.json | 3 +- .../queries/function_ceiling_scale.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_char.json | 3 +- .../queries/function_char.proto.bin | Bin 173 -> 175 bytes .../queries/function_char_length.json | 3 +- .../queries/function_char_length.proto.bin | Bin 180 -> 182 bytes .../queries/function_character_length.json | 3 +- .../function_character_length.proto.bin | Bin 185 -> 187 bytes .../query-tests/queries/function_chr.json | 3 +- .../queries/function_chr.proto.bin | Bin 172 -> 174 bytes .../queries/function_coalesce.json | 3 +- .../queries/function_coalesce.proto.bin | Bin 183 -> 185 bytes .../query-tests/queries/function_collate.json | 3 +- .../queries/function_collate.proto.bin | Bin 189 -> 191 bytes .../queries/function_collation.json | 3 +- .../queries/function_collation.proto.bin | Bin 178 -> 180 bytes .../queries/function_collect_list.json | 3 +- .../queries/function_collect_list.proto.bin | Bin 181 -> 183 bytes .../queries/function_collect_set.json | 3 +- .../queries/function_collect_set.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_concat.json | 9 +- .../queries/function_concat.proto.bin | Bin 224 -> 230 bytes .../queries/function_concat_ws.json | 3 +- .../queries/function_concat_ws.proto.bin | Bin 204 -> 206 bytes .../queries/function_contains.json | 3 +- .../queries/function_contains.proto.bin | Bin 184 -> 186 bytes .../query-tests/queries/function_conv.json | 3 +- .../queries/function_conv.proto.bin | Bin 185 -> 187 bytes ...onvert_timezone_with_source_time_zone.json | 3 +- ...t_timezone_with_source_time_zone.proto.bin | Bin 170 -> 172 bytes ...ert_timezone_without_source_time_zone.json | 3 +- ...imezone_without_source_time_zone.proto.bin | Bin 150 -> 152 bytes .../query-tests/queries/function_corr.json | 3 +- .../queries/function_corr.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_cos.json | 3 +- .../queries/function_cos.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_cosh.json | 3 +- .../queries/function_cosh.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_cot.json | 3 +- .../queries/function_cot.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_count.json | 3 +- .../queries/function_count.proto.bin | Bin 174 -> 176 bytes .../queries/function_countDistinct.json | 3 +- .../queries/function_countDistinct.proto.bin | Bin 183 -> 185 bytes .../queries/function_count_if.json | 6 +- .../queries/function_count_if.proto.bin | Bin 190 -> 194 bytes .../queries/function_count_min_sketch.json | 3 +- .../function_count_min_sketch.proto.bin | Bin 217 -> 219 bytes .../queries/function_count_typed.json | 3 +- .../queries/function_count_typed.proto.bin | Bin 66 -> 68 bytes .../queries/function_covar_pop.json | 3 +- .../queries/function_covar_pop.proto.bin | Bin 185 -> 187 bytes .../queries/function_covar_samp.json | 3 +- .../queries/function_covar_samp.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/function_crc32.json | 3 +- .../queries/function_crc32.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_csc.json | 3 +- .../queries/function_csc.proto.bin | Bin 172 -> 174 bytes .../queries/function_cume_dist.json | 3 +- .../queries/function_cume_dist.proto.bin | Bin 196 -> 198 bytes .../query-tests/queries/function_curdate.json | 3 +- .../queries/function_curdate.proto.bin | Bin 114 -> 116 bytes .../queries/function_current_catalog.json | 3 +- .../function_current_catalog.proto.bin | Bin 177 -> 179 bytes .../queries/function_current_database.json | 3 +- .../function_current_database.proto.bin | Bin 178 -> 180 bytes .../queries/function_current_date.json | 3 +- .../queries/function_current_date.proto.bin | Bin 119 -> 121 bytes .../queries/function_current_schema.json | 3 +- .../queries/function_current_schema.proto.bin | Bin 176 -> 178 bytes .../queries/function_current_timestamp.json | 3 +- .../function_current_timestamp.proto.bin | Bin 124 -> 126 bytes .../queries/function_current_timezone.json | 3 +- .../function_current_timezone.proto.bin | Bin 123 -> 125 bytes .../queries/function_current_user.json | 3 +- .../queries/function_current_user.proto.bin | Bin 174 -> 176 bytes .../queries/function_date_add.json | 3 +- .../queries/function_date_add.proto.bin | Bin 128 -> 130 bytes .../queries/function_date_diff.json | 6 +- .../queries/function_date_diff.proto.bin | Bin 158 -> 162 bytes .../queries/function_date_format.json | 3 +- .../queries/function_date_format.proto.bin | Bin 142 -> 144 bytes .../queries/function_date_from_unix_date.json | 3 +- .../function_date_from_unix_date.proto.bin | Bin 132 -> 135 bytes .../queries/function_date_part.json | 3 +- .../queries/function_date_part.proto.bin | Bin 133 -> 136 bytes .../queries/function_date_sub.json | 3 +- .../queries/function_date_sub.proto.bin | Bin 128 -> 130 bytes .../queries/function_date_trunc.json | 3 +- .../queries/function_date_trunc.proto.bin | Bin 131 -> 133 bytes .../query-tests/queries/function_dateadd.json | 3 +- .../queries/function_dateadd.proto.bin | Bin 127 -> 129 bytes .../queries/function_datediff.json | 6 +- .../queries/function_datediff.proto.bin | Bin 157 -> 161 bytes .../queries/function_datepart.json | 3 +- .../queries/function_datepart.proto.bin | Bin 132 -> 135 bytes .../query-tests/queries/function_day.json | 3 +- .../queries/function_day.proto.bin | Bin 117 -> 119 bytes .../query-tests/queries/function_dayname.json | 3 +- .../queries/function_dayname.proto.bin | Bin 121 -> 123 bytes .../queries/function_dayofmonth.json | 3 +- .../queries/function_dayofmonth.proto.bin | Bin 124 -> 126 bytes .../queries/function_dayofweek.json | 3 +- .../queries/function_dayofweek.proto.bin | Bin 123 -> 125 bytes .../queries/function_dayofyear.json | 3 +- .../queries/function_dayofyear.proto.bin | Bin 123 -> 125 bytes .../query-tests/queries/function_days.json | 3 +- .../queries/function_days.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_decode.json | 3 +- .../queries/function_decode.proto.bin | Bin 194 -> 196 bytes .../query-tests/queries/function_degrees.json | 3 +- .../queries/function_degrees.proto.bin | Bin 176 -> 178 bytes .../queries/function_dense_rank.json | 3 +- .../queries/function_dense_rank.proto.bin | Bin 197 -> 199 bytes .../query-tests/queries/function_e.json | 3 +- .../query-tests/queries/function_e.proto.bin | Bin 163 -> 165 bytes .../queries/function_element_at.json | 3 +- .../queries/function_element_at.proto.bin | Bin 188 -> 190 bytes .../query-tests/queries/function_elt.json | 3 +- .../queries/function_elt.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/function_encode.json | 3 +- .../queries/function_encode.proto.bin | Bin 186 -> 188 bytes .../queries/function_endswith.json | 3 +- .../queries/function_endswith.proto.bin | Bin 184 -> 186 bytes .../queries/function_equal_null.json | 3 +- .../queries/function_equal_null.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/function_every.json | 3 +- .../queries/function_every.proto.bin | Bin 67 -> 69 bytes .../query-tests/queries/function_exists.json | 6 +- .../queries/function_exists.proto.bin | Bin 208 -> 212 bytes .../query-tests/queries/function_exp.json | 3 +- .../queries/function_exp.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_explode.json | 3 +- .../queries/function_explode.proto.bin | Bin 176 -> 178 bytes .../queries/function_explode_outer.json | 3 +- .../queries/function_explode_outer.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_expm1.json | 3 +- .../queries/function_expm1.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_extract.json | 3 +- .../queries/function_extract.proto.bin | Bin 131 -> 133 bytes .../queries/function_factorial.json | 6 +- .../queries/function_factorial.proto.bin | Bin 191 -> 195 bytes .../query-tests/queries/function_filter.json | 6 +- .../queries/function_filter.proto.bin | Bin 208 -> 212 bytes .../function_filter_with_pair_input.json | 12 ++- .../function_filter_with_pair_input.proto.bin | Bin 246 -> 254 bytes .../queries/function_find_in_set.json | 3 +- .../queries/function_find_in_set.proto.bin | Bin 187 -> 189 bytes ...unction_first_value_with_ignore_nulls.json | 3 +- ...on_first_value_with_ignore_nulls.proto.bin | Bin 186 -> 188 bytes ...nction_first_value_with_respect_nulls.json | 3 +- ...n_first_value_with_respect_nulls.proto.bin | Bin 180 -> 182 bytes .../function_first_with_ignore_nulls.json | 3 +- ...function_first_with_ignore_nulls.proto.bin | Bin 180 -> 182 bytes .../function_first_with_respect_nulls.json | 3 +- ...unction_first_with_respect_nulls.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_flatten.json | 9 +- .../queries/function_flatten.proto.bin | Bin 213 -> 219 bytes .../query-tests/queries/function_floor.json | 3 +- .../queries/function_floor.proto.bin | Bin 174 -> 176 bytes .../queries/function_floor_scale.json | 3 +- .../queries/function_floor_scale.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_forall.json | 6 +- .../queries/function_forall.proto.bin | Bin 208 -> 212 bytes .../queries/function_format_number.json | 3 +- .../queries/function_format_number.proto.bin | Bin 188 -> 190 bytes .../queries/function_from_csv.json | 6 +- .../queries/function_from_csv.proto.bin | Bin 240 -> 244 bytes .../queries/function_from_json.json | 3 +- .../queries/function_from_json.proto.bin | Bin 394 -> 396 bytes .../function_from_json_with_json_schema.json | 3 +- ...ction_from_json_with_json_schema.proto.bin | Bin 394 -> 396 bytes .../queries/function_from_unixtime.json | 3 +- .../queries/function_from_unixtime.proto.bin | Bin 126 -> 128 bytes .../queries/function_from_utc_timestamp.json | 3 +- .../function_from_utc_timestamp.proto.bin | Bin 145 -> 147 bytes .../queries/function_from_xml.json | 3 +- .../queries/function_from_xml.proto.bin | Bin 393 -> 395 bytes .../function_from_xml_with_json_schema.json | 3 +- ...nction_from_xml_with_json_schema.proto.bin | Bin 393 -> 395 bytes .../query-tests/queries/function_get.json | 3 +- .../queries/function_get.proto.bin | Bin 178 -> 180 bytes .../queries/function_get_json_object.json | 3 +- .../function_get_json_object.proto.bin | Bin 203 -> 205 bytes .../query-tests/queries/function_getbit.json | 3 +- .../queries/function_getbit.proto.bin | Bin 181 -> 183 bytes .../queries/function_greatest.json | 3 +- .../queries/function_greatest.proto.bin | Bin 195 -> 197 bytes .../query-tests/queries/function_hash.json | 3 +- .../queries/function_hash.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_hex.json | 3 +- .../queries/function_hex.proto.bin | Bin 172 -> 174 bytes .../queries/function_histogram_numeric.json | 3 +- .../function_histogram_numeric.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/function_hour.json | 3 +- .../queries/function_hour.proto.bin | Bin 118 -> 120 bytes .../query-tests/queries/function_hours.json | 3 +- .../queries/function_hours.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_hypot.json | 3 +- .../queries/function_hypot.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_ifnull.json | 3 +- .../queries/function_ifnull.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_ilike.json | 3 +- .../queries/function_ilike.proto.bin | Bin 181 -> 183 bytes .../queries/function_ilike_with_escape.json | 3 +- .../function_ilike_with_escape.proto.bin | Bin 188 -> 190 bytes .../query-tests/queries/function_initcap.json | 3 +- .../queries/function_initcap.proto.bin | Bin 176 -> 178 bytes .../query-tests/queries/function_inline.json | 6 +- .../queries/function_inline.proto.bin | Bin 191 -> 195 bytes .../queries/function_inline_outer.json | 6 +- .../queries/function_inline_outer.proto.bin | Bin 197 -> 201 bytes .../function_input_file_block_length.json | 3 +- ...function_input_file_block_length.proto.bin | Bin 185 -> 187 bytes .../function_input_file_block_start.json | 3 +- .../function_input_file_block_start.proto.bin | Bin 184 -> 186 bytes .../queries/function_input_file_name.json | 3 +- .../function_input_file_name.proto.bin | Bin 177 -> 179 bytes .../queries/function_is_variant_null.json | 6 +- .../function_is_variant_null.proto.bin | Bin 200 -> 204 bytes .../query-tests/queries/function_isnan.json | 3 +- .../queries/function_isnan.proto.bin | Bin 174 -> 176 bytes .../queries/function_isnotnull.json | 3 +- .../queries/function_isnotnull.proto.bin | Bin 178 -> 180 bytes .../query-tests/queries/function_isnull.json | 3 +- .../queries/function_isnull.proto.bin | Bin 175 -> 177 bytes .../queries/function_java_method.json | 3 +- .../queries/function_java_method.proto.bin | Bin 216 -> 218 bytes .../queries/function_json_array_length.json | 3 +- .../function_json_array_length.proto.bin | Bin 186 -> 188 bytes .../queries/function_json_object_keys.json | 3 +- .../function_json_object_keys.proto.bin | Bin 185 -> 187 bytes .../queries/function_json_tuple.json | 3 +- .../queries/function_json_tuple.proto.bin | Bin 201 -> 203 bytes .../queries/function_kurtosis.json | 3 +- .../queries/function_kurtosis.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/function_lag.json | 3 +- .../queries/function_lag.proto.bin | Bin 217 -> 219 bytes .../queries/function_last_day.json | 3 +- .../queries/function_last_day.proto.bin | Bin 122 -> 124 bytes ...function_last_value_with_ignore_nulls.json | 3 +- ...ion_last_value_with_ignore_nulls.proto.bin | Bin 185 -> 187 bytes ...unction_last_value_with_respect_nulls.json | 3 +- ...on_last_value_with_respect_nulls.proto.bin | Bin 179 -> 181 bytes .../function_last_with_ignore_nulls.json | 3 +- .../function_last_with_ignore_nulls.proto.bin | Bin 179 -> 181 bytes .../function_last_with_respect_nulls.json | 3 +- ...function_last_with_respect_nulls.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/function_lcase.json | 3 +- .../queries/function_lcase.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_lead.json | 3 +- .../queries/function_lead.proto.bin | Bin 218 -> 220 bytes .../query-tests/queries/function_least.json | 3 +- .../queries/function_least.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/function_left.json | 3 +- .../queries/function_left.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_len.json | 3 +- .../queries/function_len.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_length.json | 3 +- .../queries/function_length.proto.bin | Bin 175 -> 177 bytes .../queries/function_levenshtein.json | 3 +- .../queries/function_levenshtein.proto.bin | Bin 189 -> 191 bytes .../function_levenshtein_with_threshold.json | 3 +- ...ction_levenshtein_with_threshold.proto.bin | Bin 195 -> 197 bytes .../query-tests/queries/function_like.json | 3 +- .../queries/function_like.proto.bin | Bin 180 -> 182 bytes .../queries/function_like_with_escape.json | 3 +- .../function_like_with_escape.proto.bin | Bin 187 -> 189 bytes .../query-tests/queries/function_ln.json | 3 +- .../query-tests/queries/function_ln.proto.bin | Bin 171 -> 173 bytes .../queries/function_localtimestamp.json | 3 +- .../queries/function_localtimestamp.proto.bin | Bin 121 -> 123 bytes .../query-tests/queries/function_locate.json | 3 +- .../queries/function_locate.proto.bin | Bin 184 -> 186 bytes .../queries/function_locate_with_pos.json | 3 +- .../function_locate_with_pos.proto.bin | Bin 190 -> 192 bytes .../query-tests/queries/function_log.json | 3 +- .../queries/function_log.proto.bin | Bin 171 -> 173 bytes .../query-tests/queries/function_log10.json | 3 +- .../queries/function_log10.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_log1p.json | 3 +- .../queries/function_log1p.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_log2.json | 3 +- .../queries/function_log2.proto.bin | Bin 173 -> 175 bytes .../queries/function_log_with_base.json | 3 +- .../queries/function_log_with_base.proto.bin | Bin 185 -> 187 bytes .../query-tests/queries/function_lower.json | 3 +- .../queries/function_lower.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_lpad.json | 3 +- .../queries/function_lpad.proto.bin | Bin 186 -> 188 bytes .../queries/function_lpad_binary.json | 3 +- .../queries/function_lpad_binary.proto.bin | Bin 83 -> 85 bytes .../query-tests/queries/function_ltrim.json | 3 +- .../queries/function_ltrim.proto.bin | Bin 174 -> 176 bytes .../queries/function_ltrim_with_pattern.json | 3 +- .../function_ltrim_with_pattern.proto.bin | Bin 183 -> 185 bytes .../queries/function_make_date.json | 3 +- .../queries/function_make_date.proto.bin | Bin 136 -> 138 bytes .../queries/function_make_dt_interval.json | 3 +- .../function_make_dt_interval.proto.bin | Bin 178 -> 180 bytes .../function_make_dt_interval_days.json | 3 +- .../function_make_dt_interval_days.proto.bin | Bin 185 -> 187 bytes .../function_make_dt_interval_days_hours.json | 3 +- ...tion_make_dt_interval_days_hours.proto.bin | Bin 192 -> 194 bytes ...tion_make_dt_interval_days_hours_mins.json | 3 +- ...make_dt_interval_days_hours_mins.proto.bin | Bin 199 -> 201 bytes ...make_dt_interval_days_hours_mins_secs.json | 3 +- ...dt_interval_days_hours_mins_secs.proto.bin | Bin 206 -> 208 bytes .../queries/function_make_interval.json | 3 +- .../queries/function_make_interval.proto.bin | Bin 175 -> 177 bytes .../queries/function_make_interval_years.json | 3 +- .../function_make_interval_years.proto.bin | Bin 182 -> 184 bytes .../function_make_interval_years_months.json | 3 +- ...ction_make_interval_years_months.proto.bin | Bin 189 -> 191 bytes ...tion_make_interval_years_months_weeks.json | 3 +- ...make_interval_years_months_weeks.proto.bin | Bin 196 -> 198 bytes ...make_interval_years_months_weeks_days.json | 3 +- ...interval_years_months_weeks_days.proto.bin | Bin 203 -> 205 bytes ...nterval_years_months_weeks_days_hours.json | 3 +- ...al_years_months_weeks_days_hours.proto.bin | Bin 210 -> 212 bytes ...al_years_months_weeks_days_hours_mins.json | 3 +- ...ars_months_weeks_days_hours_mins.proto.bin | Bin 217 -> 219 bytes ...ars_months_weeks_days_hours_mins_secs.json | 3 +- ...onths_weeks_days_hours_mins_secs.proto.bin | Bin 224 -> 226 bytes ...tion_make_timestamp_ltz_with_timezone.json | 3 +- ...make_timestamp_ltz_with_timezone.proto.bin | Bin 229 -> 231 bytes ...n_make_timestamp_ltz_without_timezone.json | 3 +- ...e_timestamp_ltz_without_timezone.proto.bin | Bin 222 -> 224 bytes .../queries/function_make_timestamp_ntz.json | 3 +- .../function_make_timestamp_ntz.proto.bin | Bin 222 -> 224 bytes ...function_make_timestamp_with_timezone.json | 3 +- ...ion_make_timestamp_with_timezone.proto.bin | Bin 225 -> 227 bytes ...ction_make_timestamp_without_timezone.json | 3 +- ..._make_timestamp_without_timezone.proto.bin | Bin 218 -> 220 bytes .../queries/function_make_ym_interval.json | 3 +- .../function_make_ym_interval.proto.bin | Bin 178 -> 180 bytes .../function_make_ym_interval_years.json | 3 +- .../function_make_ym_interval_years.proto.bin | Bin 185 -> 187 bytes ...unction_make_ym_interval_years_months.json | 3 +- ...on_make_ym_interval_years_months.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/function_map.json | 3 +- .../queries/function_map.proto.bin | Bin 196 -> 198 bytes .../queries/function_map_concat.json | 9 +- .../queries/function_map_concat.proto.bin | Bin 256 -> 262 bytes .../queries/function_map_contains_key.json | 3 +- .../function_map_contains_key.proto.bin | Bin 194 -> 196 bytes .../queries/function_map_entries.json | 3 +- .../queries/function_map_entries.proto.bin | Bin 180 -> 182 bytes .../queries/function_map_filter.json | 6 +- .../queries/function_map_filter.proto.bin | Bin 229 -> 233 bytes .../queries/function_map_from_arrays.json | 9 +- .../function_map_from_arrays.proto.bin | Bin 229 -> 235 bytes .../queries/function_map_from_entries.json | 9 +- .../function_map_from_entries.proto.bin | Bin 248 -> 254 bytes .../queries/function_map_keys.json | 3 +- .../queries/function_map_keys.proto.bin | Bin 177 -> 179 bytes .../queries/function_map_values.json | 3 +- .../queries/function_map_values.proto.bin | Bin 179 -> 181 bytes .../queries/function_map_zip_with.json | 6 +- .../queries/function_map_zip_with.proto.bin | Bin 262 -> 266 bytes .../query-tests/queries/function_mask.json | 3 +- .../queries/function_mask.proto.bin | Bin 173 -> 175 bytes ...function_mask_with_specific_upperChar.json | 3 +- ...ion_mask_with_specific_upperChar.proto.bin | Bin 180 -> 182 bytes ...ask_with_specific_upperChar_lowerChar.json | 3 +- ...ith_specific_upperChar_lowerChar.proto.bin | Bin 187 -> 189 bytes ...pecific_upperChar_lowerChar_digitChar.json | 3 +- ...ic_upperChar_lowerChar_digitChar.proto.bin | Bin 194 -> 196 bytes ...perChar_lowerChar_digitChar_otherChar.json | 3 +- ...ar_lowerChar_digitChar_otherChar.proto.bin | Bin 201 -> 203 bytes .../query-tests/queries/function_max.json | 3 +- .../queries/function_max.proto.bin | Bin 65 -> 67 bytes .../query-tests/queries/function_max_by.json | 3 +- .../queries/function_max_by.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_md5.json | 3 +- .../queries/function_md5.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_median.json | 3 +- .../queries/function_median.proto.bin | Bin 175 -> 177 bytes .../query-tests/queries/function_min.json | 3 +- .../queries/function_min.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_min_by.json | 3 +- .../queries/function_min_by.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_minute.json | 3 +- .../queries/function_minute.proto.bin | Bin 120 -> 122 bytes .../query-tests/queries/function_mode.json | 3 +- .../queries/function_mode.proto.bin | Bin 173 -> 175 bytes .../function_monotonically_increasing_id.json | 3 +- ...tion_monotonically_increasing_id.proto.bin | Bin 189 -> 191 bytes .../query-tests/queries/function_month.json | 3 +- .../queries/function_month.proto.bin | Bin 119 -> 121 bytes .../queries/function_monthname.json | 3 +- .../queries/function_monthname.proto.bin | Bin 123 -> 125 bytes .../query-tests/queries/function_months.json | 3 +- .../queries/function_months.proto.bin | Bin 175 -> 177 bytes .../queries/function_months_between.json | 6 +- .../queries/function_months_between.proto.bin | Bin 147 -> 151 bytes ...function_months_between_with_roundoff.json | 6 +- ...ion_months_between_with_roundoff.proto.bin | Bin 153 -> 157 bytes .../queries/function_named_struct.json | 3 +- .../queries/function_named_struct.proto.bin | Bin 203 -> 205 bytes .../query-tests/queries/function_nanvl.json | 3 +- .../queries/function_nanvl.proto.bin | Bin 187 -> 189 bytes .../query-tests/queries/function_negate.json | 3 +- .../queries/function_negate.proto.bin | Bin 177 -> 179 bytes .../queries/function_negative.json | 3 +- .../queries/function_negative.proto.bin | Bin 177 -> 179 bytes .../queries/function_next_day.json | 3 +- .../queries/function_next_day.proto.bin | Bin 131 -> 133 bytes .../query-tests/queries/function_now.json | 3 +- .../queries/function_now.proto.bin | Bin 110 -> 112 bytes .../queries/function_nth_value.json | 3 +- .../queries/function_nth_value.proto.bin | Bin 215 -> 217 bytes .../query-tests/queries/function_ntile.json | 3 +- .../queries/function_ntile.proto.bin | Bin 198 -> 200 bytes .../query-tests/queries/function_nullif.json | 3 +- .../queries/function_nullif.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_nvl.json | 3 +- .../queries/function_nvl.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/function_nvl2.json | 3 +- .../queries/function_nvl2.proto.bin | Bin 187 -> 189 bytes .../queries/function_octet_length.json | 3 +- .../queries/function_octet_length.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_overlay.json | 3 +- .../queries/function_overlay.proto.bin | Bin 191 -> 193 bytes .../queries/function_overlay_with_len.json | 3 +- .../function_overlay_with_len.proto.bin | Bin 198 -> 200 bytes .../queries/function_parse_json.json | 3 +- .../queries/function_parse_json.proto.bin | Bin 179 -> 181 bytes .../queries/function_parse_url.json | 3 +- .../queries/function_parse_url.proto.bin | Bin 185 -> 187 bytes .../queries/function_parse_url_with_key.json | 3 +- .../function_parse_url_with_key.proto.bin | Bin 192 -> 194 bytes .../queries/function_percent_rank.json | 3 +- .../queries/function_percent_rank.proto.bin | Bin 199 -> 201 bytes .../queries/function_percentile_approx.json | 3 +- .../function_percentile_approx.proto.bin | Bin 205 -> 207 bytes .../function_percentile_with_frequency.json | 3 +- ...nction_percentile_with_frequency.proto.bin | Bin 198 -> 200 bytes ...function_percentile_without_frequency.json | 3 +- ...ion_percentile_without_frequency.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/function_pi.json | 3 +- .../query-tests/queries/function_pi.proto.bin | Bin 164 -> 166 bytes .../query-tests/queries/function_pmod.json | 3 +- .../queries/function_pmod.proto.bin | Bin 179 -> 181 bytes .../queries/function_posexplode.json | 3 +- .../queries/function_posexplode.proto.bin | Bin 179 -> 181 bytes .../queries/function_posexplode_outer.json | 3 +- .../function_posexplode_outer.proto.bin | Bin 185 -> 187 bytes .../queries/function_position.json | 3 +- .../queries/function_position.proto.bin | Bin 184 -> 186 bytes .../queries/function_position_with_start.json | 3 +- .../function_position_with_start.proto.bin | Bin 191 -> 193 bytes .../queries/function_positive.json | 3 +- .../queries/function_positive.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/function_pow.json | 3 +- .../queries/function_pow.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_power.json | 3 +- .../queries/function_power.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_printf.json | 3 +- .../queries/function_printf.proto.bin | Bin 189 -> 191 bytes .../query-tests/queries/function_product.json | 3 +- .../queries/function_product.proto.bin | Bin 176 -> 178 bytes .../query-tests/queries/function_quarter.json | 3 +- .../queries/function_quarter.proto.bin | Bin 121 -> 123 bytes .../query-tests/queries/function_radians.json | 3 +- .../queries/function_radians.proto.bin | Bin 176 -> 178 bytes .../queries/function_raise_error.json | 3 +- .../queries/function_raise_error.proto.bin | Bin 185 -> 187 bytes .../queries/function_rand_with_seed.json | 3 +- .../queries/function_rand_with_seed.proto.bin | Bin 173 -> 175 bytes .../queries/function_randn_with_seed.json | 3 +- .../function_randn_with_seed.proto.bin | Bin 174 -> 176 bytes .../queries/function_random_with_seed.json | 3 +- .../function_random_with_seed.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_rank.json | 3 +- .../queries/function_rank.proto.bin | Bin 191 -> 193 bytes .../query-tests/queries/function_reduce.json | 6 +- .../queries/function_reduce.proto.bin | Bin 244 -> 248 bytes .../query-tests/queries/function_reflect.json | 3 +- .../queries/function_reflect.proto.bin | Bin 212 -> 214 bytes .../query-tests/queries/function_regexp.json | 3 +- .../queries/function_regexp.proto.bin | Bin 188 -> 190 bytes .../queries/function_regexp_count.json | 3 +- .../queries/function_regexp_count.proto.bin | Bin 190 -> 192 bytes .../queries/function_regexp_extract.json | 3 +- .../queries/function_regexp_extract.proto.bin | Bin 206 -> 208 bytes ...xp_extract_all_with_regex_group_index.json | 3 +- ...tract_all_with_regex_group_index.proto.bin | Bin 212 -> 214 bytes ...extract_all_without_regex_group_index.json | 3 +- ...ct_all_without_regex_group_index.proto.bin | Bin 206 -> 208 bytes ...n_regexp_instr_with_regex_group_index.json | 3 +- ...exp_instr_with_regex_group_index.proto.bin | Bin 203 -> 205 bytes ...egexp_instr_without_regex_group_index.json | 3 +- ..._instr_without_regex_group_index.proto.bin | Bin 197 -> 199 bytes .../queries/function_regexp_like.json | 3 +- .../queries/function_regexp_like.proto.bin | Bin 193 -> 195 bytes .../queries/function_regexp_replace.json | 3 +- .../queries/function_regexp_replace.proto.bin | Bin 203 -> 205 bytes .../queries/function_regexp_substr.json | 3 +- .../queries/function_regexp_substr.proto.bin | Bin 200 -> 202 bytes .../queries/function_regr_avgx.json | 3 +- .../queries/function_regr_avgx.proto.bin | Bin 185 -> 187 bytes .../queries/function_regr_avgy.json | 3 +- .../queries/function_regr_avgy.proto.bin | Bin 185 -> 187 bytes .../queries/function_regr_count.json | 3 +- .../queries/function_regr_count.proto.bin | Bin 186 -> 188 bytes .../queries/function_regr_intercept.json | 3 +- .../queries/function_regr_intercept.proto.bin | Bin 190 -> 192 bytes .../query-tests/queries/function_regr_r2.json | 3 +- .../queries/function_regr_r2.proto.bin | Bin 183 -> 185 bytes .../queries/function_regr_slope.json | 3 +- .../queries/function_regr_slope.proto.bin | Bin 186 -> 188 bytes .../queries/function_regr_sxx.json | 3 +- .../queries/function_regr_sxx.proto.bin | Bin 184 -> 186 bytes .../queries/function_regr_sxy.json | 3 +- .../queries/function_regr_sxy.proto.bin | Bin 184 -> 186 bytes .../queries/function_regr_syy.json | 3 +- .../queries/function_regr_syy.proto.bin | Bin 184 -> 186 bytes .../query-tests/queries/function_replace.json | 3 +- .../queries/function_replace.proto.bin | Bin 183 -> 185 bytes ...unction_replace_with_specified_string.json | 3 +- ...on_replace_with_specified_string.proto.bin | Bin 190 -> 192 bytes .../query-tests/queries/function_reverse.json | 3 +- .../queries/function_reverse.proto.bin | Bin 176 -> 178 bytes .../query-tests/queries/function_right.json | 3 +- .../queries/function_right.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_rint.json | 3 +- .../queries/function_rint.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_rlike.json | 3 +- .../queries/function_rlike.proto.bin | Bin 187 -> 189 bytes .../query-tests/queries/function_round.json | 3 +- .../queries/function_round.proto.bin | Bin 180 -> 182 bytes .../queries/function_row_number.json | 3 +- .../queries/function_row_number.proto.bin | Bin 197 -> 199 bytes .../query-tests/queries/function_rpad.json | 3 +- .../queries/function_rpad.proto.bin | Bin 186 -> 188 bytes .../queries/function_rpad_binary.json | 3 +- .../queries/function_rpad_binary.proto.bin | Bin 83 -> 85 bytes .../query-tests/queries/function_rtrim.json | 3 +- .../queries/function_rtrim.proto.bin | Bin 174 -> 176 bytes .../queries/function_rtrim_with_pattern.json | 3 +- .../function_rtrim_with_pattern.proto.bin | Bin 183 -> 185 bytes .../queries/function_schema_of_csv.json | 6 +- .../queries/function_schema_of_csv.proto.bin | Bin 211 -> 215 bytes .../queries/function_schema_of_json.json | 3 +- .../queries/function_schema_of_json.proto.bin | Bin 194 -> 196 bytes .../function_schema_of_json_with_options.json | 6 +- ...tion_schema_of_json_with_options.proto.bin | Bin 243 -> 247 bytes .../queries/function_schema_of_variant.json | 6 +- .../function_schema_of_variant.proto.bin | Bin 202 -> 206 bytes .../function_schema_of_variant_agg.json | 6 +- .../function_schema_of_variant_agg.proto.bin | Bin 206 -> 210 bytes .../query-tests/queries/function_sec.json | 3 +- .../queries/function_sec.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_second.json | 3 +- .../queries/function_second.proto.bin | Bin 120 -> 122 bytes .../queries/function_sentences.json | 3 +- .../queries/function_sentences.proto.bin | Bin 178 -> 180 bytes .../function_sentences_with_language.json | 3 +- ...function_sentences_with_language.proto.bin | Bin 186 -> 188 bytes ...n_sentences_with_language_and_country.json | 3 +- ...tences_with_language_and_country.proto.bin | Bin 194 -> 196 bytes .../queries/function_sequence.json | 3 +- .../queries/function_sequence.proto.bin | Bin 182 -> 184 bytes .../queries/function_session_user.json | 3 +- .../queries/function_session_user.proto.bin | Bin 174 -> 176 bytes .../queries/function_session_window.json | 3 +- .../queries/function_session_window.proto.bin | Bin 145 -> 147 bytes .../query-tests/queries/function_sha.json | 3 +- .../queries/function_sha.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_sha1.json | 3 +- .../queries/function_sha1.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_sha2.json | 3 +- .../queries/function_sha2.proto.bin | Bin 188 -> 190 bytes .../queries/function_shiftleft.json | 3 +- .../queries/function_shiftleft.proto.bin | Bin 184 -> 186 bytes .../queries/function_shiftright.json | 3 +- .../queries/function_shiftright.proto.bin | Bin 185 -> 187 bytes .../queries/function_shiftrightunsigned.json | 3 +- .../function_shiftrightunsigned.proto.bin | Bin 193 -> 195 bytes .../query-tests/queries/function_sign.json | 3 +- .../queries/function_sign.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_signum.json | 3 +- .../queries/function_signum.proto.bin | Bin 175 -> 177 bytes .../query-tests/queries/function_sin.json | 3 +- .../queries/function_sin.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_sinh.json | 3 +- .../queries/function_sinh.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_size.json | 3 +- .../queries/function_size.proto.bin | Bin 173 -> 175 bytes .../queries/function_skewness.json | 3 +- .../queries/function_skewness.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/function_slice.json | 3 +- .../queries/function_slice.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/function_some.json | 3 +- .../queries/function_some.proto.bin | Bin 66 -> 68 bytes .../queries/function_sort_array.json | 3 +- .../queries/function_sort_array.proto.bin | Bin 185 -> 187 bytes .../queries/function_spark_partition_id.json | 3 +- .../function_spark_partition_id.proto.bin | Bin 180 -> 182 bytes .../query-tests/queries/function_split.json | 3 +- .../queries/function_split.proto.bin | Bin 181 -> 183 bytes .../queries/function_split_part.json | 3 +- .../queries/function_split_part.proto.bin | Bin 193 -> 195 bytes .../queries/function_split_using_columns.json | 3 +- .../function_split_using_columns.proto.bin | Bin 181 -> 183 bytes .../queries/function_split_with_limit.json | 3 +- .../function_split_with_limit.proto.bin | Bin 187 -> 189 bytes ...nction_split_with_limit_using_columns.json | 3 +- ...n_split_with_limit_using_columns.proto.bin | Bin 188 -> 190 bytes .../query-tests/queries/function_sqrt.json | 3 +- .../queries/function_sqrt.proto.bin | Bin 173 -> 175 bytes .../query-tests/queries/function_stack.json | 3 +- .../queries/function_stack.proto.bin | Bin 194 -> 196 bytes .../queries/function_startswith.json | 3 +- .../queries/function_startswith.proto.bin | Bin 186 -> 188 bytes .../query-tests/queries/function_std.json | 3 +- .../queries/function_std.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_stddev.json | 3 +- .../queries/function_stddev.proto.bin | Bin 175 -> 177 bytes .../queries/function_stddev_pop.json | 3 +- .../queries/function_stddev_pop.proto.bin | Bin 179 -> 181 bytes .../queries/function_stddev_samp.json | 3 +- .../queries/function_stddev_samp.proto.bin | Bin 180 -> 182 bytes .../queries/function_str_to_map.json | 3 +- .../queries/function_str_to_map.proto.bin | Bin 179 -> 181 bytes ..._map_with_pair_and_keyValue_delimiter.json | 3 +- ...with_pair_and_keyValue_delimiter.proto.bin | Bin 186 -> 188 bytes ...nction_str_to_map_with_pair_delimiter.json | 3 +- ...n_str_to_map_with_pair_delimiter.proto.bin | Bin 193 -> 195 bytes .../query-tests/queries/function_struct.json | 3 +- .../queries/function_struct.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_substr.json | 3 +- .../queries/function_substr.proto.bin | Bin 182 -> 184 bytes .../queries/function_substr_with_len.json | 3 +- .../function_substr_with_len.proto.bin | Bin 189 -> 191 bytes .../queries/function_substring.json | 3 +- .../queries/function_substring.proto.bin | Bin 190 -> 192 bytes .../queries/function_substring_index.json | 3 +- .../function_substring_index.proto.bin | Bin 197 -> 199 bytes .../function_substring_using_columns.json | 3 +- ...function_substring_using_columns.proto.bin | Bin 192 -> 194 bytes .../query-tests/queries/function_sum.json | 3 +- .../queries/function_sum.proto.bin | Bin 172 -> 174 bytes .../queries/function_sum_distinct.json | 3 +- .../queries/function_sum_distinct.proto.bin | Bin 174 -> 176 bytes .../query-tests/queries/function_tan.json | 3 +- .../queries/function_tan.proto.bin | Bin 172 -> 174 bytes .../query-tests/queries/function_tanh.json | 3 +- .../queries/function_tanh.proto.bin | Bin 173 -> 175 bytes .../queries/function_timestamp_add.json | 3 +- .../queries/function_timestamp_add.proto.bin | Bin 144 -> 146 bytes .../queries/function_timestamp_diff.json | 3 +- .../queries/function_timestamp_diff.proto.bin | Bin 145 -> 147 bytes .../queries/function_timestamp_micros.json | 3 +- .../function_timestamp_micros.proto.bin | Bin 130 -> 132 bytes .../queries/function_timestamp_millis.json | 3 +- .../function_timestamp_millis.proto.bin | Bin 130 -> 132 bytes .../queries/function_timestamp_seconds.json | 3 +- .../function_timestamp_seconds.proto.bin | Bin 131 -> 133 bytes .../queries/function_to_binary.json | 3 +- .../queries/function_to_binary.proto.bin | Bin 178 -> 180 bytes .../function_to_binary_with_format.json | 3 +- .../function_to_binary_with_format.proto.bin | Bin 189 -> 191 bytes .../query-tests/queries/function_to_char.json | 3 +- .../queries/function_to_char.proto.bin | Bin 188 -> 190 bytes .../query-tests/queries/function_to_csv.json | 6 +- .../queries/function_to_csv.proto.bin | Bin 200 -> 204 bytes .../query-tests/queries/function_to_date.json | 3 +- .../queries/function_to_date.proto.bin | Bin 121 -> 123 bytes .../queries/function_to_date_with_format.json | 3 +- .../function_to_date_with_format.proto.bin | Bin 138 -> 140 bytes .../query-tests/queries/function_to_json.json | 6 +- .../queries/function_to_json.proto.bin | Bin 222 -> 226 bytes .../queries/function_to_number.json | 3 +- .../queries/function_to_number.proto.bin | Bin 190 -> 192 bytes .../queries/function_to_timestamp.json | 3 +- .../queries/function_to_timestamp.proto.bin | Bin 126 -> 128 bytes .../queries/function_to_timestamp_ltz.json | 3 +- .../function_to_timestamp_ltz.proto.bin | Bin 185 -> 187 bytes ...function_to_timestamp_ltz_with_format.json | 3 +- ...ion_to_timestamp_ltz_with_format.proto.bin | Bin 192 -> 194 bytes .../queries/function_to_timestamp_ntz.json | 3 +- .../function_to_timestamp_ntz.proto.bin | Bin 185 -> 187 bytes ...function_to_timestamp_ntz_with_format.json | 3 +- ...ion_to_timestamp_ntz_with_format.proto.bin | Bin 192 -> 194 bytes .../function_to_timestamp_with_format.json | 3 +- ...unction_to_timestamp_with_format.proto.bin | Bin 157 -> 159 bytes .../queries/function_to_unix_timestamp.json | 3 +- .../function_to_unix_timestamp.proto.bin | Bin 186 -> 188 bytes ...unction_to_unix_timestamp_with_format.json | 3 +- ...on_to_unix_timestamp_with_format.proto.bin | Bin 193 -> 195 bytes .../queries/function_to_utc_timestamp.json | 3 +- .../function_to_utc_timestamp.proto.bin | Bin 143 -> 145 bytes .../queries/function_to_varchar.json | 3 +- .../queries/function_to_varchar.proto.bin | Bin 191 -> 193 bytes .../queries/function_transform.json | 6 +- .../queries/function_transform.proto.bin | Bin 211 -> 215 bytes .../queries/function_transform_keys.json | 6 +- .../queries/function_transform_keys.proto.bin | Bin 243 -> 247 bytes .../queries/function_transform_values.json | 3 +- .../function_transform_values.proto.bin | Bin 230 -> 232 bytes .../function_transform_with_index.json | 6 +- .../function_transform_with_index.proto.bin | Bin 221 -> 225 bytes .../queries/function_translate.json | 3 +- .../queries/function_translate.proto.bin | Bin 196 -> 198 bytes .../query-tests/queries/function_trim.json | 3 +- .../queries/function_trim.proto.bin | Bin 173 -> 175 bytes .../queries/function_trim_with_pattern.json | 3 +- .../function_trim_with_pattern.proto.bin | Bin 182 -> 184 bytes .../query-tests/queries/function_trunc.json | 3 +- .../queries/function_trunc.proto.bin | Bin 127 -> 129 bytes .../query-tests/queries/function_try_add.json | 3 +- .../queries/function_try_add.proto.bin | Bin 183 -> 185 bytes .../queries/function_try_aes_decrypt.json | 3 +- .../function_try_aes_decrypt.proto.bin | Bin 191 -> 193 bytes .../function_try_aes_decrypt_with_mode.json | 3 +- ...nction_try_aes_decrypt_with_mode.proto.bin | Bin 198 -> 200 bytes ...ion_try_aes_decrypt_with_mode_padding.json | 3 +- ...ry_aes_decrypt_with_mode_padding.proto.bin | Bin 205 -> 207 bytes ...try_aes_decrypt_with_mode_padding_aad.json | 3 +- ...es_decrypt_with_mode_padding_aad.proto.bin | Bin 212 -> 214 bytes .../query-tests/queries/function_try_avg.json | 3 +- .../queries/function_try_avg.proto.bin | Bin 176 -> 178 bytes .../queries/function_try_divide.json | 3 +- .../queries/function_try_divide.proto.bin | Bin 186 -> 188 bytes .../function_try_element_at_array.json | 3 +- .../function_try_element_at_array.proto.bin | Bin 190 -> 192 bytes .../queries/function_try_element_at_map.json | 3 +- .../function_try_element_at_map.proto.bin | Bin 190 -> 192 bytes .../function_try_make_interval_years.json | 3 +- ...function_try_make_interval_years.proto.bin | Bin 186 -> 188 bytes ...nction_try_make_interval_years_months.json | 3 +- ...n_try_make_interval_years_months.proto.bin | Bin 193 -> 195 bytes ..._try_make_interval_years_months_weeks.json | 3 +- ...make_interval_years_months_weeks.proto.bin | Bin 200 -> 202 bytes ...make_interval_years_months_weeks_days.json | 3 +- ...interval_years_months_weeks_days.proto.bin | Bin 207 -> 209 bytes ...nterval_years_months_weeks_days_hours.json | 3 +- ...al_years_months_weeks_days_hours.proto.bin | Bin 214 -> 216 bytes ...al_years_months_weeks_days_hours_mins.json | 3 +- ...ars_months_weeks_days_hours_mins.proto.bin | Bin 221 -> 223 bytes ...ars_months_weeks_days_hours_mins_secs.json | 3 +- ...onths_weeks_days_hours_mins_secs.proto.bin | Bin 228 -> 230 bytes ..._try_make_timestamp_ltz_with_timezone.json | 3 +- ...make_timestamp_ltz_with_timezone.proto.bin | Bin 233 -> 235 bytes ...y_make_timestamp_ltz_without_timezone.json | 3 +- ...e_timestamp_ltz_without_timezone.proto.bin | Bin 226 -> 228 bytes .../function_try_make_timestamp_ntz.json | 3 +- .../function_try_make_timestamp_ntz.proto.bin | Bin 226 -> 228 bytes ...tion_try_make_timestamp_with_timezone.json | 3 +- ...try_make_timestamp_with_timezone.proto.bin | Bin 229 -> 231 bytes ...n_try_make_timestamp_without_timezone.json | 3 +- ..._make_timestamp_without_timezone.proto.bin | Bin 222 -> 224 bytes .../queries/function_try_multiply.json | 3 +- .../queries/function_try_multiply.proto.bin | Bin 188 -> 190 bytes .../queries/function_try_parse_json.json | 3 +- .../queries/function_try_parse_json.proto.bin | Bin 183 -> 185 bytes .../queries/function_try_parse_url.json | 3 +- .../queries/function_try_parse_url.proto.bin | Bin 189 -> 191 bytes .../function_try_parse_url_with_key.json | 3 +- .../function_try_parse_url_with_key.proto.bin | Bin 196 -> 198 bytes .../queries/function_try_reflect.json | 3 +- .../queries/function_try_reflect.proto.bin | Bin 216 -> 218 bytes .../queries/function_try_subtract.json | 3 +- .../queries/function_try_subtract.proto.bin | Bin 188 -> 190 bytes .../query-tests/queries/function_try_sum.json | 3 +- .../queries/function_try_sum.proto.bin | Bin 176 -> 178 bytes .../queries/function_try_to_binary.json | 3 +- .../queries/function_try_to_binary.proto.bin | Bin 194 -> 196 bytes ...function_try_to_binary_without_format.json | 3 +- ...ion_try_to_binary_without_format.proto.bin | Bin 182 -> 184 bytes .../queries/function_try_to_number.json | 3 +- .../queries/function_try_to_number.proto.bin | Bin 194 -> 196 bytes .../queries/function_try_to_timestamp.json | 3 +- .../function_try_to_timestamp.proto.bin | Bin 192 -> 194 bytes ...ction_try_to_timestamp_without_format.json | 3 +- ..._try_to_timestamp_without_format.proto.bin | Bin 185 -> 187 bytes .../queries/function_try_url_decode.json | 3 +- .../queries/function_try_url_decode.proto.bin | Bin 183 -> 185 bytes .../queries/function_try_variant_get.json | 6 +- .../function_try_variant_get.proto.bin | Bin 216 -> 220 bytes .../query-tests/queries/function_typeof.json | 3 +- .../queries/function_typeof.proto.bin | Bin 175 -> 177 bytes .../query-tests/queries/function_ucase.json | 3 +- .../queries/function_ucase.proto.bin | Bin 174 -> 176 bytes .../queries/function_unbase64.json | 3 +- .../queries/function_unbase64.proto.bin | Bin 177 -> 179 bytes .../query-tests/queries/function_unhex.json | 3 +- .../queries/function_unhex.proto.bin | Bin 174 -> 176 bytes .../queries/function_unix_date.json | 6 +- .../queries/function_unix_date.proto.bin | Bin 153 -> 157 bytes .../queries/function_unix_micros.json | 6 +- .../queries/function_unix_micros.proto.bin | Bin 174 -> 178 bytes .../queries/function_unix_millis.json | 6 +- .../queries/function_unix_millis.proto.bin | Bin 174 -> 178 bytes .../queries/function_unix_seconds.json | 6 +- .../queries/function_unix_seconds.proto.bin | Bin 175 -> 179 bytes .../queries/function_unix_timestamp.json | 6 +- .../queries/function_unix_timestamp.proto.bin | Bin 145 -> 149 bytes .../function_unix_timestamp_with_format.json | 3 +- ...ction_unix_timestamp_with_format.proto.bin | Bin 159 -> 161 bytes .../query-tests/queries/function_upper.json | 3 +- .../queries/function_upper.proto.bin | Bin 174 -> 176 bytes .../queries/function_url_decode.json | 3 +- .../queries/function_url_decode.proto.bin | Bin 179 -> 181 bytes .../queries/function_url_encode.json | 3 +- .../queries/function_url_encode.proto.bin | Bin 179 -> 181 bytes .../query-tests/queries/function_user.json | 3 +- .../queries/function_user.proto.bin | Bin 166 -> 168 bytes .../query-tests/queries/function_var_pop.json | 3 +- .../queries/function_var_pop.proto.bin | Bin 176 -> 178 bytes .../queries/function_var_samp.json | 3 +- .../queries/function_var_samp.proto.bin | Bin 177 -> 179 bytes .../queries/function_variance.json | 3 +- .../queries/function_variance.proto.bin | Bin 177 -> 179 bytes .../queries/function_variant_get.json | 6 +- .../queries/function_variant_get.proto.bin | Bin 212 -> 216 bytes .../query-tests/queries/function_weekday.json | 3 +- .../queries/function_weekday.proto.bin | Bin 121 -> 123 bytes .../queries/function_weekofyear.json | 3 +- .../queries/function_weekofyear.proto.bin | Bin 124 -> 126 bytes .../query-tests/queries/function_window.json | 3 +- .../queries/function_window.proto.bin | Bin 163 -> 165 bytes .../queries/function_window_time.json | 3 +- .../queries/function_window_time.proto.bin | Bin 181 -> 183 bytes .../query-tests/queries/function_xpath.json | 3 +- .../queries/function_xpath.proto.bin | Bin 136 -> 138 bytes .../queries/function_xpath_boolean.json | 3 +- .../queries/function_xpath_boolean.proto.bin | Bin 137 -> 139 bytes .../queries/function_xpath_double.json | 3 +- .../queries/function_xpath_double.proto.bin | Bin 136 -> 138 bytes .../queries/function_xpath_float.json | 3 +- .../queries/function_xpath_float.proto.bin | Bin 135 -> 137 bytes .../queries/function_xpath_int.json | 3 +- .../queries/function_xpath_int.proto.bin | Bin 132 -> 135 bytes .../queries/function_xpath_long.json | 3 +- .../queries/function_xpath_long.proto.bin | Bin 133 -> 136 bytes .../queries/function_xpath_number.json | 3 +- .../queries/function_xpath_number.proto.bin | Bin 136 -> 138 bytes .../queries/function_xpath_short.json | 3 +- .../queries/function_xpath_short.proto.bin | Bin 135 -> 137 bytes .../queries/function_xpath_string.json | 3 +- .../queries/function_xpath_string.proto.bin | Bin 136 -> 138 bytes .../queries/function_xxhash64.json | 3 +- .../queries/function_xxhash64.proto.bin | Bin 199 -> 201 bytes .../query-tests/queries/function_year.json | 3 +- .../queries/function_year.proto.bin | Bin 118 -> 120 bytes .../query-tests/queries/function_years.json | 3 +- .../queries/function_years.proto.bin | Bin 174 -> 176 bytes .../queries/function_zip_with.json | 6 +- .../queries/function_zip_with.proto.bin | Bin 227 -> 231 bytes .../query-tests/queries/groupby_agg.json | 24 +++-- .../query-tests/queries/groupby_agg.proto.bin | Bin 210 -> 226 bytes .../queries/groupby_agg_columns.json | 6 +- .../queries/groupby_agg_columns.proto.bin | Bin 90 -> 94 bytes .../queries/groupby_agg_string.json | 6 +- .../queries/groupby_agg_string.proto.bin | Bin 107 -> 111 bytes .../query-tests/queries/groupby_avg.json | 6 +- .../query-tests/queries/groupby_avg.proto.bin | Bin 94 -> 98 bytes .../query-tests/queries/groupby_count.json | 3 +- .../queries/groupby_count.proto.bin | Bin 86 -> 88 bytes .../query-tests/queries/groupby_max.json | 6 +- .../query-tests/queries/groupby_max.proto.bin | Bin 94 -> 98 bytes .../query-tests/queries/groupby_mean.json | 6 +- .../queries/groupby_mean.proto.bin | Bin 94 -> 98 bytes .../query-tests/queries/groupby_min.json | 6 +- .../query-tests/queries/groupby_min.proto.bin | Bin 94 -> 98 bytes .../query-tests/queries/groupby_sum.json | 6 +- .../query-tests/queries/groupby_sum.proto.bin | Bin 94 -> 98 bytes .../query-tests/queries/groupingSets.json | 6 +- .../queries/groupingSets.proto.bin | Bin 106 -> 110 bytes .../queries/grouping_and_grouping_id.json | 9 +- .../grouping_and_grouping_id.proto.bin | Bin 142 -> 148 bytes .../query-tests/queries/hll_sketch_agg.json | 3 +- .../queries/hll_sketch_agg.proto.bin | Bin 77 -> 79 bytes .../hll_sketch_agg_with_columnName.json | 3 +- .../hll_sketch_agg_with_columnName.proto.bin | Bin 77 -> 79 bytes ...tch_agg_with_columnName_lgConfigK_int.json | 3 +- ...gg_with_columnName_lgConfigK_int.proto.bin | Bin 83 -> 85 bytes .../hll_sketch_agg_with_column_lgConfigK.json | 3 +- ...sketch_agg_with_column_lgConfigK.proto.bin | Bin 83 -> 85 bytes ..._sketch_agg_with_column_lgConfigK_int.json | 3 +- ...ch_agg_with_column_lgConfigK_int.proto.bin | Bin 83 -> 85 bytes .../query-tests/queries/hll_union_agg.json | 3 +- .../queries/hll_union_agg.proto.bin | Bin 76 -> 78 bytes .../hll_union_agg_with_columnName.json | 3 +- .../hll_union_agg_with_columnName.proto.bin | Bin 76 -> 78 bytes ...nName_allowDifferentLgConfigK_boolean.json | 3 +- ..._allowDifferentLgConfigK_boolean.proto.bin | Bin 82 -> 84 bytes ...g_with_column_allowDifferentLgConfigK.json | 3 +- ...h_column_allowDifferentLgConfigK.proto.bin | Bin 82 -> 84 bytes ...olumn_allowDifferentLgConfigK_boolean.json | 3 +- ..._allowDifferentLgConfigK_boolean.proto.bin | Bin 82 -> 84 bytes .../query-tests/queries/join_condition.json | 3 +- .../queries/join_condition.proto.bin | Bin 150 -> 152 bytes .../queries/join_inner_condition.json | 3 +- .../queries/join_inner_condition.proto.bin | Bin 148 -> 150 bytes .../resources/query-tests/queries/pivot.json | 3 +- .../query-tests/queries/pivot.proto.bin | Bin 99 -> 101 bytes .../queries/pivot_without_column_values.json | 3 +- .../pivot_without_column_values.proto.bin | Bin 87 -> 89 bytes .../query-tests/queries/rollup_column.json | 3 +- .../queries/rollup_column.proto.bin | Bin 92 -> 94 bytes .../query-tests/queries/rollup_string.json | 3 +- .../queries/rollup_string.proto.bin | Bin 96 -> 98 bytes .../queries/select_typed_1-arg.json | 9 +- .../queries/select_typed_1-arg.proto.bin | Bin 98 -> 104 bytes .../queries/select_typed_2-arg.json | 6 +- .../queries/select_typed_2-arg.proto.bin | Bin 101 -> 105 bytes .../queries/select_typed_3-arg.json | 9 +- .../queries/select_typed_3-arg.proto.bin | Bin 128 -> 135 bytes .../queries/select_typed_4-arg.json | 12 ++- .../queries/select_typed_4-arg.proto.bin | Bin 157 -> 165 bytes .../queries/select_typed_5-arg.json | 15 ++- .../queries/select_typed_5-arg.proto.bin | Bin 182 -> 192 bytes .../resources/query-tests/queries/toJSON.json | 6 +- .../query-tests/queries/toJSON.proto.bin | Bin 187 -> 191 bytes .../queries/to_avro_with_schema.json | 3 +- .../queries/to_avro_with_schema.proto.bin | Bin 103 -> 105 bytes .../queries/to_avro_without_schema.json | 3 +- .../queries/to_avro_without_schema.proto.bin | Bin 69 -> 71 bytes .../queries/to_protobuf_messageClassName.json | 3 +- .../to_protobuf_messageClassName.proto.bin | Bin 123 -> 125 bytes ...rotobuf_messageClassName_descFilePath.json | 3 +- ...uf_messageClassName_descFilePath.proto.bin | Bin 359 -> 361 bytes ...messageClassName_descFilePath_options.json | 6 +- ...geClassName_descFilePath_options.proto.bin | Bin 407 -> 411 bytes .../to_protobuf_messageClassName_options.json | 6 +- ...rotobuf_messageClassName_options.proto.bin | Bin 172 -> 176 bytes .../query-tests/queries/where_column.json | 3 +- .../queries/where_column.proto.bin | Bin 69 -> 71 bytes .../query-tests/queries/width_bucket.json | 3 +- .../queries/width_bucket.proto.bin | Bin 94 -> 96 bytes .../resources/query-tests/queries/window.json | 21 ++-- .../query-tests/queries/window.proto.bin | Bin 344 -> 358 bytes .../connect/planner/SparkConnectPlanner.scala | 18 ++-- .../planner/SparkConnectPlannerSuite.scala | 35 ++++++- 1175 files changed, 1522 insertions(+), 756 deletions(-) diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala index 8d57a8d3efd44..7802d9750bbc3 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/internal/columnNodeSupport.scala @@ -73,13 +73,19 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { .setColName(regex) planId.foreach(b.setPlanId) - case UnresolvedFunction(functionName, arguments, isDistinct, isUserDefinedFunction, _, _) => - // TODO(SPARK-49087) use internal namespace. + case UnresolvedFunction( + functionName, + arguments, + isDistinct, + isUserDefinedFunction, + isInternal, + _) => builder.getUnresolvedFunctionBuilder .setFunctionName(functionName) .setIsUserDefinedFunction(isUserDefinedFunction) .setIsDistinct(isDistinct) .addAllArguments(arguments.map(apply(_, e)).asJava) + .setIsInternal(isInternal) case Alias(child, name, metadata, _) => val b = builder.getAliasBuilder.setExpr(apply(child, e)) @@ -156,6 +162,7 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { case CaseWhenOtherwise(branches, otherwise, _) => val b = builder.getUnresolvedFunctionBuilder .setFunctionName("when") + .setIsInternal(false) branches.foreach { case (condition, value) => b.addArguments(apply(condition, e)) b.addArguments(apply(value, e)) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala index 4cb03420c4d07..94729d34f37b5 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/internal/ColumnNodeToProtoConverterSuite.scala @@ -128,19 +128,22 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { .setFunctionName("+") .setIsDistinct(false) .addArguments(attribute("a")) - .addArguments(expr(_.getLiteralBuilder.setInteger(1))))) + .addArguments(expr(_.getLiteralBuilder.setInteger(1))) + .setIsInternal(false))) testConversion( UnresolvedFunction( "db1.myAgg", Seq(UnresolvedAttribute("a")), isDistinct = true, - isUserDefinedFunction = true), + isUserDefinedFunction = true, + isInternal = true), expr( _.getUnresolvedFunctionBuilder .setFunctionName("db1.myAgg") .setIsDistinct(true) .setIsUserDefinedFunction(true) - .addArguments(attribute("a")))) + .addArguments(attribute("a")) + .setIsInternal(true))) } test("alias") { @@ -247,10 +250,12 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { expr( _.getWindowBuilder .setWindowFunction( - expr(_.getUnresolvedFunctionBuilder - .setFunctionName("sum") - .setIsDistinct(false) - .addArguments(attribute("a")))) + expr( + _.getUnresolvedFunctionBuilder + .setFunctionName("sum") + .setIsDistinct(false) + .addArguments(attribute("a")) + .setIsInternal(false))) .addPartitionSpec(attribute("b")) .addPartitionSpec(attribute("c")) .addOrderSpec(proto.Expression.SortOrder @@ -276,7 +281,8 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { _.getUnresolvedFunctionBuilder .setFunctionName("sum") .setIsDistinct(false) - .addArguments(attribute("a")))) + .addArguments(attribute("a")) + .setIsInternal(false))) .addPartitionSpec(attribute("b")) .addPartitionSpec(attribute("c")))) testWindowFrame( @@ -310,7 +316,8 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { _.getUnresolvedFunctionBuilder .setFunctionName("+") .addArguments(expr(_.setUnresolvedNamedLambdaVariable(catX))) - .addArguments(attribute("y")))) + .addArguments(attribute("y")) + .setIsInternal(false))) .addArguments(catX))) } @@ -330,7 +337,8 @@ class ColumnNodeToProtoConverterSuite extends ConnectFunSuite { .setFunctionName("when") .addArguments(attribute("c1")) .addArguments(expr(_.getLiteralBuilder.setString("r1"))) - .addArguments(expr(_.getLiteralBuilder.setString("fallback"))))) + .addArguments(expr(_.getLiteralBuilder.setString("fallback"))) + .setIsInternal(false))) } test("extract field") { diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 093997a0d0c56..7edcbcac15c73 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.py +++ b/python/pyspark/sql/connect/proto/expressions_pb2.py @@ -40,7 +40,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"\xe1\x31\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\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\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_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe1\x01\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType"]\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x42\x36\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"\x97\x32\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\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\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\x82\x02\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\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\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_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe1\x01\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType"]\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,7 +54,7 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_EXPRESSION"]._serialized_start = 133 - _globals["_EXPRESSION"]._serialized_end = 6502 + _globals["_EXPRESSION"]._serialized_end = 6556 _globals["_EXPRESSION_WINDOW"]._serialized_start = 2060 _globals["_EXPRESSION_WINDOW"]._serialized_end = 2843 _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2350 @@ -88,49 +88,49 @@ _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5158 _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5344 _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5347 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5551 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5553 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5603 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5605 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5729 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5731 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5817 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5820 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5952 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5955 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6142 - _globals["_EXPRESSION_ALIAS"]._serialized_start = 6144 - _globals["_EXPRESSION_ALIAS"]._serialized_end = 6264 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6267 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6425 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6427 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6489 - _globals["_EXPRESSIONCOMMON"]._serialized_start = 6504 - _globals["_EXPRESSIONCOMMON"]._serialized_end = 6569 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6572 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6936 - _globals["_PYTHONUDF"]._serialized_start = 6939 - _globals["_PYTHONUDF"]._serialized_end = 7143 - _globals["_SCALARSCALAUDF"]._serialized_start = 7146 - _globals["_SCALARSCALAUDF"]._serialized_end = 7360 - _globals["_JAVAUDF"]._serialized_start = 7363 - _globals["_JAVAUDF"]._serialized_end = 7512 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7514 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7613 - _globals["_CALLFUNCTION"]._serialized_start = 7615 - _globals["_CALLFUNCTION"]._serialized_end = 7723 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7725 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7817 - _globals["_MERGEACTION"]._serialized_start = 7820 - _globals["_MERGEACTION"]._serialized_end = 8332 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8042 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8148 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8151 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8318 - _globals["_LAZYEXPRESSION"]._serialized_start = 8334 - _globals["_LAZYEXPRESSION"]._serialized_end = 8399 - _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8402 - _globals["_SUBQUERYEXPRESSION"]._serialized_end = 8627 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 8534 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 8627 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5605 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5607 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5657 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5659 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5783 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5785 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5871 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5874 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 6006 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 6009 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6196 + _globals["_EXPRESSION_ALIAS"]._serialized_start = 6198 + _globals["_EXPRESSION_ALIAS"]._serialized_end = 6318 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6321 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6479 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6481 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6543 + _globals["_EXPRESSIONCOMMON"]._serialized_start = 6558 + _globals["_EXPRESSIONCOMMON"]._serialized_end = 6623 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6626 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6990 + _globals["_PYTHONUDF"]._serialized_start = 6993 + _globals["_PYTHONUDF"]._serialized_end = 7197 + _globals["_SCALARSCALAUDF"]._serialized_start = 7200 + _globals["_SCALARSCALAUDF"]._serialized_end = 7414 + _globals["_JAVAUDF"]._serialized_start = 7417 + _globals["_JAVAUDF"]._serialized_end = 7566 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7568 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7667 + _globals["_CALLFUNCTION"]._serialized_start = 7669 + _globals["_CALLFUNCTION"]._serialized_end = 7777 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7779 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7871 + _globals["_MERGEACTION"]._serialized_start = 7874 + _globals["_MERGEACTION"]._serialized_end = 8386 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8096 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8202 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8205 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8372 + _globals["_LAZYEXPRESSION"]._serialized_start = 8388 + _globals["_LAZYEXPRESSION"]._serialized_end = 8453 + _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8456 + _globals["_SUBQUERYEXPRESSION"]._serialized_end = 8681 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 8588 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 8681 # @@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 0a6f3caee8b54..1a8c60f673054 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -847,6 +847,7 @@ class Expression(google.protobuf.message.Message): ARGUMENTS_FIELD_NUMBER: builtins.int IS_DISTINCT_FIELD_NUMBER: builtins.int IS_USER_DEFINED_FUNCTION_FIELD_NUMBER: builtins.int + IS_INTERNAL_FIELD_NUMBER: builtins.int function_name: builtins.str """(Required) name (or unparsed name for user defined function) for the unresolved function.""" @property @@ -864,6 +865,11 @@ class Expression(google.protobuf.message.Message): When it is not a user defined function, Connect will use the function name directly. When it is a user defined function, Connect will parse the function name first. """ + is_internal: builtins.bool + """(Optional) Indicate if this function is defined in the internal function registry. + If not set, the server will try to look up the function in the internal function registry + and decide appropriately. + """ def __init__( self, *, @@ -871,20 +877,34 @@ class Expression(google.protobuf.message.Message): arguments: collections.abc.Iterable[global___Expression] | None = ..., is_distinct: builtins.bool = ..., is_user_defined_function: builtins.bool = ..., + is_internal: builtins.bool | None = ..., ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "_is_internal", b"_is_internal", "is_internal", b"is_internal" + ], + ) -> builtins.bool: ... def ClearField( self, field_name: typing_extensions.Literal[ + "_is_internal", + b"_is_internal", "arguments", b"arguments", "function_name", b"function_name", "is_distinct", b"is_distinct", + "is_internal", + b"is_internal", "is_user_defined_function", b"is_user_defined_function", ], ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["_is_internal", b"_is_internal"] + ) -> typing_extensions.Literal["is_internal"] | None: ... class ExpressionString(google.protobuf.message.Message): """Expression as string.""" diff --git a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto index a01b5229a7b74..bbe605a47f4ff 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -261,6 +261,11 @@ message Expression { // When it is not a user defined function, Connect will use the function name directly. // When it is a user defined function, Connect will parse the function name first. bool is_user_defined_function = 4; + + // (Optional) Indicate if this function is defined in the internal function registry. + // If not set, the server will try to look up the function in the internal function registry + // and decide appropriately. + optional bool is_internal = 5; } // Expression as string. diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_add.json b/sql/connect/common/src/test/resources/query-tests/queries/column_add.json index cfa40fac8c6f9..3b8219884aa0b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_add.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_add.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_add.proto.bin index 10b410b5b08b5391e052221f1c78c50c60c5d016..9b9889cf775fece77483cda64569204f0f44beda 100644 GIT binary patch delta 37 ocmdnUxS5fSi%Eb{YRyEp>0+W%!d#5nLaai}T#ShjDoKL@0Dd3^WdHyG delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SN+Laai}T#ShjDhU8`vIR8& diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_and.json b/sql/connect/common/src/test/resources/query-tests/queries/column_and.json index d3f8cd0e73cbc..2fa4c654cce1d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_and.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_and.json @@ -25,7 +25,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -38,9 +39,11 @@ "literal": { "double": 0.5 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_and.proto.bin index 241f1a9303b2cab44f6315b6109bd6b2c98a2908..99111eba7191cf0186c8801c12ef092c38d94b37 100644 GIT binary patch delta 57 zcmcb@c#DyZi%Eb{>heUk=}Ig+_e=?Z32CS1&kc_~6dQUY9zc0#N|%v_9#LM&WN23$gtQsNW$ Givs`zPYD|U diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_between.json b/sql/connect/common/src/test/resources/query-tests/queries/column_between.json index 20927b93d8438..de970b1cdf343 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_between.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_between.json @@ -25,7 +25,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -38,9 +39,11 @@ "literal": { "integer": 20 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_between.proto.bin index d03dd02a2f36a39ac6c146a07ddcc7995cafbb17..324e26b35750a15ca9553391dd113a31b19038ad 100644 GIT binary patch delta 71 zcmX@lc$JZji%Eb{>fA)O=@#Zvrd-U4c_~7oQo>wJcD6#SLd;x@i9#$~Oa@#U3=m-( L6k!n!1`P%P$5#pb delta 65 zcmcc0c%G4si%Eb{>cm90=|+Z9`drM3c_~7|Qi5DecD6#SLd;x@i9#$~Oa@#KK^qi7 F5deoB31k2O diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json index bd3ac671fca33..71f6d6b3ec3ab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.json @@ -22,7 +22,8 @@ "literal": { "integer": 255 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseAND.proto.bin index 4815bc7dd1a20f973bbbaa37ac0ebb8ad51efa4b..13e798120873a783306969cfa483b132ae0a5af7 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!d#4MLaai}T#SiAtX#|n{~0wH0D&e38vp7pW1LR^eBLaai}T#SiAEL=>3OJeE$V- delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$hwWLaai}T#SiAEL=;Q3t1wa4* diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json index c51eb3140c339..108d66745d34d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.json @@ -22,7 +22,8 @@ "literal": { "integer": 78 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_bitwiseXOR.proto.bin index 70c61f9620576c79f35b55a8a9f833fbbc1c8364..994283d7a94315cfc21228fce784b42c2b7fe9d7 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR^e-Laai}T#SiAEL=>B2%%0$hx7Laai}T#SiAEL=CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWF0+W%!d#5{Laai}T#ShjDoKL@0DfczX#fBK delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SOHLaai}T#ShjDhU8`?gcpj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.json b/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.json index f4171c2792fbd..5ee6cfe40b1ec 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.json @@ -22,7 +22,8 @@ "literal": { "string": "suffix_" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_endsWith.proto.bin index 03f41a339f00cfabba8079614d2c358c0698e3de..3f3db0c90bc19bdb0885d7972aa27f3287d2e77d 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json b/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json index eea1da49bc59e..44e11ad2b8942 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_eqNullSafe.proto.bin index 22de941ad44b005bef12386de58e75c2cff87dc7..0614560048a9b6708b79222752e8ed2bea1f5c71 100644 GIT binary patch delta 39 qcmdnYxRsHOi%Eb{YTZP(=@Q~nqFl^2wsu0SLd;x@i4ZDDg8=}MYX$!R delta 37 ocmdnWxS5fSi%Eb{YRyEp>0+W%!d%QYwsu0SLd;x@i4ZCY0ED>(&j0`b diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_equals.json b/sql/connect/common/src/test/resources/query-tests/queries/column_equals.json index 7397f4fb46acd..1f05b249eb00b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_equals.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_equals.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_equals.proto.bin index e226de59ddcd42064e6b17cee5b04e9961de3b8a..cad0e9b14a81452c1370dcd47edc688be43b12e1 100644 GIT binary patch delta 37 ocmdnUxS5fSi%Eb{YRyEp>0+W%!d#5DLaai}T#ShjDoKL@0Dn>icK`qY delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SNYLaai}T#ShjDhU8{zy(JD diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_geq.json b/sql/connect/common/src/test/resources/query-tests/queries/column_geq.json index 9f24bc251739f..4c7f5339409f0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_geq.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_geq.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_geq.proto.bin index 1c4af866109abcab5d43441026193b4c99206642..a68ee6cc8b6a7d69211e2e3eef566e1502e9614d 100644 GIT binary patch delta 38 pcmdnQxP_68i%Eb{YVAa}>EdEiB3w*%wnD5z%v_9#5GqN70RV_h1-t+N delta 36 ncmdnOxQUUCi%Eb{YV}06>7pW1LR?ICwnD5z%v_9#5Gn}(d?*Es diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_gt.json b/sql/connect/common/src/test/resources/query-tests/queries/column_gt.json index 4bb8fb41f249d..74be85e709ed5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_gt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_gt.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_gt.proto.bin index 44ca37fbb404821ea9e49c404bcd8d872799b694..e85f5a3e23552b01014a1173200ae831c0412023 100644 GIT binary patch delta 37 ocmdnUxS5fSi%Eb{YRyEp>0+W%!d#4YLaai}T#ShjDoKL@0DofzcmMzZ delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SMtLaai}T#ShjDhU8{&jm;T diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.json b/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.json index 47c1b63abe319..60a62c4595ac8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.json @@ -22,7 +22,8 @@ "literal": { "string": "%fOb%" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_ilike.proto.bin index 285400db7daf5a5c5045f89905de51f81bad5f92..368bebd9ea48d683aafd6e472cc348a8d44e4cf6 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.json b/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.json index f594918ed930a..12d3d19d7797a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_isNaN.proto.bin index 1030abda5b8c2ac243e2e880eca30bcdb153c88f..8c2fad75be346d4d3da6765a9ca193935aea1415 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+_#eRu?Laai}T#QK?3;=z>25kTU delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%V#eRu?Laai}T#QKoa9ajF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json b/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json index f34d3f4eac552..6af0e5bfdb4e3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_isNotNull.proto.bin index e8cccdf024934f913ad2a3387bd39e696c980c38..cdc382e44ee22d3a78c7539e9721a97be504a753 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zuA#eVrEex*4%Laai}T#V@&3;?XC2weaG delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4g#eVrEex*4%Laai}T#V@en?eXJ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.json b/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.json index 74e990622a3a7..a6ac6534ecd55 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_isNull.proto.bin index 8fc24a9e21b38735e94407b294da2d6633ce14d4..0ea4d6f2ffe894e2cabc8ce1a536bea04bfa8f1c 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RBw#eStZIYO*L%v_A=8Vmr6mIoIA delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J6A#eStZIYO*L%v_A=0DII1=Kufz diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isin.json b/sql/connect/common/src/test/resources/query-tests/queries/column_isin.json index d8811a4e780b5..b34abc941cce8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_isin.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_isin.json @@ -30,7 +30,8 @@ "literal": { "string": "foo" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_isin.proto.bin index 365e07f35bb4864771b3c4fcbbb1a03cf47b2fbf..cde6686dd40641ab1aa207407736c49b7b0bc4a3 100644 GIT binary patch delta 24 gcmX@bc!rUUi%Eb{>gYta>CAdkIujRYF=;RW08How#Q*>R delta 21 dcmX@Zc#4sYi%Eb{>hMIi>C8G(S`!y&0RTIj1(g5* diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_leq.json b/sql/connect/common/src/test/resources/query-tests/queries/column_leq.json index cda8694c0439e..55388f667c448 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_leq.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_leq.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_leq.proto.bin index e8463292e4040ba9632cd395678cb4faf00015d7..692ccad0aa9c344514e9c45922d6fd6060b1db92 100644 GIT binary patch delta 38 pcmdnQxP_68i%Eb{YVAa}>EdEiB3w*1wnD5z%v_9#5GqN70RV_71-bwL delta 36 ncmdnOxQUUCi%Eb{YV}06>7pW1LR?HXwnD5z%v_9#5Gn}(dC7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_lt.json b/sql/connect/common/src/test/resources/query-tests/queries/column_lt.json index c927e75de181b..1264a0e43a54a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_lt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_lt.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_lt.proto.bin index f4c3a110b126be2c5144ba63cf41501c0cba3bb4..083c8d46611d17d397549cb9bbff04433e5e51b6 100644 GIT binary patch delta 37 ocmdnUxS5fSi%Eb{YRyEp>0+W%!d#3tLaai}T#ShjDoKL@0DnORb^rhX delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SL?Laai}T#ShjDhU8{u?0o| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.json b/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.json index 0c5a78eea2dff..d11494f159a5a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.json @@ -22,7 +22,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_modulo.proto.bin index 55bfeba04ed662ba1861709262d6cb28c4d4b863..a86b5e5de63e4304f856e575dd52680d674fa5ee 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR^fhLaai}T#SiAEL=>B2%%0$hx$Laai}T#SiAEL=0+W%!d#46Laai}T#ShjDoKL@0DcbzWB>pF delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SMRLaai}T#ShjDhU8`qXjeo diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_not.json b/sql/connect/common/src/test/resources/query-tests/queries/column_not.json index 2f873196ba1d0..3fa58e874d75d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_not.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_not.json @@ -18,7 +18,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_not.proto.bin index 19609b6ee85a572d5ca885388e0ee0eae722bcde..51ea1c4d20bc7caf3a97746088dc54ce8c6bd4d1 100644 GIT binary patch delta 29 kcmZ3l>3qCW++2)`LM&WN5{w!Q08xqrH2?qr delta 27 icmZ3@xRQ~Li%Eb{YSBct={($0oLr2GLM&WN5{v*yyaWXR diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.json b/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.json index 589d57a18b94b..093770f4563be 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.json @@ -25,9 +25,11 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_not_equals.proto.bin index cdf0b4290e61e399c9e8c201b372776195978904..cc13a11b480135d291629ebea1ddd6437af52175 100644 GIT binary patch delta 46 xcmdnNxQCIAi%Eb{YU@O{>GEC#eC5?qXmLc&slT#U9ttU}COjEN8`2>_k<1@8a= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_or.json b/sql/connect/common/src/test/resources/query-tests/queries/column_or.json index ae1424f763feb..324bfc850d2dc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_or.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_or.json @@ -25,7 +25,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -38,9 +39,11 @@ "literal": { "double": 0.5 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_or.proto.bin index 69f219e938a4e4971fc1d92febc5d870167f4f5a..a52ba0707a755790606615b84eceb26521f4ba96 100644 GIT binary patch delta 75 zcmcc2c$1Nhi%Eb{>e58E={D9dZv8>GGyh##~JKMM6SS0$hxCLaai}T#SiAEL=CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.json b/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.json index 431e13d818639..9a9036b3cf963 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.json @@ -22,7 +22,8 @@ "literal": { "string": "prefix_" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_startsWith.proto.bin index fa1132c73de7b27bbfa33cacad31f7752006063c..366011b3c3968a157f4e1bb1b0718f2d23c87667 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_substr.json b/sql/connect/common/src/test/resources/query-tests/queries/column_substr.json index 3b02117cc6e5b..5beaf7c4c3711 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_substr.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_substr.json @@ -26,7 +26,8 @@ "literal": { "integer": 3 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_substr.proto.bin index 636a46a4806267fa3cbb44df5cd56f31811ee891..5eedae63ea95c987ef41b93e6e4e9f71242e2c75 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.json b/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.json index d15c2941ee1bd..68faab9cbb05a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_subtract.proto.bin index f5716427588ed565c2aa0c4d61630840122677f5..2a341fb5201ed050cb05c7e7c78809904553265d 100644 GIT binary patch delta 37 ocmdnUxS5fSi%Eb{YRyEp>0+W%!d#5HLaai}T#ShjDoKL@0DeLRX8-^I delta 35 mcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?SNcLaai}T#ShjDhU8`&;>UD diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json b/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json index 0db558e49e38c..b4c76e2c6719b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.json @@ -18,7 +18,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_unary_minus.proto.bin index 66343bea4e29b5835bab7e8f5927dc56ee4bcd3e..53277e9dd2452c384340e71194d374158cf29189 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR=hqsp*L&nPsU$EL=>B2%%0$dz(o diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json b/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json index db2ceccfd22ab..f2223c20e569c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.json @@ -25,7 +25,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, { "literal": { @@ -42,7 +43,8 @@ "literal": { "integer": 20 } - }] + }], + "isInternal": false } }, { "literal": { @@ -52,7 +54,8 @@ "literal": { "string": "high" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/column_when_otherwise.proto.bin index 031c3683c5e6d2657955f6434196351253d1c5e8..e22f469ea24908baf6e68ddae9fc61ff758aebe2 100644 GIT binary patch delta 101 zcmaFM_?eN7i%Eb{>di#9>0zN#!CWlm8L4?fB2q$Jj5b27Ld;x@i9#$~Oa@#U3_|Q& mtXa%C`Q=!oMS#*=TpU?!xv43crMW^JTx?k^8JX!B8Vmr%wGr$9 delta 95 zcmey&_?D54i%Eb{>e)oL=>h&yzFaKj8L4?fLQ(=;j5b27Ld;x@i9#$~Oa@#+>|Cr_ h%sKhxSR_S+xVSj7*m6@-GD~xXIJnrdSTZuxGXP@E5mW#G diff --git a/sql/connect/common/src/test/resources/query-tests/queries/cube_column.json b/sql/connect/common/src/test/resources/query-tests/queries/cube_column.json index 5b9709ff06576..b4d884568354b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/cube_column.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/cube_column.json @@ -30,7 +30,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }, "name": ["count"] diff --git a/sql/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/cube_column.proto.bin index d46e40b39dcfec95244ab4af93c4cd4905d60d02..99a704c0c7f07f1dc16d04244b9cecb33b268006 100644 GIT binary patch delta 42 tcma!vW8-2HVDyTZ$Yw1oVDXX9cLU{rFP$fhkODkaRtXe-1f#KOgtnF1m$xHK35PK5*r delta 33 lcmZ>FW#eKJU{tc7$fhkKEG5XrXe-1f#KOgtnF1m$xBx}Z1K0on diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.json b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.json index 662aa746af243..adbc647c186de 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.json @@ -41,9 +41,11 @@ "literal": { "string": "zstandard" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_with_options.proto.bin index 5da5c48b411534bcff5a05d54be44559bc5e599a..eba3a4648ca60bebee5c1dcc596a8e4dd20d79dd 100644 GIT binary patch delta 34 pcmZ3>xQUUCi%Eb{YV|}mEtUpGsk({AR?Lo4_7e+iSu_|l7yyRu2UP$7 delta 28 jcmdnQxR#NPi%Eb{YS~0KE#?}js);66%=S{Y6ANttUV;Yj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.json b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.json index da2840f2d3a0b..0ef3262f1eb4a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.json @@ -22,7 +22,8 @@ "literal": { "string": "{\"type\": \"string\", \"name\": \"name\"}" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/from_avro_without_options.proto.bin index 4dd12e2dbe1dd7f18f2ae997e3acd65b9038f227..629804e8608aa04880b18ecb501975a79c8fd111 100644 GIT binary patch delta 22 dcmXRYV&h^GV3f+4$fn8cB;_#C$c{;a0RS-$1Qq}Q delta 19 acmXRaVB=yEV3f+5$fn8cAZ0hv$PNG`%md~C diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.json b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.json index 375c0f9324c3f..fe2efd928ccf4 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.json @@ -26,7 +26,8 @@ "literal": { "binary": "CvwBCgxjb21tb24ucHJvdG8SDXNwYXJrLmNvbm5lY3QisAEKDFN0b3JhZ2VMZXZlbBIZCgh1c2VfZGlzaxgBIAEoCFIHdXNlRGlzaxIdCgp1c2VfbWVtb3J5GAIgASgIUgl1c2VNZW1vcnkSIAoMdXNlX29mZl9oZWFwGAMgASgIUgp1c2VPZmZIZWFwEiIKDGRlc2VyaWFsaXplZBgEIAEoCFIMZGVzZXJpYWxpemVkEiAKC3JlcGxpY2F0aW9uGAUgASgFUgtyZXBsaWNhdGlvbkIiCh5vcmcuYXBhY2hlLnNwYXJrLmNvbm5lY3QucHJvdG9QAWIGcHJvdG8z" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath.proto.bin index 07d4c6c5b286fc2bd2b5b40dcc59ef8fbc2e925f..db667ef8ee6b247575b60a525a8fba79a93b17b8 100644 GIT binary patch delta 26 icmaFK^qPr{i%Eb{>d8blEtZ{3Qrk9~m@+bHFaQ8z9tMv9 delta 23 fcmaFO^pc5d{0tEtc&}Qd>5fm@)zYPH6^T diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.json b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.json index db9371b64ef72..93974afec3566 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.json @@ -37,9 +37,11 @@ "literal": { "string": "2" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/from_protobuf_messageClassName_descFilePath_options.proto.bin index 00fd58da6be84026c160fad8e6fc4fefd3de53e0..a7262d64522c697c36ac85d89eb0e446e9a8aff0 100644 GIT binary patch delta 36 scmbQqJeQe`i%Eb{YT86LEtc0zQZF`|m@zUNNa;<^Wz=EOV9;Oy0Gt{J+yDRo delta 31 ncmbQsJd>G?i%Eb{YSKhDEtcm@QcpITm@zWzN$E_^Wz+!xbe0Er diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_abs.json b/sql/connect/common/src/test/resources/query-tests/queries/function_abs.json index 13df3437ddabe..aa589275670b8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_abs.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_abs.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_abs.proto.bin index 86cfbc09a8f9192175bf58a216aaa4d9129a6392..6bda0e50aa83759766242d8f3499edabc2f97a0f 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b?lNyS2}Ld;x@i5d(5YrX}$ delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY39)NyS2}Ld;x@i2z^B1&aUx diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_acos.json b/sql/connect/common/src/test/resources/query-tests/queries/function_acos.json index 7506c0f6cb630..82543692456c1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_acos.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_acos.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_acos.proto.bin index cc6a279cb188eca27a5150f60bb93c5b329cac1b..98bc0d821d7d0b8794f6b22c33545c832f6c3ed5 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)F$@#@XtU}COj7b^{0CltmGXMYp delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(1a$@#@XtU}COj7b1!$_4`f diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.json b/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.json index 6a83b4ab008bc..82a69e9f74166 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_acosh.proto.bin index e16ed2ba92e3f62c288343a1050bfcb1a51e238b..48c57cb1c9f9b7297b1825cee97589035879fafd 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+#$@#?@Laai}T#QK?3;=)E2BH7} delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%F$@#?@Laai}T#QKoat8)- diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.json b/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.json index b1b2e78a08435..97b9a00d5ea3e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_add_months.proto.bin index 6abacc9cc2b408396dd9bbcf5eb1e5792d8c8601..c396f24928cf50e83e2d209c46644df7ba4a4a85 100644 GIT binary patch delta 45 zcmZo-Y++>MViI7Ks+-7`ASWv&&Bc|Nk`kYrpI4GmEW|3r%*B`@#KOg7z@)(d0Mj@K AcK`qY delta 43 ycmZo+Y+_{NViI7Ks+q`^AR{d$$;FkJk`kYrpI4GmEW|3r%*B`@#KOg7zytunW(Y?B diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json index 4204a44b44ce0..28beb401cd650 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt.proto.bin index f635e1fc689b1127bebea3b7ad0930f73869006b..40687059a8c451ae01c4389899926ffbb8860f8a 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.json index 9c630e1253494..0436dd1a60c85 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode.proto.bin index 41d024cdb7eed42bd3f921c0e60d77f2596e9733..8e9a324c2fde5cc0296b55eb81e2c31d77269227 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.json index 8f5be474ab4b3..56ad10f6f74bf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_decrypt_with_mode_padding.proto.bin index cd6764581f2caeceb4715a72502bc959c7594c59..97091b52e6c5987696fff05c168677424c986745 100644 GIT binary patch delta 24 gcmX@fc$$%oi%Eb{>c~X4>CC!P+7lOOGHEaX08E$#zW@LL delta 21 dcmX@jc#@Hgi%Eb{>d-{C>CDc!`mXi%Eb{>hwgm>C7flMiUnpFljIV08YpS=>Px# delta 21 dcmcb_c!80Pi%Eb{>f}VW>C8q_1``(;002D?1-SqK diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.json index 06469d4840547..89d07a44e8440 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt.proto.bin index c7a70b51707f321c5e0ca295920fd6a287693c1d..0089323b6bbe1087caa7bcb478e502de7047d3dd 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.json index 7eb9b4ed8b4ed..afef2dba9aad9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode.proto.bin index ecd81ae44fcbd94b4cc80e673f3fd5374a0db33e..3d89f200e609cfe1dd077e8c63f0a48d546d91b2 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.json index 59a6a5e35fd42..8617d2d9d928a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding.proto.bin index 9de01ddc5ea69ea1e7d6afadc38b6de546df6a4f..3888e9a1d075d907bedb765da84c709b9c4c0c36 100644 GIT binary patch delta 24 gcmX@fc$$%oi%Eb{>c~X4>CC!P+7lOOGHEaX08E$#zW@LL delta 21 dcmX@jc#@Hgi%Eb{>d-{C>CDg+_e>C9$QCKDGJF=;RW08eNI^#A|> delta 21 dcmcb@c!`mXi%Eb{>hwgm>C7flMiUnp0RTOV1;qdW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.json index eb0e178fd3534..cb790e822a52f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aes_encrypt_with_mode_padding_iv_aad.proto.bin index ee39beb07cee40266e191880056383ec1a74dc9a..7ff11b9868b994c0baf45bb4ee30c7c8ff855dc0 100644 GIT binary patch delta 24 gcmcb~c$<-pi%Eb{>dHj6>CCoL))N<4GHEaX08y9*9{>OV delta 21 dcmcc3c$1Nhi%Eb{>e58E>CDzrmJ=6P0sucf1@8a= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.json index df1813aed64c5..8e113b8874a5f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.json @@ -35,7 +35,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -55,7 +56,8 @@ "nameParts": ["x_3"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate.proto.bin index c43f4e6dbbc1b5f8b8a07f72f123bb04be7d636a..12456f54ab438bd09d6ab253f362725bf0136d3b 100644 GIT binary patch delta 41 wcmey)_?wZ9i%Eb{>dQp7>CCZG(GwR)vTFxva!E>wPdp&Xq`@%psuGX{0P;Etwg3PC delta 35 rcmey(_??lBi%Eb{>cd2~>CBN*;S(1~vTFvZbBRldPCOtw@wyTKwVn!4 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.json b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.json index 956b42db65639..b46810e63a304 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.json @@ -35,7 +35,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -57,14 +58,16 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }, "arguments": [{ "nameParts": ["x_3"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_aggregate_with_finish_lambda.proto.bin index cf32ea4ddd3e77109f6ddee1087c3148b86b2f08..70fbe778cb7153d5ced4af03d8e80fb7e3e06056 100644 GIT binary patch delta 87 zcmZo+>SAK&ViI7KYG#_qK8-m?Dr@2bNp|fZO)g0(@regSO*I&VSh<)h;tfGmWxSD) fVvszSn3MRjSdq7x5@>I<=QF;~PJf~d-PBO&=9 bSuPPNAudL3A@(A$l(7&C7n1=KM4d4JtV|By diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_any.json b/sql/connect/common/src/test/resources/query-tests/queries/function_any.json index 4512c060d703b..73332625c2995 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_any.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_any.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "flag" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_any.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_any.proto.bin index 9b014b58da57c6c5927ca7d86249c5c51da1fcec..ff038010b22c2fcb9156665c6a30c339ff263726 100644 GIT binary patch delta 33 ocmZ>CX5(TKV3e|*$fhYGEG5XroS0WB#397S#gdkjn6AM908$?XXaE2J delta 31 mcmZ>EWaDBIV3e|+$fhYID8C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.json index 7d4f5a2de38e8..9307c56feb4af 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_any_value_with_respect_nulls.proto.bin index 124a7ad7efe09dc270854fe534335c46a0fd4402..fc2e75c2015b9bcf2e6bddb0ba923751a6cb4751 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zt_d6n^Hi8-aILaai}T#Sht3;?b*2y*}c delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4Qd6n^Hi8-aILaai}T#ShToTCUs diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json index 5579faf119647..773fd7df0ab30 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct.proto.bin index bac82f670b2985e53a8a93dab1f95fed7db4d92c..9c7bd463ab943f11f79afbb01acbf91b672af427 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json index 851862082ca04..0224877077bc5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.json @@ -22,7 +22,8 @@ "literal": { "double": 0.1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_count_distinct_rsd.proto.bin index fd61420fd1e45137acffa63804ac17babbc9f3b9..79b285eb0a28f23817b9beee4947dfe6cff747f6 100644 GIT binary patch delta 24 gcmX@jc$Sfki%Eb{>exiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.json b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.json index 490a2dcd86967..925f95e618930 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.json @@ -26,7 +26,8 @@ "literal": { "integer": 20 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_approx_percentile.proto.bin index ae73716fa4319c2bf20e9555c7eecfb42280f05d..478af7d5ff53a39d590e96372157ffefefdcc0c1 100644 GIT binary patch delta 24 gcmX@hc%G4si%Eb{>cm90>CA>w`V$xEGHEaX08Q8h*8l(j delta 21 dcmX@lc$Sfki%Eb{>exiK>CF04x)T@Z0suT!1*ZT2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array.json index 20fe495bb9bf4..99152d4e998d1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array.proto.bin index 2b679eb4c6db1a09ed86b85afb2f189f05b1521b..25fd26ed8ec81e7ed372b5b0d707f6214ae4e95c 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.json index a3197ce95068a..0e4e0fe4dd504 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_agg.proto.bin index c7306df86214e1b41a5df3b32f6a3e98172fafd5..4b30c105a10786ce77777b69798c2f83a087563b 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zt_MMa5~@rmi_Laai}T#Sht3;?Yy2wVUF delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4QMMa5~@rmi_Laai}T#ShTo5=_) diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.json index cabd44c063dec..b6af59d5a1cc1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.json @@ -22,7 +22,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_append.proto.bin index 76f2f0255bf2548627d704f33f4805bf7d44fd39..3e2ac2115ff67b15c95ca2d66b38c87ec0a66ae8 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.json index c3ebf313190c2..93b449217eb51 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_compact.proto.bin index 949d66cb951f0a1ec480ff6df1251ecdc1e41bbc..5c244efd0258f5dcc4e3d5a5867b6113dbe5c8aa 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.json index a362d66d9d64d..349927b7cfd9e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.json @@ -22,7 +22,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_contains.proto.bin index d8764f60364c282d93e0f80f2a37d99589136e3a..af333721d6944a11be837a16bd8cc80fb3815de9 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json index d38f4194bcd2b..00b65fc1665d5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_distinct.proto.bin index e6359c074bf232dede314d4eaa2a395aeb3062a3..98dfa75ac01b3289c8865e21b640c3c32ad1a749 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.json index 17d50c87161d6..81ed93a29524d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.json @@ -33,9 +33,11 @@ "literal": { "integer": 4 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_except.proto.bin index 692511b2f74a6629fbefd6fe6d7e1eeb2efa20ac..1037a1ffe38db9273e37495bfc96f8f30bce2f7b 100644 GIT binary patch delta 54 zcmcb_c#V;bi%Eb{>ik5u>C6^VW)l~PDacAmbFn5C6(v>*v2ZaNFoH=YFv+69puqqD DO(F@T delta 50 zcmcb{c!`mXi%Eb{>hwgm>C7flMiUo^$w*5{a*v2ZaNFoH=YFv$V{Bw7hT diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.json index f4540edbf4108..5bd114b61ad40 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.json @@ -26,7 +26,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_insert.proto.bin index 6e2178ad124e90f28be230a0bfce7bf4c6d038ab..a44ca96ace24a23163426003e60f999f5430a6f4 100644 GIT binary patch delta 24 gcmX@ec$krmi%Eb{YR^Qr>CCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json index 1b95a6724f86d..daa94e5aed678 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.json @@ -29,9 +29,11 @@ "literal": { "integer": 4 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_intersect.proto.bin index 67fb497cf270c4755f4dd4e0d1e3f21353a53c15..dca31097549b8da14136b363176901f84bcb6fd1 100644 GIT binary patch delta 48 zcmX@lc$txni%Eb{>dZv8>CC25#uFDvN{LH}a*v2ZaNaDhn{4F(Mc01;~m AI{*Lx delta 44 wcmcc2c%G4si%Eb{>cm90>CA>w`V$vOiit`IbFn5C6(v>*v2ZaNaDhn{0P3&^*8l(j diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.json index 94e8c176cefbf..cbecc842d0b2c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.json @@ -22,7 +22,8 @@ "literal": { "string": ";" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_join.proto.bin index fbab1b208605d69b34c23b861dc9ba36706215d3..0b2959acac4486e22a936eb940e7e43c6bd0ffef 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC75ZY7-ZzFljIV080o3p#T5? delta 21 dcmX@cc!ZISi%Eb{YVSn0>C9?UDiar|0025C1#tiX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.json index ba67984758a5a..2bf3706f92b88 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_max.proto.bin index f7a98c08cd1753188da38b3a4d4d52ca74ad22b8..b8009a1fb6193cb2ea4bc9159f233d013475239a 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zt_MMa5~@wtf=Laai}T#Tt23;?b%2zLMg delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4QMMa5~@wtf=Laai}T#TszoREdEiB3zt_MMa5~@wu6KLaai}T#Tt23;?br2z3Ae delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4QMMa5~@wu6KLaai}T#TszoR0`a diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.json index 4c212cb028273..20c7794a7de6b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.json @@ -22,7 +22,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_position.proto.bin index 4ef2b11273f251be4cf9ea3caf37a176862e67e0..ee2811ae0de0f91eb78c49d1bb3f131c02201e4d 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.json index ededeb015a227..ff6bd2b0e33fc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.json @@ -22,7 +22,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_prepend.proto.bin index 837710597e7b67f3d4a5aacae3f73ef99f42b002..8ad00dfca7a040e84e75ffe9cb73e6a0c9fac1cc 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.json index 8c562247714a4..f769471cd9791 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.json @@ -22,7 +22,8 @@ "literal": { "integer": 314 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_remove.proto.bin index 95e2872ad77bd8e9a1ff26f78d891029ce8e33be..fd44cfb3372addf1964b64905deeca11194eecd7 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json index c9d9f1f9ca79d..0d218470c1ec1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.json @@ -22,7 +22,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_repeat.proto.bin index e370db16e977c6dff69d6ee6a30e7ad3958e099e..6302ae6ee3f0c4af97e83c93fda0c186682c698a 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.json index c1c618bc7f11f..ac279580a09e1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_size.proto.bin index 47949dfbbda29d8d4814e63dafe85245deef75d7..97554f7ecc930fc1a7583b544362a5a25a1d9a00 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh{wMMa5~@x_@{sY0wm%v_AA8VmrtUI_C5 delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzU5MMa5~@x_@{sY0wm%v_AA0I0+W!vFvP diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.json index 406dc54c8cd2f..74a038895b36a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort.proto.bin index 2074caae1638497588955b43bb3a16c9d91e2065..6ac4fd09dc0ca6ecf99e18da5160316729df916d 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh{wMMa5~@x}Q?B|@x1%v_AA8Vmrt0+W%!dzU5MMa5~@x}Q?B|@x1%v_AA0I4tt&;S4c diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json index f8178ddd64aaf..7a36c03476279 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.json @@ -31,7 +31,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -40,7 +41,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_sort_with_comparator.proto.bin index c506889388c97be8dd20051255ee2450a5ff227b..77719f7334985be0f01e5ebafa3a1c7b1aa9e486 100644 GIT binary patch delta 68 zcmcb|_=u5>i%Eb{>h?so>C7%tP7@c1=xGOOa!E>wb1~`)u@|v&F;~PJLKv0tMj8x4 KP!S*n<^uo$Vhu+C delta 64 zcmaFFc#n~di%Eb{>iR^s>C6sNb`uwfXlVwibBRldaxv-(u@|v&F;~PJLKv0tMnX_N HkOJ}m%GV6v diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.json b/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.json index 7d54079cdb47e..841888bcb1497 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.json @@ -33,9 +33,11 @@ "literal": { "integer": 3 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_array_union.proto.bin index fc3d9d7cd0fd1f09f4afbf76e6d5de9af1f17a98..ee743e69702ae45f9a89367ab35984c12a82e18c 100644 GIT binary patch delta 54 zcmcb}c$JZji%Eb{>fA)O>CEO*rV|&4D#%JnbFn5C6(v>*v2ZaNFoH=YFv+aJpuqqD DOc@EC delta 50 zcmcc0c#)Bfi%Eb{>eNKG>CDDbh7%Wv%1BE|a*v2ZaNFoH=YFv$!6BW(#e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json index ce1d288e00d78..73b49b729edfa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.json @@ -29,9 +29,11 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_overlap.proto.bin index 216f306507d40b1502dd2e4f40b2723d39ad2d4c..b55567912384245a5007b2fdd5ec12e21f1b4490 100644 GIT binary patch delta 48 zcmX@dc!`mXi%Eb{>hwgm>C7flMiUoENQp~{a*v2ZaNFoH=Y4F(Mc01zGt ADgXcg delta 44 wcmcb_c#e^ci%Eb{>i9&q>C6UFdJ`8&h>1!GbFn5C6(v>*v2ZaNFoH=Y0O_X)#sB~S diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json index f24ee44835eb4..ce0c0ce75ab9c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.json @@ -29,9 +29,11 @@ "literal": { "integer": 20 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_arrays_zip.proto.bin index 67c867e6d450cf9413f38573f7666155c851bc68..c1baf7e190a151e8bad3d10f7389ae1da30fcc82 100644 GIT binary patch delta 51 zcmX@hc#)Bfi%Eb{>eNKG>CDDbh7%Wv$Vy2`aB&o;7M7;wC8r9pa4{J$f=Lk#1`P%P DKW7Sg delta 47 zcmcb}c$Sfki%Eb{>exiK>CF04x)T?ONJ>bFad8x<7M7;wC8r9pa4{J$f=Lko6Nd^F diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.json index 3c4dcb70fead3..4c98e62f9dbb9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ascii.proto.bin index 5989bd3b5c606fad172c1b4f95f2c5b9f265351c..cd46fe994aeddb801967e7080e0902a1d6c3c350 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+##mSkOLaai}T#V@&3;=(^2BH7} delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%F#mSkOLaai}T#V@eaqxSo-Xi%Eb{YQ;ph=|X~1{9G)F#hH0RtU}COj7b^{0CoTdH~;_u delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(1a#hH0RtU}COj7b1#3>B2%%0$i+##hG~-Laai}T#QK?3;=*A2B!c3 delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%F#hG~-Laai}T#QKoa!v+! diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json b/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json index 5520b70a0250b..83cd89e5b9afa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.json @@ -25,13 +25,15 @@ "literal": { "integer": 0 } - }] + }], + "isInternal": false } }, { "literal": { "string": "id negative!" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_assert_true_with_message.proto.bin index 6992604efe1b3d410d3a5ed096a026b015adb9e4..923478e910580c74a2206b4f904ea93b5a56f58d 100644 GIT binary patch delta 59 zcmcb@c!QCRi%Eb{>f%JU>1tL|7F^tk#l@*bCGjOirKv)qQo>w}c0z1IEL=>PDIn5- ML1W@UVI~a*0J)0|5C8xG delta 70 zcmcb?c!iOTi%Eb{>g+_e>E>oqCS2T!#l@*bCGjOirKv)~Qi5EJc0z1IEL=>PDIn5- XK}dj$FN-HLMIkRWJ+UOSEL9NzRSXbo diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atan.json b/sql/connect/common/src/test/resources/query-tests/queries/function_atan.json index 3ae4e7ef188ec..2a873025e6254 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_atan.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_atan.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_atan.proto.bin index b932086941f458acce7a46b85b5be7a00e99bba6..c458d693ca1273ce4e46f7b3d9a0f288844916ff 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)FC5d@LtU}COj7b^{0CltmF#rGn delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(1aC5d@LtU}COj7b1!&ISPh diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.json b/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.json index 7d08116c40ae6..53a03d1324f25 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_atan2.proto.bin index 372ae8358494e402d6af99a7322c7c27ca31fe42..25a25871185e701b0e53db6a4fc60ba5251a51fe 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.json b/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.json index 8daec8813917e..f78b9f6421e89 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_atanh.proto.bin index 0aa2f3527ae9c38faa31463850e5a0791a44ff5b..eebe635bc33e051d998d99bbfde1fb133383ad4a 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+#C5d?%Laai}T#QK?3;=)C2A}`{ delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%FC5d?%Laai}T#QKoatQ`- diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_avg.json b/sql/connect/common/src/test/resources/query-tests/queries/function_avg.json index b433f1ea89c29..a3d8868671520 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_avg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_avg.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_avg.proto.bin index 9d9bd296dbdda5bb75bdfb49978878d3412589da..eb1078cd83ba7401becde624165419748bc13582 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b?lW$8k!Ld;x@i5d(5Y$652 delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY39)W$8k!Ld;x@i2z_A1(N^( diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_base64.json b/sql/connect/common/src/test/resources/query-tests/queries/function_base64.json index 97739dca283ef..8ba9e38dd538e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_base64.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_base64.json @@ -26,7 +26,8 @@ } } } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_base64.proto.bin index fc854d974752bc5418a3618ac2a977afb2f22654..0ab0e038829c3cb15fa76c4a41656c56e5431adf 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bin.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bin.json index 304e56504bad9..56a5ce889d93d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bin.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bin.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bin.proto.bin index e8d55fb8d6149ece0fa4b64e63fa71fe17489038..64790b75bc090d77f6d4e3f5d397b60b2c33caef 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b>)nR!C2Ld;x@Ng50QYv={P delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY394nR!C2Ld;x@NdRB&1&;s# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.json index 83b2bcf599f7c..3d156cec87a17 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_and.proto.bin index ad81bec6f0865200e96892111af0ab40d2b02406..af643d406b41e528ef7f3b139c141a99c0c4803e 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=gnI-Xwc_~7yLd;x@i5d(5l>B2%%0$l7#nI-Xwc_~7yLd;x@i2#Le2SNY< diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.json index d5c6b698f7f16..277244f769144 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_count.proto.bin index 875e17d974e2187a222080c319fb6ca0575ed51a..1eb39a6fb7346b3b4b7450bfd80dc09014af6b6c 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3ztFnI-Yb`K5U!Laai}T#Sht3;?bJ2zCGf delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_3lnI-Yb`K5U!Laai}T#ShToNEY1 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.json index 39425c5e3ffbc..9e2b2d901308f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.json @@ -22,7 +22,8 @@ "literal": { "integer": 0 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_get.proto.bin index cd0f4098374c0c3cb287b803007e6942dd4b8377..e3769a795a2f4bc5c6c7a04fecf09a615f88e957 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.json index df21871cb535d..1f09a52756b80 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_length.proto.bin index 860c2eaec0e854e4875191c6d6807955a1f8fd31..13617e95afd7329bf6bda75365041853a73e7c63 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh`_nI-W#sd?!o8A7Z=%v_A=8Vmrr>j>Nc delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzTQnI-W#sd?!o8A7Z=%v_A=0H=QltN;K2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.json index c8e1b2acfe4e0..be21d9557e076 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_or.proto.bin index a52907474fb9654ab0ad062589d484e3a89b3b34..9e4ef6d43a175f4ee337d097d97de77ae5e943b3 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RA#nI-Z0MMA7X%v_9#8Vmr6(gzs; delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J5FnI-Z0MMA7X%v_9#0DKGv=>Px# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.json index 463e6fc5322f2..250d2d97c5b0b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bit_xor.proto.bin index c4a9a5e654f2bd9cc02bdb01e3316691ad8db169..33f0570e72f1064e50e78d5e8bf498b0f1a9e0b0 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=gnI-WR`9(skLd;x@i5d(5mYD~b delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l7#nI-WR`9(skLd;x@i2#Pp2WS8Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.json index 8956c3f303fea..1e8a60bae7e3d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bit_position.proto.bin index 8dcf7b1718d4b93f774243e97654bcc161667501..849fa465d39db8f8abbe18bd45a1e7c02c119093 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.json index 8956c3f303fea..1e8a60bae7e3d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_bucket_number.proto.bin index 8dcf7b1718d4b93f774243e97654bcc161667501..849fa465d39db8f8abbe18bd45a1e7c02c119093 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.json index 910702e90e0ed..aac1f57b5d9b6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_construct_agg.proto.bin index deae4a3aea0725e626343a550459191b15c68337..8c57d776b1d6303e5c765157718380d834934701 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.json index b2e9f11efbcbd..95095c7c28922 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "bytes" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_count.proto.bin index a568b2dae42086fa40ad5478cacf2355247931ac..216c51d3c4d2985ace2584c38c24b14caea81754 100644 GIT binary patch delta 43 ycmebFW#eKJV3cy7$fhYLD<#dvlayJKn^+K^oL`z(BE%`g&c&KkS&~|;!2kfm#tHfW delta 41 wcmebEX5(TKV3cy5$fhYHEhWjtlayJKn^+K^oL`z(BE%`g&c&KkS&~`|0Iuu_$^ZZW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.json index 23e37246647b9..041ed9cb6330f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "bytes" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bitmap_or_agg.proto.bin index d27edc376628820a515e2781938c9d59ed521018..41309fc648cda5076d6f97d8040560c197da9ccd 100644 GIT binary patch delta 44 zcmebAW8-2HV3hKh$fhYTCndwho0M6Sn^+K^UlgC1o-V{G#LmTP&(jJ= delta 42 xcmebCVdG*FV3cy3$fhYPBPGSfo0M6Sn^+K^UlgC1o-V{G#LmTHK_BJY0-*Laai}T#Sht3;AXBrTwIKGLaai}T#ShTP=Ex* diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.json index 9b58c898242d9..3e013976a418f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "flag" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_and.proto.bin index 6ea3860027c861b15dec624a21da4bf9b4b6a284..e6e52551364e1915bc3adcd69fe73e9cfa1634ce 100644 GIT binary patch delta 38 tcmZ?sVB=yEV3cy2$fhYNAtlDek(8gG6Q7uuBE%uY#>JABlbEi-004wr2Y~

JABlb8+wbejiJ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.json index 763b019b05f1f..7cf57c57812fd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "flag" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bool_or.proto.bin index dd928c8dc8a314cfefde2ef4fd65bbdce23635b9..207083a8a12a09143fd60187b288fa6c6efd5558 100644 GIT binary patch delta 37 scmZ>DXX9cLV3cy4$fhYFE+xvvo|K=T6Q5rs#397S#gdkjn6AM90D5BwC;$Ke delta 35 qcmZ>FW#eKJV3e|-$fhYKDkaRto|K=T6Q5rs#397S#gdkjm<|AIT?Y67 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bround.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bround.json index 585a0befb224d..d42711c424c46 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bround.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bround.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bround.proto.bin index 8625ccb1a58f147413a74351703ba80943b54f20..40e173d9df4a030715e5cc92cfdc555359de2301 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.json b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.json index 3f35d627f9a54..8c3ec028f0a54 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim.proto.bin index 200dac07a0bb780f619179e51ec36861436e98f6..e2cbbefb551a4b18aa25d6d6db771847a292fcbf 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*~B}JLJLaai}T#V@&3;=++2DbnJ delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$aB}JLJLaai}T#V@ea=!+L diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.json b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.json index cf0476340ccb3..9f8b77f176f72 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_btrim_with_specified_trim_string.proto.bin index d7669c93b2b8918512cf1cc4a1bf88b882174771..25e48c663b97e37c09e30961de3797add06e2716 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.json b/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.json index 971660144a5bc..002222d2f7765 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": true } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_bucket.proto.bin index 1b389401f15e61958df6e874ed636ca918b98ecf..874d68be22a5304cf7d3e4c54c74b836adeea719 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaZ07h8_NB{r; delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.json b/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.json index 0d78dd471f20c..a4c83ac456721 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.json @@ -19,7 +19,8 @@ "unparsedIdentifier": "g" } }], - "isUserDefinedFunction": true + "isUserDefinedFunction": true, + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_call_function.proto.bin index aee05767813f9949e4e07faba4c507c96c8a4dbe..bde6f21637beaf99fa5c49c6b67c431f87a2ed85 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR_pl`Q@oaLaai}T#V@oj2a98kU0kL delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$i*)`Q@oaLaai}T#V@oi~xbZ2EPCR diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.json b/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.json index e2b3dd0428793..82b5dadd84073 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "f" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_cardinality.proto.bin index 54c8cfe843433f09d0a7d4c9af0c34f209d2b848..232a89d09c6e8d4980fb794f91a2f5ef32d9c7b2 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.json index 5a9961ab47f55..d6f4769366fe7 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil.proto.bin index 3761deb1663a2fba0fd8e7e69eafd1a4d0f83415..2b18da5724b81f5a51033f1decc07f19a4126a0c 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)_shK%KtU}COj7b^{0CiplDgXcg delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2FshK%KtU}COj7b1!h6VZn diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json index bda5e85924c30..7413bb77656f5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ceil_scale.proto.bin index 8db402ac167e0b1a1908e94a935376d3124a3876..a658e6f879f893549b5e821dcf8e50dd98ed7548 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFgM=shK%KtU}COj7dT)TucT`8VmrRLI$D$ delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dxuLshK%KtU}COj7dT)TucT`0Fg=tasU7T diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.json index 99726305e8524..878003697424f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling.proto.bin index cc91ac246a57c37ca5553326f5c8e57b4af2cba5..57b7616c179309ea43b8c5c5ae305d3b8d07612e 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{?0shK&MdFeu|Ld;x@Ng50Ql~M>B2%%0$l9LshK&MdFeu|Ld;x@NdSes2TcF~ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.json index c0b0742b12157..c1fdfb5dfad50 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ceiling_scale.proto.bin index 30efc42b9d2bcabddfdc0c6b80c08971c759feff..ba54d5d4d1edcf7da4440bfdf5158d30215eef82 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_char.json b/sql/connect/common/src/test/resources/query-tests/queries/function_char.json index 593139a0a584a..70f6fbcd475f6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_char.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_char.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_char.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_char.proto.bin index 21c3dad55657b299d356c9acb0e9e20607f0334e..df7fb55ab4ab0a28ebb70d50c09adc7e0614d4e2 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)_8Hq(gtU}COjENcy0Ci&qDgXcg delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2F8Hq(gtU}COjEMkfiv{`s diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.json index 3e408260d7020..94323cf8e727e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_char_length.proto.bin index 7f290c6ddc623542f08a82a55159a4a1c94b5439..93096e7ff40517931a442c4b5844da30cf732bb3 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.json index ad12dde8a956c..286b1e9d6040f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_character_length.proto.bin index f1762971d4eca5bdcaadd7537b1689a6a0eae123..12932bdb3087d20c410eec95ed7c129cc0bb97a9 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_chr.json b/sql/connect/common/src/test/resources/query-tests/queries/function_chr.json index 28366f87e10d7..a7d551a94c194 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_chr.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_chr.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_chr.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_chr.proto.bin index dc665d294ecb66690d66d3b062c2b37ad9e0c1dc..7b51a92f63aae95d6e172301484a44b59d415c3d 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@Q8AU>@Ld;x@i5d(5Yz_s& delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3Al8AU>@Ld;x@i2z^@1(E;& diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.json b/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.json index 497922b5df75c..523f853552b5e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.json @@ -22,7 +22,8 @@ "literal": { "integer": 3 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_coalesce.proto.bin index ec871018489c29f2af156ee595a7a0f2e68de5aa..6ceb52f4de47e92ec04954deec1f6d89d1e13bd5 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collate.json b/sql/connect/common/src/test/resources/query-tests/queries/function_collate.json index 8bb6c2c4c3726..f1af032978d9e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_collate.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_collate.json @@ -22,7 +22,8 @@ "literal": { "string": "UNICODE" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collate.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_collate.proto.bin index dda4a00a395b62a63f2793d58f8244fc2f20a717..17ce43c3de332efb42f2e3a014ba91127ee6a6d7 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collation.json b/sql/connect/common/src/test/resources/query-tests/queries/function_collation.json index dac04b3b9858f..50ad1268ea9d6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_collation.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_collation.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collation.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_collation.proto.bin index 739994a486026087bddf313dce9db780d051890a..3cb87236be8554caf6e2824b0fd3d3783a7ad0b4 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zuw`8hd>C7JnoLaai}T#V@&3;?b)2zmej delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_55`8hd>C7JnoLaai}T#V@eoRkPk diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.json b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.json index c5bae4baef352..81210507bc861 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_list.proto.bin index e3827b9f650ae9e2eaf93fde15b770df3a033d8f..a66294b2c475d15c33a4eb2ac6e034100d15c8fe 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.json b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.json index 615386d050e14..4ebf94c214391 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_collect_set.proto.bin index 5fb97f27d25b6d10b278c309d06844e449565782..abbd935086a33f967e251fd377ad124b2e58424b 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_concat.json b/sql/connect/common/src/test/resources/query-tests/queries/function_concat.json index bad1ad6f3b90e..cd3702e86b3dc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_concat.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_concat.json @@ -29,7 +29,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -42,9 +43,11 @@ "literal": { "integer": 40 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_concat.proto.bin index 7411f55f1474749f49a7d8331e192cd03729c379..87076f350fe4214ee56ce189cfa284595f1c0f0a 100644 GIT binary patch delta 88 zcmaFB_>7T_i%Eb{>i$Hw>7E`^Zd`21`FY8SB|@x1%v_AALgG@QT&#&jMTwO{EL=gGhY>CR444qR->`FY8SB|@x1%v_AALZVW_T&#&jMTwO{EL=| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json b/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json index b9ba89b42185c..9468c641c0a81 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_concat_ws.proto.bin index 2fbc4f709044891c736c5d72f466c5fa765b2975..34d0bbdeb8794e4294f2d9293eecad172b99b196 100644 GIT binary patch delta 24 gcmX@Zc#e^ci%Eb{>i9&q>C6UFdJ`AuFljIV08NMm(EtDd delta 21 dcmX@dc!rUUi%Eb{>gYta>CAdkIujS@002A}1)%@{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_contains.json b/sql/connect/common/src/test/resources/query-tests/queries/function_contains.json index b7cb12d9aa9a3..3a10154044dab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_contains.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_contains.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_contains.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_contains.proto.bin index 8864968a9dc3bb286141f086d386c7f1ce07490b..80eeae9cd02c29da30ade0dea0f781e36a8cd348 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_conv.json b/sql/connect/common/src/test/resources/query-tests/queries/function_conv.json index c6734936bfcd1..9e65edfed6ee6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_conv.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_conv.json @@ -26,7 +26,8 @@ "literal": { "integer": 16 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_conv.proto.bin index 373b997b79240a1d3b20a7ca25a336d2ed8a063e..3161da1cbca425a6422878dd6aad295f4c76e98c 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.json index b27d7e2b55fae..6d0b24f2c8c72 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_with_source_time_zone.proto.bin index 8ef4e3bdce29e3e3325496e7e8fa7d4b1a34e853..26e096c4c3f20a478005eea972d7c3c1ec16f8f1 100644 GIT binary patch delta 24 fcmZ3*xQ3C9i%Eb{YUxC_L}nK$r-@k(Od1RTL%;<) delta 21 ccmZ3(xQdaDi%Eb{YVkz2L}n)`hlyDZ05lN=3jhEB diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.json index b072c89d42bd1..40f599741a9d2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_convert_timezone_without_source_time_zone.proto.bin index c6d1db9b8fb1dd65e7cbec810dc8a74285245ffc..a7f5317512a32110c294b12e3d4b78e9c274a408 100644 GIT binary patch delta 24 fcmbQnID?Umi%Eb{YT`t;L}mjiy@^>mOd1RTJ%9wm delta 21 ccmbQiIE|5wi%Eb{s(&I|BD0>9&crMo053TNlK=n! diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_corr.json b/sql/connect/common/src/test/resources/query-tests/queries/function_corr.json index 6fadb0385622b..c7c44457d4e0d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_corr.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_corr.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_corr.proto.bin index fdeeb4fd12d190304153cfb2f53d05f5de3f43b5..5aac92504c0bdd298e90e3c035bde1e3d12b0feb 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cos.json b/sql/connect/common/src/test/resources/query-tests/queries/function_cos.json index f7072dff03404..f9ee2077f33a0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_cos.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_cos.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_cos.proto.bin index 09fd198b097c047391553f954ccbe79eee9c6f87..f7592d6ab63c9e2cf3b1cf8326e738951e22290e 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@Q`Ncx4Ld;x@Ng50QY-k0` delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3Al`Ncx4Ld;x@NdRD01)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.json b/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.json index 3bcab61d37a0d..78b0d73b977ab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_cosh.proto.bin index 54d5da8fabfa619e60b522af35b1b6c4bb6e0fea..5619d48d203970d9e3ffa2dd51fd640cc119360a 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)_`NbJRtU}COj7b^{0CpS(IsgCw delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2F`NbJRtU}COj7b1#BnAoq diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cot.json b/sql/connect/common/src/test/resources/query-tests/queries/function_cot.json index 62ce963fa8737..1e521c2e6d7f6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_cot.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_cot.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_cot.proto.bin index e79c32660a772019247472262e931f516e443b57..e8ba807d9506a7a0d4774c7bb298e6d1a3f9e782 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@Q`6WWELd;x@Ng50QY;pz6 delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3Al`6WWELd;x@NdRD81)Kl? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count.json b/sql/connect/common/src/test/resources/query-tests/queries/function_count.json index 126a0ca242c52..613ba5510828f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_count.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_count.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_count.proto.bin index 6c87a809ad0c473caa9376fa5e48adee32d0451f..1e9b69069e05d5daefe368fff25dcf9a73cf8fe0 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i-g`K5U!Laai}T#Sht3;=-@2D|_O delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%_`K5U!Laai}T#ShTa}frN diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json index eb211ceb239aa..6a844d694ae8f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.json @@ -23,7 +23,8 @@ "unparsedIdentifier": "g" } }], - "isDistinct": true + "isDistinct": true, + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_countDistinct.proto.bin index 591e2300ec6891dfc8458cbf5ecd3c5665dbe2fa..b25c393cea04879cb76e5ad9b8be6726d9a762a8 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.json b/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.json index 669477bbc5dd8..f19ce9ced1a5d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.json @@ -25,9 +25,11 @@ "literal": { "integer": 0 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_count_if.proto.bin index 07c65ebaa429378bb7015aaeced15b599f729984..97306ae5253e4bf52e20917a75bdeb63f502bec5 100644 GIT binary patch delta 52 zcmdnTc!-gWi%Eb{YWGC8>8dJHN?aVt`K5U!@tJ8tB2q$JjCMk-Ld;x@i9#$~Oa=@Z GKnMUacL{6& delta 48 zcmX@axQ~&Ii%Eb{YWqaC>52+ca$FqA`K5U!@tJ8tLQ(=;jCMk-Ld;x@i9#$~Oa=@9 D3AYId diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.json b/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.json index 94be79dcc33e5..badc965380235 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.json @@ -30,7 +30,8 @@ "literal": { "integer": 11 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_count_min_sketch.proto.bin index 11bcae8062e82bbeaa18d3cee8a47b5a43806657..d2bc25513fde37e1830e794a4a238313979cfe96 100644 GIT binary patch delta 24 gcmcb~c$<-pi%Eb{>dHj6>CCoL))N<4GHEaX08y9*9{>OV delta 21 dcmcc3c$1Nhi%Eb{>e58E>CDzrmJ=6P0sucf1@8a= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.json b/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.json index 1c5df90b79cd1..db67d7e86bd46 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_count_typed.proto.bin index 44b613eb40c6f326cf744277245c7e1657b56a73..852290d484be3d7a9e035b54321c4ae1883e6d31 100644 GIT binary patch delta 32 ncmZ>AVdG*FV3e|($fhkUBqhMbnw(#nS0cnJ#LUH*sKEdLQBMU6 delta 30 lcmZ>9V&h^GV3e|%$fhkQAjQYUnw(#nS0cnJ#LUH*2mnT_1ls@r diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json index 3c4df70a5fbfc..294d055796e08 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_pop.proto.bin index 4a7202f15e768beb3324a64dcf1b24bc16610e1e..09939663ce3bfe925dbb075bf067668928298627 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json index 7c723069e4671..f366b2f62ae50 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_covar_samp.proto.bin index ebff687730e3522484ca6f61fb60158cfd694e46..c8ef18e85fe6798d674d23510d9d98aa30fe56af 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_csc.json b/sql/connect/common/src/test/resources/query-tests/queries/function_csc.json index 88504ed9c5280..68cc4e0096f4b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_csc.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_csc.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_csc.proto.bin index 0ed5022a73adfd41cba57219dd1a66dca397585b..8bf716cbab8a1e0e1c0167dc809adccb106af49c 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@Q#mPdfLd;x@Ng50QYxD)b delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3Al#mPdfLd;x@NdRB^1&;s# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json b/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json index ac48841199075..a2978d3242ca6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.json @@ -15,7 +15,8 @@ "window": { "windowFunction": { "unresolvedFunction": { - "functionName": "cume_dist" + "functionName": "cume_dist", + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_cume_dist.proto.bin index 7578245aabe3a4a2f12ad0c230ef47ddb457f9fc..1dea3404d5a55cee8e81f8e03ef0cc7529dfdff2 100644 GIT binary patch delta 38 ucmX@Yc#M&ai%Eb{YX3yG=^`3YYFzwMyj+~grMao`DVfD38VnQHivj?%XbJ!T delta 36 scmX@cc!ZISi%Eb{YVSn0=|XBzDqOr$++3W=rMao`DVfD36E}(i0HCr7&Hw-a diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.json b/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.json index c344f5271704d..5b49d10fb5c56 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "curdate" + "functionName": "curdate", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_curdate.proto.bin index 6ec58b57c2a7ab86f4bdcedc6e6681adf2284648..b307dc12a647ed0ee3db85ad93138c5a17e1be08 100644 GIT binary patch delta 27 icmXRaVdG*FV3f+6$d0+W%!d(2xrA0-lc_s15i6x0S`RN)A0I_-sQ~&?~ delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?WK`rA0-lc_s15i6x0S`RM?hE(#<7 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.json b/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.json index dfa59fd5fe50a..1624c30ce4000 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "current_database" + "functionName": "current_database", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_current_database.proto.bin index fdb11c9c8bd06898609d4e048bf7e20dd6604222..541891bc4418b6c81f4dc87f5902c017d32b9678 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3uH=rA0-lc_r~Fi6x0iiN&cJ3;?}K3bg7pW1LR7pW1LR@^wrA0-lc_s11$r-7+i5d(5rUeP( delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$hB_rA0-lc_s11$r-7+i2#wK39|qI diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json index 16af5eb9ba084..bea08c2ee6af0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "current_timestamp" + "functionName": "current_timestamp", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timestamp.proto.bin index 5a1f3de6c3a9aa5b6ca9613ccf407c195407d91d..8691a0468fa82f5e6983f4050cc6437b1624266f 100644 GIT binary patch delta 37 scmb=aW8-2HV3ewu$d(`>E+xt(m|R*^l$uu(Uy_-dT3nKtTcE)J0IjhKegFUf delta 35 qcmb=cVdG*FV3aDG$d(``DkaP%m|R*^l$uu(Uy_-dT3nKtTL1u;{|Zh3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.json index 082d7f5bae6f6..d3807c4dbbe92 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "current_timezone" + "functionName": "current_timezone", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_current_timezone.proto.bin index a780830516bc0876c7b2c541cade271cc825ab10..bf06690e07afcf5731e2874c901f9172409e1578 100644 GIT binary patch delta 36 rcmb=fW#eKJV3aDK$d({3CMCipkX%|+l$uu(Uy_-dT9u!ds=)vNq3#L} delta 34 pcmb=eX5(TKV3aDI$d(`~A|=EnkX%|+l$uu(Uy_-dT9u!d3IL5H3Eltz diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.json b/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.json index 30ddb80f884c2..f7bf8d9f91509 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "current_user" + "functionName": "current_user", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_current_user.proto.bin index 0a79078fd70972be227cb16f0f860d2129bd91b6..df4ea4e2cc4da6585d3cdda2f4ac7cd5b8a59f6b 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$e=FrA0-lc_s0s#i>Oa3;>Xc2_pai delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|W)qrA0-lc_s0s#i>OAeYXhn diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.json b/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.json index f81ad3335242c..1c022cdafbc3b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_date_add.proto.bin index f4dbc16b05c1d15257ec6aeb7909685664f0edf9..d1e296f485cb7cd1caad2bb1b85a7b0d4e1dc136 100644 GIT binary patch delta 43 ycmZo*Y+_{NViI7Ks+q`^AR{d$$;FY9Sdto_n35vID#Xmim?Ff&#bm&w!2kfXaR+Vy delta 41 wcmZo-Y+z*LViI7Ks+!1_ASEd!&c%_ESdto_n35vID#Xmim?Ff&#bm$)0HnbOJ^%m! diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.json b/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.json index b6094ff8734d6..a4fc25f1bf0db 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.json @@ -33,9 +33,11 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_date_diff.proto.bin index 5621af09474d8f60b6aada81e0a6c339b2b2dff8..bd37bb76df8e81cb8f3d0323520f9c805c626901 100644 GIT binary patch delta 48 zcmbQoxQLOBi%Eb{YW767L`f?t3og!-#FEtbl+3g=Ayy$~F2)ogB`Jl84RS0R3>pjo D12PHi delta 43 ycmZ3)IFFHyi%Eb{YWhUBL`gF#6E4n_#FEtbl+3g=Ayy$~F2)og1u40S4RQd`x(SH@ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.json index 9b3d469ed4e98..6ef48b7f015a3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.json @@ -22,7 +22,8 @@ "literal": { "string": "yyyy-MM-dd" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_date_format.proto.bin index 7226c20974b2aad00fef0613e0ce8da65d0f6f53..f28036fdcc2d99bcfe0e34658be6058cbcab2ee3 100644 GIT binary patch delta 24 fcmeBUoWRJ&#U#Ke)j5$Zky%4ZZDN)RlLi9-I{*Ze delta 21 ccmbQh*vH7m#U#Ke)jp9eky%YjWnz{J04+%bV*mgE diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.json b/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.json index ada0747743b78..08769597e4d7c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.json @@ -18,7 +18,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_date_from_unix_date.proto.bin index 5cbd76eda907436e86c96fd007d67f15ef57de76..ba9b9047893d7d550a0c22792c96698369b0354f 100644 GIT binary patch delta 25 gcmZo+Y-eQWViI7KYG9nmmcT48B|9-wnn{BJ06F#qTmS$7 delta 21 ccmZo?Y++>MViI7Ks+-83z$_~zJuyof04oUuEC2ui diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.json b/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.json index 5e8d075c4e2d0..51590b395c19a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_date_part.proto.bin index 368ecb676c1fef4515d097da286f179da8c94b3f..cddd5c1954888fb2cc9fa9a4fba28796b026892e 100644 GIT binary patch delta 25 gcmZo=>|kW)ViI7KYGjY-MEQViI7Ks-MV~ATK8+!^K)sRGOD8#45zh#aJT5#l?}umYbPZT9T^4007Rf B32y)Z delta 44 zcmZo=Y-VKRViI7Ks-4J|AS)v!#l>1uRGOD8#45zh#aJT5#l?}umYbPZT9OI?y5$K! diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.json b/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.json index 2658c724d287a..e8d272bafa922 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_dateadd.proto.bin index e72a77f0e2394390783577a1f865a11d3d7fbe0c..1dcbd5fc64ca2f2fdd44c297c7732bd361b87276 100644 GIT binary patch delta 41 wcmb<#WMt!F5@3|7p2(IUEhQzv#h#K_lA4&3BE%}h%*B`@#KOg7z@)(d0H+fN2LJ#7 delta 39 ucmZoMViI7Ks+-83z$_~zJuyof04oUuEC2ui diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_day.json b/sql/connect/common/src/test/resources/query-tests/queries/function_day.json index c1e4b4d13fb7a..712ad56df109d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_day.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_day.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_day.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_day.proto.bin index e72a4a354c31d957cbc1389185e525ea4af9b811..9115ae09ce8fc164d0bc85a21d22d686b253c628 100644 GIT binary patch delta 30 lcmXRdXX9cLV3aDH$d(`|AjQYUoRU~6#45zh#h9YO002~S1nB?( delta 28 jcmXRfW#eKJV3f+A$d#45zh#h9YO004S@2BrW2 delta 32 ncmb=fWaDBIV3aDJ$d({1BqhMbo|0IZmzbL>#45zh#h3yBY?B6d diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json index 3e453c1f7a652..76b0a6e03f27c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofmonth.proto.bin index 3a2973e21e5a04e2f638043a68fc875650863672..8a63ee68777e8512bf8455d4fc0dc6e1738d7ef4 100644 GIT binary patch delta 37 scmb=aW8-2HV3ewu$d(`>E+xvvm6BMQpO%}SSCS#bD#Xmin4-Y|0G)aW6#xJL delta 35 qcmb=cVdG*FV3aDG$d(``DkaRtm6BMQpO%}SSCS#bD#Xmim;wNdng`_o diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json index 74715de151e77..7cea8fdc7745d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofweek.proto.bin index fceea203c790ed4c2bc0e2bc5db204ebb99b3927..6c9a6252a48edeb92901c0ff5d6136efb66500f3 100644 GIT binary patch delta 36 rcmb=fW#eKJV3aDK$d({3CMCkfnUYwUpH`llnk~dC#LUH*qQL+FkrM}$ delta 34 pcmb=eX5(TKV3aDI$d(`~A|=GdnUYwUpH`llnk~dC#LUH*0sw(M2V?*M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json index d23c6790a47dd..3579113015448 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_dayofyear.proto.bin index a526b449ae0a4ab0cd217927a131ed7893de2ce5..ef1d60c231599caf9cfc33e687b7e0bb020300e5 100644 GIT binary patch delta 36 rcmb=fW#eKJV3aDK$d({3CMCkfnUYwUpH`WgSR}+M#LUH*qQL+Fkw*ua delta 34 pcmb=eX5(TKV3aDI$d(`~A|=GdnUYwUpH`WgSR}+M#LUH*0sw($2WbER diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_days.json b/sql/connect/common/src/test/resources/query-tests/queries/function_days.json index 9e20c48729a30..b9a73878304b6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_days.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_days.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": true } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_days.proto.bin index b0a8472f8c4ff56d5da67e907328382563fc5ba7..b81c13479a36fede7ca2a18f8318c648f5e1e881 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(4iIv4dtU}COjENeI0Cq11J^%m! delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0PiIv4dtU}COjEMkgFa`|( diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_decode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_decode.json index 6be60808e64f3..2aaae4466b5fc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_decode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_decode.json @@ -30,7 +30,8 @@ "literal": { "string": "UTF-8" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_decode.proto.bin index 18b8bbcf6a01df1cd01a71f35c59afd0fb1df297..d71f018e81ee8705a740f9ae1e24ab543c5b1c69 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.json b/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.json index e096b07e4dc6e..f366faf9ac7c5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_degrees.proto.bin index e2d264bb2e108d8446934a365bf9ec36f4f8c1a8..efe6bb3a93cca8cd85acf77066cb06c3ef229365 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=Asp&>B2%%0$l7Vsp&cB*{>7trZ>RbX+d|X^9sd>ez@kNPw*%}NJH;4iNzTgUH delta 55 zcmX@kc$ATii%Eb{YTrb*=^E-$s$6_hJX~BUsd>ez@kNPw*+Q&B%v_9#KtVPk7A~gD J6ahvFMgU#K3fTYv diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_e.json b/sql/connect/common/src/test/resources/query-tests/queries/function_e.json index c99c04a6befdb..12f1ed4146bb1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_e.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_e.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "e" + "functionName": "e", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_e.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_e.proto.bin index 49f6c12fbcc7271f3c56c535489e43f3e699f461..e08ad39fe02f77775d73dce0497e7af82ddb640d 100644 GIT binary patch delta 23 ecmZ3?xRjBNi%Eb{YTiV)=^X4*tXzz#8Vmq9H3S*} delta 21 ccmZ3=xR{ZRi%Eb{YR*Kq>1?c0%v_AA058o0>Hq)$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.json b/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.json index ef5551440934c..562f84a0abfb3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.json @@ -22,7 +22,8 @@ "literal": { "string": "bob" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_element_at.proto.bin index 993818c6cb4bfc0b774b8d393b029cb845479689..6cbf58ebaaf63d87d07bc3888e20b7f1891c10d7 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_elt.json b/sql/connect/common/src/test/resources/query-tests/queries/function_elt.json index fe7dd29f91a33..0573d25e70879 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_elt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_elt.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_elt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_elt.proto.bin index d719db6f89c37d6e1612e2505751fa36c9574fbd..429c39ca81b26d635c2836f62a517ad54499358e 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFCAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.json b/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.json index bc53edfa25f5e..50963dd1c9d0a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_equal_null.proto.bin index 2855f3ebbf3c38c73c42894864d19fcc1215381b..ad21a1e96b9de0276eb558237459e1d48a79f52c 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFEW#eKJV3e|-$fhYKDkaRtnp&1xR4K$E#Ky&vmXnyS!2keeN(NZ~ delta 33 ocmZ>DX5(TKV3e|*$fhYGEG5Xrnp&1xR4K$E#Ky&vmXnwc09f({DF6Tf diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_exists.json b/sql/connect/common/src/test/resources/query-tests/queries/function_exists.json index 3ae49d13c5fc6..caa2c6170f251 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_exists.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_exists.json @@ -31,14 +31,16 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, "arguments": [{ "nameParts": ["x_1"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_exists.proto.bin index d808227fdc6598b69abceaded1de07f88b35d544..ac30b86a396da48fadb8bcc75bba80927e8543ef 100644 GIT binary patch delta 70 zcmcb>c!iOTi%Eb{>g+_e>E>oqCR}W(6`92)#X_t?%v_AALW)81Tw+opT#R-?>_x0x V%oXv5LM&WN23#5pLJ+P70{{$Q44wc0 delta 66 zcmcb@c!80Pi%Eb{>f}VW>BdG<23%~Z6`92)#X_t?%v_AALh?bfTq06JT#R-?>_x0x R%oXv5LM&WN23$fA4gl9l3^o7& diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_exp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_exp.json index d317efef75eee..a814a7f3a4937 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_exp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_exp.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_exp.proto.bin index 7def20c94df00b42fc350194d2122106ed1f52f3..0803af7c262c1e35c0b3a9b09d137bf897d4b538 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b?_6$L`9Ld;x@Ng50QY{doB delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AF6$L`9Ld;x@NdRD@1)~4} diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_explode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_explode.json index 35ad40ccdd04f..8c0094b3a39d5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_explode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_explode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_explode.proto.bin index 9c15f942bb11d409b8c87adbcca841b8b8c59723..fb5d06b5447096aa80a40551b06c5f6867ead65c 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{>r6$Lr@DXBuNLd;x@sTvFbmcR#= delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l8=6$Lr@DXBuNLd;x@sQ`xf2W9{O diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json b/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json index efd7f4b524d47..12c753a38be70 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_explode_outer.proto.bin index 9f2cf9554dd15048b2717cb4d7f697aec50a0d13..8e9ecc3cfacb029ed9538e012b95e8974c4078be 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.json b/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.json index d425a6de709b7..70cd7b9284654 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_expm1.proto.bin index 3c310cb04ce3d84b7d69cbb19ce7b19a4837694e..daa03048ab5b54e3ff170eb1045a2a2774e7058e 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i-A6$QD5Laai}T#QK?3;=$_27dqm delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%l6$QD5Laai}T#QKoaXJQ1 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_extract.json b/sql/connect/common/src/test/resources/query-tests/queries/function_extract.json index 6ccdb2f9d7531..851ae5de2741a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_extract.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_extract.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_extract.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_extract.proto.bin index 91553c3b94bcf70ccc9f74ec192884f3b92cf0fd..1d47c480d7708742e96267a583c79f4d5e79218f 100644 GIT binary patch delta 46 zcmZo>Y-MEQViI7Ks-MV~ATK8+!^NIjQBstcTq4B5#g@fVnVMK6#45zh#h9YO007?` B2}u9| delta 44 zcmZo=Y-VKRViI7Ks-4J|AS)v!#l@alQBstcTq4B5#g@fVnVMK6#45zh#h3yB%bf`s diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.json b/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.json index 7f13a10480915..e762531e0be86 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.json @@ -25,9 +25,11 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_factorial.proto.bin index ac776eb60d2b0e933be5c640608ab94c77e0d831..4863eee97ac1c63f89c4d901ce48d466debe0fa1 100644 GIT binary patch delta 53 zcmdnbc$krmi%Eb{YR^Qr>1wJ{%3PdjiOD7TMVW~?LLyQ^T#TwhtU}COjEO=lTucUB I8Vni?06|&_!Tc!iOTi%Eb{>g+_e>E>oqCR}W3nK>n?MMA7X%v_AALW)81Tw+opT#R-?>_x0x V%oXv5LM&WN23#5pLJ+P70{{n;41@px delta 66 zcmcb@c!80Pi%Eb{>f}VW>BdG<23%}unK>n?MMA7X%v_AALh?bfTq06JT#R-?>_x0x R%oXv5LM&WN23$fA4gk{R3>*Le diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json b/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json index f2b85c21af755..94f04a9111a88 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.json @@ -34,7 +34,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -47,9 +48,11 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -58,7 +61,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_filter_with_pair_input.proto.bin index 8cf5f2d65cf29db6bcc9cba5c0a968232409ab7c..b927f1c623a44df870ffbd07d8684b52b30dcf0d 100644 GIT binary patch delta 112 zcmeyy_>Yl|i%Eb{>ia~t>4^zaaa?R^nK>n?MMA7X%v_AALf%21T$WPiT+E4iDMDgW tB3z7iLhMDXT+9{mhC(b{Oa@#U3@GB2@kU^ACJhFl3{)YI0*Y%e001Y16T|=j delta 104 zcmeyz_>GZ`i%Eb{>itBv>5&mqVO(r!nK>n?MMA7X%v_AALass1T*gv{T+E4iDMBJr mLR^e?LhMDXT+9{mhC(b{Oa@#iqLuMRU{NL^s1hIr6axT(r4r-- diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.json b/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.json index 538651b52c424..424357c82f13d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_find_in_set.proto.bin index 26abfa0e394c3b3a3c536461976b761b92dc191b..662aec53059a717a2aa784449569f154fbfde68f 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.json index af55fe44ae8ca..bab44ade0cfaa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_first_value_with_ignore_nulls.proto.bin index 7121c820aa73710a11cdc8f53bea2c3f5b860f3f..6c72610c94265943de2df79a8ac93d29e8cdb6e2 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.json index dc33bad3c506a..ec4c9da731d65 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_ignore_nulls.proto.bin index cb029dfd26be90c0065ba7403fc1cfd06045843a..a995bb05e6f26dfa3f1d0c1e03c48e6930443a52 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.json index 0e315cc6b1bce..244f2d1f9575e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_first_with_respect_nulls.proto.bin index bf1d48903dfab4452317cff114e7346e7deac092..eb52e483099445726edf53db27b500a76cc8c47a 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.json b/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.json index 1f04630fd5f31..3118b0620c0e3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.json @@ -32,11 +32,14 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_flatten.proto.bin index 9a684850f9cfa204cffb9e779abfc921005be654..1d930ab05303cef248f5135f25f903a89dd69240 100644 GIT binary patch delta 59 zcmcc0c$<-pi%Eb{>dHj6=}NXz)?DmqIf*4Dsd++rQaW6$iA6<;l|rmS%v_AALQ+x^ M6Zea;X)u5w0K0Y#Hvj+t delta 52 zcmcc3c$JZji%Eb{>fA)O=}P8Ord;f4If*4Dsd++LQW{*WiA6<;l|rmS%v_AALK0G9 I6ZeY&09^45n*aa+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_floor.json b/sql/connect/common/src/test/resources/query-tests/queries/function_floor.json index 78924f5f33627..9bf149ea295d9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_floor.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_floor.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_floor.proto.bin index b52696ca4d00a39df7878fc3233965ac10363329..1f529ad20219f7a0f1bb1000fd47fa4758174670 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+VIr;fTLaai}T#QK?3;=-22DShI delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$)Ir;fTLaai}T#QKoa?%Ed diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json b/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json index 394621e4dd314..af37c822a21df 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_floor_scale.proto.bin index ee0665bab644c56e9b47bbbfebc6927314cbdd63..f2510d8540374f5153071dc78114d2e9f5f570ef 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_forall.json b/sql/connect/common/src/test/resources/query-tests/queries/function_forall.json index 4a4914d6a9b1d..a9084fe59bdb6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_forall.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_forall.json @@ -31,14 +31,16 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }, "arguments": [{ "nameParts": ["x_1"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_forall.proto.bin index 7fc28216945893e4cb376d26e3669492ae8877d3..b72863b1db28b751412ec54ede09666844f430ff 100644 GIT binary patch delta 70 zcmcb>c!iOTi%Eb{>g+_e>E>oqCR}W3`9+C2IYO*L%v_AALW)81Tw+opT#R-?>_x0x V%oXv5LM&WN23#5pLJ+P70{{k)41NFr delta 66 zcmcb@c!80Pi%Eb{>f}VW>BdG<23%}u`9+C2IYO*L%v_AALh?bfTq06JT#R-?>_x0x R%oXv5LM&WN23$fA4gk^l3>E+Y diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.json b/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.json index daa648c0a599e..6f376efc9fa81 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.json @@ -22,7 +22,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_format_number.proto.bin index 81e2c4d5fd54d67b817acf9d8a0e06ea4fef18d9..2f156c433670311298b5af27bdbd73b5dbc93689 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.json b/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.json index 798e79e6618f5..60242d45866aa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.json @@ -33,9 +33,11 @@ "literal": { "string": "FAILFAST" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_from_csv.proto.bin index 8acd3b619b41e8d250aa4b283ccb9b67e93bcc3d..2a3a14f8914a1acb66abd0aef146007033f5e2c1 100644 GIT binary patch delta 58 zcmeys_=S;;i%Eb{>g`0f>C9nLArluEs3=Owb1~;876@^0v1PI3=BK0z@o;fvakx2p O`nWj;hiEWpFaQ9!lntu@ delta 54 zcmeyu_<@m)i%Eb{>g7bX>C8b=0TUM(D9B66axv#776@^0v1PI3=BK0z@o;fvakx2p K`nWj;hX4S7U=2qA diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.json b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.json index ddfa91abca05e..850ae645c0497 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.json @@ -22,7 +22,8 @@ "literal": { "string": "{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json.proto.bin index ad95d0f2b343d1dc58a66f45ae9856689ae62881..5a1990c95729034117af1aff1a42bea4cc15e0b4 100644 GIT binary patch delta 25 hcmeBT?qO!*ViI7KYMsb7o#ic~)T@aLUovSh002=t2S@+_ delta 22 ecmeBS?qX)+ViI7KYM#h8o#i#7)QgD=UjhI-VFwrh diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.json b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.json index ddfa91abca05e..850ae645c0497 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.json @@ -22,7 +22,8 @@ "literal": { "string": "{\"type\":\"struct\",\"fields\":[{\"name\":\"id\",\"type\":\"long\",\"nullable\":true,\"metadata\":{}},{\"name\":\"a\",\"type\":\"integer\",\"nullable\":true,\"metadata\":{}},{\"name\":\"b\",\"type\":\"double\",\"nullable\":true,\"metadata\":{}}]}" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_from_json_with_json_schema.proto.bin index ad95d0f2b343d1dc58a66f45ae9856689ae62881..5a1990c95729034117af1aff1a42bea4cc15e0b4 100644 GIT binary patch delta 25 hcmeBT?qO!*ViI7KYMsb7o#ic~)T@aLUovSh002=t2S@+_ delta 22 ecmeBS?qX)+ViI7KYM#h8o#i#7)QgD=UjhI-VFwrh diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json b/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json index 81d6608adb18f..bbe54e42d2fe8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.json @@ -18,7 +18,8 @@ "literal": { "long": "1" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_from_unixtime.proto.bin index b1c34caaf62f0435d7fa7fea4086938acba55845..5b92281e1f289a24b69bada099280c5b07d6112a 100644 GIT binary patch delta 40 vcmbEdEiB3#VrsUh220;J- delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR`%0sUexiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.json b/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.json index ef33382022a53..8bb777bfc37b2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.json @@ -22,7 +22,8 @@ "literal": { "integer": 0 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_getbit.proto.bin index 15575e4f7cbb906c617b165fb83d54fac994d35c..b7684864503db80f49945acd48f54c3c61dc1b06 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.json b/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.json index bf5d50edec84f..92414ef27da5b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.json @@ -31,7 +31,8 @@ } } } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_greatest.proto.bin index 44d9d5f8cfb2d37651fd69971b3738a4cf245f94..a1ba220f294299f74f8c6bfe2c18e896a13b66c1 100644 GIT binary patch delta 24 gcmX@ic$ATii%Eb{YTrb*>CEaCCE9$`cnT0{}Vc1#18R diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hash.json b/sql/connect/common/src/test/resources/query-tests/queries/function_hash.json index 6ef504a006457..04363f49f72b8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_hash.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_hash.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_hash.proto.bin index 284700c4c5ea9b1bbee3350b7e51064d8a6a8b58..dea61a10eeb5251493c0232420fc91853ca9a889 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hex.json b/sql/connect/common/src/test/resources/query-tests/queries/function_hex.json index af9d0dd298277..7cfc7a5647adf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_hex.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_hex.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_hex.proto.bin index 9d8c3b5e235841425f66d5732d93cf4873205359..52d2da3ea1e8b9817143fb3f41b138f2e54e5ef5 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b>KsTD%3Ld;x@i5d(5Y-I(? delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY38fsTD%3Ld;x@i2z_#1)2Z= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.json b/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.json index 548b4977ddc50..2758e1a0638f6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.json @@ -22,7 +22,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_histogram_numeric.proto.bin index 81dbcd476ecbcd65a74113112d9316d5cef53eb0..465557aa7f48bf810e6d0862ade9a8138fefa52a 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hour.json b/sql/connect/common/src/test/resources/query-tests/queries/function_hour.json index 2621b9f81913c..e13d18f244903 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_hour.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_hour.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_hour.proto.bin index 6cdb50364c13308673f05afeae6fb2254b99f445..f57e2333ba8559cbefd50fab7f932756870ba540 100644 GIT binary patch delta 31 mcmXS`VB=yEV3aDF$d(`^D8>B2%%0$i*a`K3k0Laai}T#Shti~xat2E_mX delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a#<`K3k0Laai}T#ShTb7=;Y diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.json b/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.json index 2d0d6be0164bc..42fe4ac079293 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_hypot.proto.bin index 3ad07a2a1ee45f61d24ea0298c995263f497a4d7..8dce8d0b02a00febfe523f4c29989058a2f1d095 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.json index 2a426fe6fff3a..577d45ddfe6e8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ifnull.proto.bin index c1307c2be8caa1a9cc665eceef49f36c4663fe28..9840001010b019c6b46fab441ba8143996ef1a6f 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.json index 46b1b87e03246..aa97c83dba131 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike.proto.bin index b1c50e3aaf4b2afc5e12078cf22a3505abe383f7..8dd2e803b22344b087605965cf7f92739d129baa 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.json index 6392912efe85d..f3fcfd7a86b4e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.json @@ -26,7 +26,8 @@ "literal": { "string": "/" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ilike_with_escape.proto.bin index de0d89f2c8ceeedc5bc7f97384bc057288a27b63..6a5714f5731c4a71ee87615cde9d12a5f91fc952 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.json b/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.json index 896bb3d0209da..733efb38bac41 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_initcap.proto.bin index 72df35bd9b387524d26b680f532ae85c8e506d01..9129dbf467c2879631641e519050bd0e5e94f0f7 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{>bd6^~2i3LKeLd;x@=^6|GmNN&B delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l8wd6^~2i3LKeLd;x@=>Ud#2VVdH diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_inline.json b/sql/connect/common/src/test/resources/query-tests/queries/function_inline.json index 4abdac736d0fe..f63e0184608ee 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_inline.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_inline.json @@ -21,9 +21,11 @@ "unresolvedAttribute": { "unparsedIdentifier": "f" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_inline.proto.bin index 261e28e3acaa8620adad798e0548874c3178c75f..105edc74658df18e65bff792c60fe35603b9c031 100644 GIT binary patch delta 53 zcmdnbc$krmi%Eb{YR^Qr>1wJ{%3N%jc{!PRsY2pXqFh|Li3Raxi8-aI#X_t?%v_9V I8Vni?0ANH6NdN!< delta 49 zcmX@ixSx@Yi%Eb{YR5#j=}L-H@?310c{!PRsY0Ss!dzUri3Raxi8-aI#X_t?%v_9V E04($id-{C={nj{np`}Yc{!PRsqy)xC8hqT)Bw_@nwlQrK!b2 OtU}COjACCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.json b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.json index a85f58c3b9aaf..d05cfb4126397 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "input_file_block_start" + "functionName": "input_file_block_start", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_block_start.proto.bin index 6fa8027cc82e211ea2f8d9ff539daee2ff9333c6..eed5dac21f88a139d9bd821a7378c12d4714d2ce 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json index 47f2e461eba46..7801d2afa72b1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "input_file_name" + "functionName": "input_file_name", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_input_file_name.proto.bin index c3c6414d5d8817ad9652025e75e780a7dda8992b..b1eae731498e603b55a5f9902c6a456c34807f9e 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!d(2Bc?G2<@oAYksquMgYta>H2z7I$Zpj#qni{MVX0tCGmNsIXOb&Qlea31&Kw)sqtCG R`FTRDLd;x@=^6|g3;-jR5L*BM delta 58 zcmX@Zc!H6Qi%Eb{>fl7S>DpRS8eIID#qni{MVX0tCGmNsIXOb2Qo>wZ1&Kw)sqtCG N`FTRDLd;x@=>Xqf5BC57 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.json b/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.json index f594918ed930a..12d3d19d7797a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_isnan.proto.bin index 1030abda5b8c2ac243e2e880eca30bcdb153c88f..8c2fad75be346d4d3da6765a9ca193935aea1415 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i+_#eRu?Laai}T#QK?3;=z>25kTU delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%V#eRu?Laai}T#QKoa9ajF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.json b/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.json index 6ec209e7c24fc..eb011fe4664eb 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_isnotnull.proto.bin index 1b37308d2475328d596cdbb7c7b0923a663aa2c1..5bd84491a2b942e9ff7aad8e29a65a4ac83afab7 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zuA#d-N9d8IiyLaai}T#V@&3;?i52%P`` delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_4g#d-N9d8IiyLaai}T#V@eo?-}X diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.json b/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.json index 7443fc97f42cf..11608a0de2195 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_isnull.proto.bin index 3d1fbd4dedfe7396aa90812fcecd4fc7c7203cd1..05217e9e9085bc08ff3b94c9ccae01f1b0fe569e 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RBw#eStZIYO*L%v_9#8Vmr6ga;7- delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J6A#eStZIYO*L%v_9#0DH~`;Q#;t diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.json b/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.json index 196dd4869577f..828ccaf2c570e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_java_method.proto.bin index b5cd2ea0e929bf326bbfb0cf85bb4540d1738ff8..971395d4b35736d44aa70786615258220dd15d65 100644 GIT binary patch delta 24 gcmcb?c#DyZi%Eb{>heUk>C84#RudOkFljIV08vN=82|tP delta 21 dcmcb`c!QCRi%Eb{>f%JU>C9GA784g(002J!1?d0) diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.json index 36223a451e3f8..1789c0bef0840 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_json_array_length.proto.bin index 817c803d83033a41016e50f97106635a3a912a4a..79aefffa10332be7b50c29b28a76f5b10d1f321b 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFCCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json b/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json index 32de63452c364..4f2b072db3e25 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.json @@ -30,7 +30,8 @@ "literal": { "string": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_json_tuple.proto.bin index e51be42b38d34af7e4025da5b5a42158ec7b3913..60b1685ece0f1aaebb08a6c2fa89cdaf7b5cda3e 100644 GIT binary patch delta 24 gcmX@fc$$%oi%Eb{>c~X4>CC!P+7lOOGHEaX08E$#zW@LL delta 21 dcmX@jc#@Hgi%Eb{>d-{C>CD0+W%!dx8Lr9~zA#hJxItU}COjENcy0Hj_BEC2ui delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OQgr9~zA#hJxItU}COjEMk~Vh8*H diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lag.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lag.json index dd1cba376f3c7..f0f38c35c0c5f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_lag.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lag.json @@ -35,7 +35,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lag.proto.bin index 7fd85861fb8c85b18daae2c6b8f911d617f321f4..3abeec8a65bfbef3129e1878889fc8a6530f6b43 100644 GIT binary patch delta 49 zcmcb~c$<-pi%Eb{>dHj6>Fl;q)?6x5N)wkTh-)wiu?jJBF(yj!aIp!oa4}`32rx=8 F0ssrz2($nI delta 47 zcmcc3c$1Nhi%Eb{>e58E>Fm}~mRw3w3KN$ohzhX^F>^5{O7U>939)c7Wu^!)N-zQd D`Hu*I diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.json b/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.json index 2cb1635caf47e..cbd450c2859fc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_last_day.proto.bin index 1afb5c02ae34757b16b96a3610dcc6cedae3c1b4..925a919daffc04f52edbcc3d1c98114948c8cde9 100644 GIT binary patch delta 35 qcmb=bVdG*FV3aDG$d(``DkaRtk&{?l5}%S-Da0zo%*9xu!2ke-%?B?4 delta 33 ocmb=aV&h^GV3aDE$d(`?EG5Xrk&{?l5}%S-Da0zo%*9v&0Cvd+{{R30 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.json index e78a456082cbd..18e55564d6ac0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_ignore_nulls.proto.bin index c04f8385995ee9ca79b0c55b1d45211ad3f6a2d4..0baa09d55bc200cf000636d94987e40cf53bab42 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.json index cb509997e6533..d1388758fe8a6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_last_value_with_respect_nulls.proto.bin index cee9838b70438c4b64e7ed3bd722f8cb6f9ebb09..80bf3dfcc9abb348c244dc26d5bdc534eb0cab47 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh`#iNz)HWr;bZsY0wm%v_9#8Vmrs@d)Ms delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzTAiNz)HWr;bZsY0wm%v_9#0H|>Ywg3PC diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.json index 6d1be02c78545..cb147d6998478 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_ignore_nulls.proto.bin index f6590582c6f5afc1c9171b0601afdc5a9532c182..2477c1e58803d95a648047a0a525b7601f182230 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFgLFiNz&CtU}COjEO=lTuc&-8VmrSV+Nl9 delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dxsliNz&CtU}COjEO=lTuc&-0FrSAZvX%Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.json b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.json index f26e5887ed527..dd68e3d189c03 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.json @@ -22,7 +22,8 @@ "literal": { "boolean": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_last_with_respect_nulls.proto.bin index 69221737be671a1ed43e70d8551dfe47f04a7be4..a4a02664b5030772239fc374edfd8a2803aaac2f 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFgLFiNz&CtU}COjEO=lTuc%S8VmrSU0+W%!dxsliNz&CtU}COjEO=lTuc%S0FrP9ZU6uP diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.json index a1610815b6c7d..a3a293bc7b1b8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lcase.proto.bin index d5627abb0a5d08363e26cb2ea442c3d327bc9577..e18e7e6781e767f834d008dcc4c5148162c51594 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*)$%)0OLaai}T#V@&3;=(+2B81| delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$K$%)0OLaai}T#V@eaqI?h diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lead.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lead.json index ef76586d381dd..aab3c54f48954 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_lead.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lead.json @@ -32,7 +32,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lead.proto.bin index 9bcdcdb3617a9b95b9af03892a9cf434358358f1..f72eea8071743c448898c2ed2832e9d5c30d05b6 100644 GIT binary patch delta 49 zcmcb`c!!aVi%Eb{>gq(c>FjnheUk>FhR9R$R(biW8S8iVCp`F>^5{O7U>939)c7Wu^!)N-zQd D`kM%d diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_least.json b/sql/connect/common/src/test/resources/query-tests/queries/function_least.json index 403531c9f6958..d0bcb0b07e404 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_least.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_least.json @@ -31,7 +31,8 @@ } } } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_least.proto.bin index c9ead802a9616778b643cf4e4224be7ec7970e97..bcb27b7e021148381f16f640eb05e977f111ebce 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_left.json b/sql/connect/common/src/test/resources/query-tests/queries/function_left.json index e629782ba6d5b..dbf9948ee3555 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_left.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_left.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_left.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_left.proto.bin index 497cf68194e88dc0aff3d9e3786b2c2689136252..c5b4ff7f567631d5a5fc04be6489c1cfe08d179b 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_len.json b/sql/connect/common/src/test/resources/query-tests/queries/function_len.json index 884f875a961da..3b353abe2eac6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_len.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_len.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_len.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_len.proto.bin index 939a6c9c3360b5e50c10aee85e3f41123e6b0b09..f787d98c0698e8e30d2502b5cb0c0f6d20cd7451 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b>qsd+-ILd;x@=^6|GY(oXg delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY38T9e1)2Z= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_length.json index f2c3c69255897..cbb6cf10974ad 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_length.proto.bin index a14f94085b3b698ef6d58529b8ce91edfd26bcfb..f4a53e9fa48d4731e324a2c911f31368aa86ea3c 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RAlsd?!o8A7Z=%v_A=8Vmr7CkG+` delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J4~sd?!o8A7Z=%v_A=0DMmd@&Et; diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json index 10caaf184fee5..878dc8f564869 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.json @@ -22,7 +22,8 @@ "literal": { "string": "bob" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein.proto.bin index 75b48541b766397336a93e13c5942a3b5c78526e..0742bd0585665544bb0d24be59f45a15b3cf45ff 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.json b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.json index 5cc30772e8e88..aceb63829aa8a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.json @@ -26,7 +26,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_levenshtein_with_threshold.proto.bin index 22e1a3328756eec03f6e9a10d18f29dabccc00a4..2ba96d65869c6902f61af81ec78db64f18b7b6c0 100644 GIT binary patch delta 24 gcmX@ic$ATii%Eb{YTrb*>CEaCCE9$`cnT0{}Vc1#18R diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_like.json b/sql/connect/common/src/test/resources/query-tests/queries/function_like.json index 3ce3431e50f8f..23b9578abf5b3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_like.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_like.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_like.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_like.proto.bin index d9a13f5c79bce2a6e37362877c5d67c5782118de..2f27591f3d68e86fe5a47b1d87672ac491854502 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.json b/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.json index 0313398f0ad60..04e27ea1de9e1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.json @@ -26,7 +26,8 @@ "literal": { "string": "/" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_like_with_escape.proto.bin index cc5fefe193fb9d36548dbe3933f66da41c1e9a1d..0668e85576e0487c80e19c59b197a26322e19c1a 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ln.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ln.json index ababbc52d088d..4fe4ccd2aa34d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ln.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ln.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ln.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ln.proto.bin index ecb87a1fc410205cf387950c7e60b407d310d50e..3296765cb86cdfdca6369820e85adc2760a69b30 100644 GIT binary patch delta 31 mcmZ3@xR#NPi%Eb{YS~1#=>q&xyj)BxSElTi%Eb{YRN>l>3qCW++0jKc|xp0%v_8~09X12DF6Tf diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json index 68281d2e6d9d1..5683069790afa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "localtimestamp" + "functionName": "localtimestamp", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_localtimestamp.proto.bin index b1a9e70c7c8029b525104bc5072dc23c07605bff..e280db3af4ccd7906f3a3d0fdc90e7ab1176efc7 100644 GIT binary patch delta 34 pcmb=dX5(TKV3aDI$d(`~A|=Gdmy@5Im{XFOn_66wm|LL1004?<2@(JR delta 32 ncmb=fWaDBIV3aDJ$d({1BqhMbmy@5Im{XFOn_66wm|FkC7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json b/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json index 269f39701608a..87fa8050f4005 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.json @@ -26,7 +26,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_locate_with_pos.proto.bin index 162ab0108c132eedfbeecf7239fe8359ec9c6d3e..b11ee8f8ae770954c7dd997f75d5c478bf651225 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log.json b/sql/connect/common/src/test/resources/query-tests/queries/function_log.json index ababbc52d088d..4fe4ccd2aa34d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_log.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_log.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_log.proto.bin index ecb87a1fc410205cf387950c7e60b407d310d50e..3296765cb86cdfdca6369820e85adc2760a69b30 100644 GIT binary patch delta 31 mcmZ3@xR#NPi%Eb{YS~1#=>q&xyj)BxSElTi%Eb{YRN>l>3qCW++0jKc|xp0%v_8~09X12DF6Tf diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log10.json b/sql/connect/common/src/test/resources/query-tests/queries/function_log10.json index 13292d83c4727..bced949b9aaf1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_log10.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_log10.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_log10.proto.bin index 22d4655a6efbd07b312cd611c7528022f9f488da..1a363b7043dc32ba0eb810fbf1b0c86dae921c69 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*)`RRrRLaai}T#QK?3;=tl1~&iz delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$K`RRrRLaai}T#QKoZmk9f diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.json b/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.json index 4e9e6847c3c36..95942299457da 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_log1p.proto.bin index 9a72c377b0cc488a9333c485c68d38f9a5805620..6097a554cee23bc5cfbc7d02677fdcc26eb3b177 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*)`RRrQLaai}T#Sht3;=#426g}d delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$K`RRrQLaai}T#ShTaI6MK diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log2.json b/sql/connect/common/src/test/resources/query-tests/queries/function_log2.json index ec29e154a0e1d..4fe3d22e03415 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_log2.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_log2.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_log2.proto.bin index 34e37806505408315155610a7f28fb1ed6bbb562..39aab70e5ac3ef7606742a40637ab21a889fb0ce 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(K`RPVNtU}COjENcy0CVmJ00000 delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0f`RPVNtU}COjEMkeY6Z{$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json b/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json index 6bc2a4ec3335a..3d6e05077dd6c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_log_with_base.proto.bin index 2e64e15ed5555f285a98a089e8aacb4a58865b0f..95f9352d73f93de7a12e413d4d3e92833b29c7fe 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lower.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lower.json index f7fe5beba2c02..e2b4a5816e686 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_lower.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lower.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lower.proto.bin index 7c736d93f77290834160f512253b6b9423262796..f9f4d930ebc23152e95fc8be10469fb7a0879439 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*)`Q@oaLaai}T#V@&3;=;f2EqUU delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$K`Q@oaLaai}T#V@eb2bK& diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.json b/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.json index b9f3e6700bfa4..7cdd2b75dadc6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.json @@ -26,7 +26,8 @@ "literal": { "string": "-" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_lpad.proto.bin index 470ab1cc44add5872568eacabf43ddd7a08fa843..8576f2b0be3b7377e6929a08385535dc4466eaaa 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWF@dt>B2%%0$i*)B}JLJLaai}T#V@&3;=;e2EhOT delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$KB}JLJLaai}T#V@eb2kQ& diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json index 01dc81bdae7bc..f67f689d0d950 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ltrim_with_pattern.proto.bin index 0cd62226c9716a638e99894e1bf426e1adc5f812..6a9d3bf7b00ef3d65155720adda5598992e986d4 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.json index a363298dd123a..0f78921e965e5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.json @@ -26,7 +26,8 @@ "literal": { "integer": 14 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_date.proto.bin index 0526825fccadeec8951afd684fdb85af8d1faf77..c47ea55dbe6cc161d21a9a4dc6d86e5f6a812c59 100644 GIT binary patch delta 24 fcmeBR>|$i&ViI7KYM#iJ$gCu#FfmJxNrM3ZIUodY delta 21 ccmeBT>|kW$ViI7KYMjWH$gChGH!({N04vo5KL7v# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.json index 24cd85d7ae442..66e5da74d5e45 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "make_dt_interval" + "functionName": "make_dt_interval", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval.proto.bin index 09c5a25a10cc2177ac35f10eda95ac2e025a5ddf..00bedb03e9dc4d6bb5f9f908e1aa6e9f5ca89b9f 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3uHwiP@>~DJAilc_pbuWr;Z&3;?_O3bz0N delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR~DJAilc_pbuWr;ZerQiyO diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.json index fb65f1fcd9def..9fe3d9dedbc83 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days.proto.bin index 11c67d98f9f4982e5897d2d8c04c6565478434e6..70cd3d296df6a302268620d420e8d95d59d8941e 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.json index e08a9c3b083ee..f902694ef7774 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours.proto.bin index bd16de042f61d451587bbd26508b02c0863041d1..d8107489e1e78456958e743fcd25ed47d244ed7e 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.json index a4b8c14538ae3..5a760c1b469fb 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins.proto.bin index 7595205c6bb0eb39320d8d601c40227610ef8180..89a249a8535fc21cfd1ea13df5aa7d989d4f6990 100644 GIT binary patch delta 24 gcmX@kc#@Hgi%Eb{>d-{C>CDcB*{>CBo^>Jt~J0{}YV1%m(p diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.json index 20eaa7521d3d9..745012b755db3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_dt_interval_days_hours_mins_secs.proto.bin index 6db1bc8b51bea447b0dc754936b3e832763a198c..ba6a2b4e10f82758a314aa7a2f7707ced4cded92 100644 GIT binary patch delta 24 gcmX@dc!80Pi%Eb{>f}VW>C8q_1``+PF=;RW08S_c-2eap delta 21 dcmcb>c#e^ci%Eb{>i9&q>C6UFdJ`Au0RTLc1+4%8 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.json index cdbe616565287..eceeeddd4b15b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "make_interval" + "functionName": "make_interval", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval.proto.bin index 8d4327eeff426999b0bb85e2c65721ad38274b45..c569945e7b4c15283ab8272271d6aff0143b3ec9 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?T}0iP@>~nRz9tMP-RO8VmrLkO^x5 delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9L@biP@>~nRz9tMP-RO0EQ+BI{*Lx diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.json index e5afa5ec3349a..e49924238ced8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years.proto.bin index 7be990a47aba6668f494c0b98ca08ad5648ccc44..84ed402ba145fd3974f32bab826bd60f7c422019 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.json index 9de86b70c169e..a7f87a2015bc1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months.proto.bin index 219cc5a023d450a63dfac77460bf6640130ca55d..a886b4d11afd3ddc4d567ed6e5539dc9ec9e3211 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.json index c387757c6f739..7ca71d529325b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks.proto.bin index 7fb48227f69c7fec20ffbc028a76ac208fc948f7..af3f2d4b4519f4201b3a0d5ffefcfba3e205bb0f 100644 GIT binary patch delta 24 gcmX@Yc#M&ai%Eb{YX3yG>C75ZY7-ZzFljIV080o3p#T5? delta 21 dcmX@cc!ZISi%Eb{YVSn0>C9?UDiar|0025C1#tiX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.json index 54274116ee951..d0741256492d1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days.proto.bin index 58d62b76ac5f94e24ee973b0579fdad0e262cf1c..0d3355c516bdd0761c7037143867e4667a81ba5c 100644 GIT binary patch delta 24 gcmX@jc$Sfki%Eb{>exiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.json index a6e343532ec2c..aecddb3dcb53f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.json @@ -34,7 +34,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours.proto.bin index 3133c2d497ea6ffc25dc1bf9338d0e5276f132c5..52b889c8ad81b3cbbc2823900dee43d78330d937 100644 GIT binary patch delta 24 gcmcb_c!iOTi%Eb{>g+_e>C9$QCKDGJF=;RW08eNI^#A|> delta 21 dcmcb@c!`mXi%Eb{>hwgm>C7flMiUnp0RTOV1;qdW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.json index d921fda962896..b5ed4c2e8d4e8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins.proto.bin index cf2ad98b5c16cc0893a36e9f886cd9cf55155fd4..7d06032ee8a45b73216847fdda73536d0d1e7927 100644 GIT binary patch delta 24 gcmcb~c$<-pi%Eb{>dHj6>CCoL))N<4GHEaX08y9*9{>OV delta 21 dcmcc3c$1Nhi%Eb{>e58E>CDzrmJ=6P0sucf1@8a= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.json index 09f95bc933b10..372b13ce44621 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.json @@ -42,7 +42,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_interval_years_months_weeks_days_hours_mins_secs.proto.bin index dca680fd90b61fc7815eac109ac3f3824791ab7b..033d88d328ed170703b5d10c9b027fa2422cd27c 100644 GIT binary patch delta 24 gcmaFB_=u5>i%Eb{>h?so>C7%tP7@b6FljIV08_{YNdN!< delta 21 dcmaFF_<)g(i%Eb{>gGhY>C8@24igtR002Pm1{nYV diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.json index a58259eefe742..7572d311648c0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.json @@ -42,7 +42,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_with_timezone.proto.bin index 3eac6534c651028e7cdaf2a3772def9322513d00..ecc490af0f3c49b7d60037dc12154fbe69fa8993 100644 GIT binary patch delta 24 gcmaFL_?(fAi%Eb{>cK>|>CB!|?h_ZdGHEaX099B9X8-^I delta 21 dcmaFP_>_^2i%Eb{>fS`Q>CEm@t`iry0sulK1~&iz diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.json index 5c87a856fc6c0..9e10c499ee9ee 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ltz_without_timezone.proto.bin index b35f9fd47460758c639f7f040ac3960b6608334f..2baa0d4f269d475e883689439fd52b9c4c9b37e5 100644 GIT binary patch delta 24 gcmcb|_<)g(i%Eb{>gGhY>C8@24igvHF=;RW08=OiJpcdz delta 21 dcmaFBc#n~di%Eb{>iR^s>C6sNb`uxa0RTXA1`PlJ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.json index 4cc4f1a11acdc..6241859ae9269 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_ntz.proto.bin index 5a6554443cecaca465939b24cf3c89e5f03aefa0..17dc37d99f63f10d1dee2105b415c2cd4d95703f 100644 GIT binary patch delta 24 gcmcb|_<)g(i%Eb{>gGhY>C8@24igvHF=;RW08=OiJpcdz delta 21 dcmaFBc#n~di%Eb{>iR^s>C6sNb`uxa0RTXA1`PlJ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.json index a7a5ff132c083..a8d427ae58c0c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.json @@ -42,7 +42,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_with_timezone.proto.bin index 77c0d5961c804fe293f800fe60331afe57c8df5c..660c4399e35f4499dadccfea8d334103a19d5bb1 100644 GIT binary patch delta 24 gcmaFJ_?VH6i%Eb{>dr*A>CCQD&J!0nGHEaX08|(TPXGV_ delta 21 dcmaFN_>hr}i%Eb{>efWI>CDbjjuRI+0suiR1|I+b diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.json index 286ed33f82e10..8426f3dd45143 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_timestamp_without_timezone.proto.bin index f91efead687a81e312b54266bab1416a5b314667..348befe79dbf8441fabf5418a984d0cea05b180d 100644 GIT binary patch delta 24 gcmcb`c!!aVi%Eb{>gq(c>CARgHWL?EF=;RW08!`$B>(^b delta 21 dcmcb^c#DyZi%Eb{>heUk>C84#RudOk0RTUH1@!;` diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.json index dc1848be0cc42..f37ca015eed91 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "make_ym_interval" + "functionName": "make_ym_interval", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval.proto.bin index eaffc7c237094e0599099a5208904cee90a31888..0c63f66caf9a0fde42ce8b47cae51f0d1fcc5a12 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3uHwiP@>~mAUbmc_pbuWr;Z&3;?{h3dH~b delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR~mAUbmc_pbuWr;Zerhy8T diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.json index d789064ad9b8e..81b70cf342205 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years.proto.bin index 1938b7c53bdd12b23f499579246e475cd9aaba8d..7486df5d7530d2eac3eba6104b6f3ddb28be3bb0 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.json b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.json index aeffbbb4a1a14..1eb67c528682e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_make_ym_interval_years_months.proto.bin index f03f6ecce83dea11157072c299b463a3fbdc9f67..5a85f5981b948a2c31e88c92421279d3721a3d82 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map.json index ca9d3bf2bcc71..830445d3facf8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map.json @@ -30,7 +30,8 @@ "literal": { "string": "dummy" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map.proto.bin index 229a48b75131dd161d898fa5e332e98b1f5cc30b..32b233fa939db8c1e82e3ad19dabd9be3bf0684c 100644 GIT binary patch delta 24 gcmX@Yc#M&ai%Eb{YX3yG>C75ZY7-ZzFljIV080o3p#T5? delta 21 dcmX@cc!ZISi%Eb{YVSn0>C9?UDiar|0025C1#tiX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.json index f56f6cee20ab0..6e076de494412 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.json @@ -55,11 +55,14 @@ }, "name": ["b"] } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_concat.proto.bin index 0a76d3a1193ea04f82f0d1636378c59c2ee3064b..6c04979af6da78033010ac5a1e49f99bc316eb8f 100644 GIT binary patch delta 53 zcmZo*YGY#KViI7K`ah9vI&+3p+QbDSqW)68T+F$N1w!mxtXa%y`T0WDQkD}>nzLyz GfFJ-`L<~a! delta 46 zcmZo;YG7jHViI7K`ZC9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.json index 0226506545010..310b3980931a1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "f" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_entries.proto.bin index f1451d4ad7ba4847ba1128a90cb38fa3fc0d4dda..1050436839a0a108dcbe36e1d8153b04e8710062 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.json index b50e77c0bf8e9..48c7871244f19 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.json @@ -31,7 +31,8 @@ "literal": { "string": "baz" } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -40,7 +41,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_filter.proto.bin index 7f3d0c31fd6feebdae69670c0c29000f9d3458d9..a6143e55099ffdbd11314189b6b6240e0f7e3fbc 100644 GIT binary patch delta 51 zcmaFL_>z&0i%Eb{>d{2D>CE0zo)Z^{up0##a4AU1P24Rbq`@G>%Eep}ZwR6)J E08Cs9RR910 delta 47 zcmaFK_>_^2i%Eb{>fS`Q>CEm@t`irCup0#Famh)^Ox!KQC&bFdToG>wqAKH!03U1$ A?EnA( diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json index 1eb1f7d2ef066..0ce0f1f547174 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.json @@ -25,7 +25,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -38,9 +39,11 @@ "literal": { "string": "two" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_arrays.proto.bin index f5333b1c882bcd6d208f5316356bb3f13a2748d8..e85aba1a35ce7a055149d5fcb3a80a78636a35db 100644 GIT binary patch delta 72 zcmaFL_?nT8i%Eb{>d8d5>4v^i-dy~-i3Ra#Mfth$iA6<;mBm8hQlea}V73qo7n1=a Vm}Jsm5R#RWo_I!*MT0?u0RYL_5aR#< delta 65 zcmaFO_>_^2i%Eb{>fS`Q>H6+cu3Y@Ni3Ra#Mfth$iA6<;mBm7$Qo>xUV73qo7n1=a Om}C-?mXe%!P7(l97!XYW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json index 1e48a1c2082df..e505d884d85f6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.json @@ -34,7 +34,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["x_1"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -43,9 +44,11 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_from_entries.proto.bin index 0dd0d313509916aa73b088db1d52f7a9d8fd34ec..88d178e75c4feda9827188e96897b964c3f0ebb1 100644 GIT binary patch delta 90 zcmeyt_>Yl|i%Eb{>ia~t>Bb3Caa;nqi3Ra#Mfth$sd*(unW@D>9#U>xoFzqxdBth@ rMY%$(Ld;x@sX_)pdR%f+G7~RIiD)ngv2rn2#2bRB%6KCU1`P%Psf=PV>Bdn~5nKYfi3Ra#Mfth$sd*(unW@D>E>cchoFzqxdBth@ lMY%$(Ld;x@sX}@|I$SbRQWGyo@d>eVF;~PJf~d-PBLE#R7Ek~H diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.json index 5af013295cd9f..1e25cd90ac88c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "f" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_keys.proto.bin index ee19968bacc2c9253e6d1d819bc4888bf78d0e8b..b45fde5acd6bd491b40d5793810689768854f838 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx7=i3Rc5sg=b-tU}COjA0+W%!dzUri3Raxi8-aI#X_t?%v_9V0H_uSxBvhE diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json b/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json index d13bd8dce75f3..9e913743009d1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.json @@ -53,7 +53,8 @@ } } } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -64,7 +65,8 @@ "nameParts": ["z_3"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_map_zip_with.proto.bin index 2770b083e32ef84f00756cd157497addfc1921ae..65dba7ed7bbd13eece18d2964551fca82049fd43 100644 GIT binary patch delta 42 wcmZo;>SAK&ViI7KYG#_qK8-m?Dr@2bS$3}=4=zI~{fS3(nKT$CeiQ|g0MB&_BLDyZ delta 55 zcmeBTYGY#OViI7K`p-C#eHwFyRNBM^vg{r~Ze03Ox)YD+iVCrEF;~PJf~d-PBM?;; GZwvru>xSo-Xi%Eb{YQ;ph=|X~1{9G)#iN)DMtU}COjOiK-0Cpn=J^%m! delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(1~iN)DMtU}COjOhSrBL)rt diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.json b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.json index 571d514e72ded..6dc93c69e10e8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.json @@ -22,7 +22,8 @@ "literal": { "string": "X" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar.proto.bin index 0f6c4b579c4f54a94fd051dd05f540bb376d4cc3..699d662fcca48afc476ccf8da95485e261cf8e07 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.json b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.json index ae527d70cf162..671a19a8900af 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.json @@ -26,7 +26,8 @@ "literal": { "string": "x" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar.proto.bin index 5a6b4d7caa60e58dccda311241d89f9bf9de49f0..c754da066d573844ea8b2db2a9339ea24a92e6cd 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.json b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.json index e7fee11d3169e..f12f44b3ecb38 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.json @@ -30,7 +30,8 @@ "literal": { "string": "n" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar.proto.bin index f0a2e7cb643af656c46652416df9cd55f95c145a..473f724d4126c9b7384a9b01eed0a57d47b32111 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.json b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.json index d6076ae558bc7..1ddb661b636bb 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.json @@ -34,7 +34,8 @@ "literal": { "string": "*" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_mask_with_specific_upperChar_lowerChar_digitChar_otherChar.proto.bin index cb5f090361b203d99a5880ecc5e106b06d622220..bf641173435f3d96855f6a0c9e7dae1da621ac91 100644 GIT binary patch delta 24 gcmX@fc$$%oi%Eb{>c~X4>CC!P+7lOOGHEaX08E$#zW@LL delta 21 dcmX@jc#@Hgi%Eb{>d-{C>CDCX5(TKV3e|*$fhkMD8EWaDBIV3e|+$fhm8FU8BnoSRr7#3sbT#gv%>06b3wUjP6A diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.json b/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.json index da311e340cc50..1048a30325e5a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_max_by.proto.bin index 284c2453af8bd5f3af72d89637ddacb0c7899f60..1f1832962cb3450dce553a2fe3b978069d9afcef 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_md5.json b/sql/connect/common/src/test/resources/query-tests/queries/function_md5.json index e8718594b0be3..d954d60a9c68a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_md5.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_md5.json @@ -26,7 +26,8 @@ } } } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_md5.proto.bin index d3ec7c26a2ede0e92ccb62c1993d8037a6b29b95..87ee03e94008128fd219749f16994775309b5fb4 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_median.json b/sql/connect/common/src/test/resources/query-tests/queries/function_median.json index 7331454b9ecb0..a358a25d85705 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_median.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_median.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_median.proto.bin index 59533e5be59924090d07ab742f028ae80a9c05fe..5a80a2f7cd44b278634522289008d55cc8146fdc 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RC5sVSL>c|xp0%v_9#8Vmr6B?kxq delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J6gsVSL>c|xp0%v_9#0DFN3*Z=?k diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_min.json b/sql/connect/common/src/test/resources/query-tests/queries/function_min.json index 1b7266b6774e4..3fba2b795a224 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_min.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_min.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_min.proto.bin index b82f4c5309222c6d38f08e88b8e6b8169c949303..2ae3da3391fa531a48798fd82a0c9c5a14a60690 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@AnR!C2Ld;x@i5d(5Y-t6_ delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AVnR!C2Ld;x@i2z_*1(^T< diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.json b/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.json index d2478f5e81abe..6c9b99ad7d43d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_min_by.proto.bin index ddc642b95000c2b3405e3940d8481236621d6679..da76415ec74a0de7849330d6cc1f7dfd8c842043 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_minute.json b/sql/connect/common/src/test/resources/query-tests/queries/function_minute.json index 7c749cdff82f5..c94a8703d38e9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_minute.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_minute.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_minute.proto.bin index e81b7dad85331d6581c5bfd0886895443f07ec39..b0b743773e9026b7361a82599c8da0f6173d301a 100644 GIT binary patch delta 33 ocmb=ZV&h^GV3aDE$d(`?EG5XrmYbPZT9PWnD#XmiSfarI0CCj@VgLXD delta 31 mcmb=bVB=yEV3aDF$d(`^D8K9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_mode.json index 8e8183e9e0883..a4f3b601ad47a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_mode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_mode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_mode.proto.bin index dca0953a387b1b29ac78aca4c6bf293ef9119c88..e3dfc96922e62dd754d73b02ec45cf357068e3fc 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)#`6;PFtU}COjENcy0Cm^~F#rGn delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(1~`6;PFtU}COjEMkf@&*9_ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json b/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json index 0a14f1008976e..01ca4536c97f9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "monotonically_increasing_id" + "functionName": "monotonically_increasing_id", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_monotonically_increasing_id.proto.bin index 724ce3ac6904ce00d54fbcd274041b0a25f79acd..2e86a0566afafdc730d3c0edba4e90d9ac20b156 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_month.json b/sql/connect/common/src/test/resources/query-tests/queries/function_month.json index 7ea1e5d0375e9..6343ed28faa01 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_month.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_month.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_month.proto.bin index b97100a6fe2ecebc0a448a28a90c74cdd5d69d8b..1c09b5d1f26c2c900bfda19304575b4d94ac01c9 100644 GIT binary patch delta 32 ncmXS~WaDBIV3aDJ$d({1BqhMbnwy_jk|D$@#LUH*qQL+FXSN03 delta 30 lcmb;{XX9cLV3aDH$d(`|AjQYUnwy_jk|D$@#LUH*0svcT1+M@A diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.json b/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.json index c5ad3485252f1..7da09908cadaa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_monthname.proto.bin index 4518bb8d74253a98c6fec30f76a1d3cb6466a6a7..958b9b34dff866ddedab87e911f74f10f8ec3891 100644 GIT binary patch delta 36 rcmb=fW#eKJV3aDK$d({3CMCkfnVX+il988~n<~UA#LUH*qQL+FkwdCKq3BeqKpNaePv0NqK5&o{)eP9~Vz@X;D#XUP*jP OVo9nHs}M66V+sJoh!1@L diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json b/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json index d11bfbd7f2426..f03709aece83c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.json @@ -16,7 +16,8 @@ "functionName": "months_between", "arguments": [{ "unresolvedFunction": { - "functionName": "current_date" + "functionName": "current_date", + "isInternal": false } }, { "unresolvedAttribute": { @@ -26,7 +27,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_months_between_with_roundoff.proto.bin index bf9c545911ffdec7a30be779876f4e30bcb3f534..a072570756411ca469a69a8072ffb5248d4f5e96 100644 GIT binary patch delta 69 zcmbQqIG2%)i%Eb{YT87$L^D$ws2}TVD0B+O}KmY&$ delta 65 zcmbQsIFpf$i%Eb{YSKiuL?c5feJ;M-{JfHk;`pT0lJeBlJRt!oJ}#c*(xRf&yps5o U#FA7YRv~6C#uOnIE+z>^04gjG+yDRo diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.json index c4d92131ed06c..a23893ddb0fcf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_named_struct.proto.bin index b595cfc282036d68c081be7877ff21e9b19de6c8..87c310823426b6c1cb7214d34b447260c84b13f6 100644 GIT binary patch delta 24 gcmX@jc$Sfki%Eb{>exiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.json b/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.json index 69daab270c2b9..151c3d830716f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_nanvl.proto.bin index f314a73dcae6583f46ecc4871aaf6fe316fd9772..edddf04956a7415df4b784463c3bf8ad3073c093 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_negate.json b/sql/connect/common/src/test/resources/query-tests/queries/function_negate.json index e269fabe44be1..96f9359193fbd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_negate.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_negate.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_negate.proto.bin index 9c56c111ceee620255a78beba6fbb5b7ee055781..f4d42e82e0c0dbd1010446dfab03509bdbbbaaf8 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx7Asp*L&nPsU$tU}COjENcy0HLl2`v3p{ delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OPVsp*L&nPsU$tU}COjEMk|6bH)y diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_negative.json b/sql/connect/common/src/test/resources/query-tests/queries/function_negative.json index e269fabe44be1..96f9359193fbd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_negative.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_negative.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_negative.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_negative.proto.bin index 9c56c111ceee620255a78beba6fbb5b7ee055781..f4d42e82e0c0dbd1010446dfab03509bdbbbaaf8 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx7Asp*L&nPsU$tU}COjENcy0HLl2`v3p{ delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OPVsp*L&nPsU$tU}COjEMk|6bH)y diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.json b/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.json index 486523dcad3ec..ad6f3bb22ff82 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.json @@ -22,7 +22,8 @@ "literal": { "string": "Mon" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_next_day.proto.bin index a97bd75f129dbbfd063b4ce894e63db4703706c3..8cece90ab671b52fc6370d6fdd7c5dc5a0978422 100644 GIT binary patch delta 46 zcmZo>Y-MEQViI7Ks-MV~ATK8+!^M%8T2T_8l2|FkD#Xmim?Ff^#hS(Jo1dq_007tB B2_XOg delta 44 zcmZo=Y-VKRViI7Ks-4J|AS)v!#l?}AT2T_8l2|FkD#Xmim?Ff^#hS(Jo1X^&#IFeP diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_now.json b/sql/connect/common/src/test/resources/query-tests/queries/function_now.json index 98556585c3e31..1ceb0bd1366ff 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_now.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_now.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "now" + "functionName": "now", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_now.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_now.proto.bin index a8fcd67fa19824a36079a873d636b95afc371388..f9ab22b653c09122824c71f45e5d35f699fd5b6f 100644 GIT binary patch delta 23 ecmc~RVB=yEV3f+5$de58E={D9;mRw3w3S69dB^mK$i8-aILaai}T#V^LEL=@m?*`=#Rd|}Oc7v|U<3eE{|t5j delta 73 zcmcb~c%6}ri%Eb{>cT{}=~k9e=3EL=a$KBwB^mK$i8-aILaai}T#V^LEL=fl7S>0(+@8eD=>{9LSgC7C&?LM&WN1}qv36E}(i0J)_I!vFvP delta 56 zcmX@Xc#M&ai%Eb{YX3yG>6#i*YFzwMyj-k#C7C&?LM&WN1}s9XLd;x@iBddVY#@Qm J6ahvFMgTrR2{Zrz diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.json b/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.json index 3892eb19fc52c..ac9f5620f9243 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_nullif.proto.bin index 9bbf5f4ccb8ac489b47b3f7d296c1a4515467b01..0217381b686ff386cd8bb8fb9678edda09034fa6 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.json b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.json index 483448c26d114..1bf1f22bcad2b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl.proto.bin index 21a9b37eb65ecf9008f82f20bddf43c52fe960db..663f7c714883c908f07e937288ab0c3f356dc9c4 100644 GIT binary patch delta 39 pcmdnYxRsHOi%Eb{YTZP(=@Q~nqFl^*WjR8uLd;x@=}=080RWtQ2BQD~ delta 37 ncmdnWxS5fSi%Eb{YRyEp>0+W%!d%RGWjR8uLd;x@=}-y)j)Der diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.json b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.json index 8db7f9ba6292c..408d6eba0a05b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_nvl2.proto.bin index 8b7f90bf27552e33fae32f21d3104f6e8929abed..627a671a085e250ee6139b55f97f5f5bd514ec8f 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.json b/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.json index 7be9ac82662a4..bfd624216f70d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_octet_length.proto.bin index 484ebbb6487b07883ee03136d27e9a1ea110a454..3f94747ea25954118b1b89db897fbbd14f7d1417 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.json b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.json index b580570f923a6..28cb276fe4cb1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.json @@ -26,7 +26,8 @@ "literal": { "integer": 4 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay.proto.bin index 2110ae9c146104723d1b83ae67c8fd91f392ec18..43d75d4a0723130a07b8013ece4b1bf8c6dfc3ba 100644 GIT binary patch delta 24 gcmdnbc#x5ei%Eb{YS%=z>CDPfiW3*eGifjY07-ZSg8%>k delta 21 dcmX@exSx@Yi%Eb{YR5#j>CB2!@)H-x0{}Sj1ycY3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json index 99d5426c46fba..79f0be1011dab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.json @@ -30,7 +30,8 @@ "literal": { "string": "3" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_overlay_with_len.proto.bin index 9a09d28d84fde55e0fb5c7575e6eedb7af76ee6f..9521c75b4a83e551dc41c4b7e0aec42e950fbd76 100644 GIT binary patch delta 24 gcmX@cc!H6Qi%Eb{>fl7S>C9SE8WR_&F=;RW086L^tpET3 delta 21 dcmX@Xc#M&ai%Eb{YX3yG>C75ZY7-Zz0RTFq1$_Vj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.json b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.json index dfcf56c19223e..4a84cbe99b65d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_json.proto.bin index a7187fa2c1af0edcc58e257ffed7cf48671139f2..cea247c862173440fed53af3f425613e6e96721d 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh`BiABY!@ma0+W%!dzShiABY!@maCCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.json b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.json index bd627911ef22d..862e3b5aa8d3a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_parse_url_with_key.proto.bin index 231622cbd8a6bf9488cd0e9755ff3e3f8e20583e..543b9dd6f43dca82bd8238d09f24615eb5d92a3b 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json b/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json index d8778ec8cd81d..4770d3d81d6d5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.json @@ -15,7 +15,8 @@ "window": { "windowFunction": { "unresolvedFunction": { - "functionName": "percent_rank" + "functionName": "percent_rank", + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_percent_rank.proto.bin index d668f7e1504cb1d3a7f727bed1e342e356c47208..2dcfbf3777ee9156f4ccf2b40105bf61758c5ea2 100644 GIT binary patch delta 59 zcmX@kc#@Hgi%Eb{>d-{C={nj`np{Fs0$e-=sYS`Dc_r~hiFw%?3_`3z%v_9#QaoI2 OLM&WNnJEH{5{v+#a|{6h delta 57 zcmX@fc$|@qi%Eb{>cB*{=~|jm>RbX+d|W&QsYS`Dc_r~hiFw&VtU}COjEPb_Tx>!t MTuhlM0*n%j0EZ6?%>V!Z diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json index 6289464de2a37..16f5f1f2b4b7b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.json @@ -26,7 +26,8 @@ "literal": { "integer": 20 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_approx.proto.bin index f44ec86888f6ccd9c503b0e7115a3fbaaa3f3a26..159c21c647729d7165a93cc3bdba39d29b6eb96e 100644 GIT binary patch delta 24 gcmX@hc%G4si%Eb{>cm90>CA>w`V$xEGHEaX08Q8h*8l(j delta 21 dcmX@lc$Sfki%Eb{>exiK>CF04x)T@Z0suT!1*ZT2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.json b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.json index f57804426643d..fb548fd0233e1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.json @@ -26,7 +26,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_with_frequency.proto.bin index 91d6279f9bd8c60cfb377e080ba6fc50ef9a1ea6..13be09fc2901a9dcbe05c92ea8899b5da3790be1 100644 GIT binary patch delta 24 gcmX@cc!H6Qi%Eb{>fl7S>C9SE8WR_&F=;RW086L^tpET3 delta 21 dcmX@Xc#M&ai%Eb{YX3yG>C75ZY7-Zz0RTFq1$_Vj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.json b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.json index 44e2c98a4dc60..3af4be6aad5cd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.json @@ -22,7 +22,8 @@ "literal": { "double": 0.3 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_percentile_without_frequency.proto.bin index 45b807e5ffbd20b3cf6c2d71f6d5f6f6184fc785..93156e49b45562a0229dafaa14449de11084f393 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_pi.json b/sql/connect/common/src/test/resources/query-tests/queries/function_pi.json index 46474dfd8e369..d73ca1d6ca691 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_pi.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_pi.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "pi" + "functionName": "pi", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_pi.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_pi.proto.bin index 14f018904bfb70238a8993f1478b799a4b288952..33fee152702575f9fd9e4a9a4da69b05e10d216b 100644 GIT binary patch delta 24 ecmZ3&xQvmFi%Eb{YW_sF>6{!=Ku$rX1_J;+=>(Mk delta 22 dcmZ3+xP+07i%Eb{YVJg~>FjJ$EL=0+W%!dxr`x%nwVtU}COjEO=lTucUB0Fsghi~s-t diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.json index f8a9db37e62be..261ee78e20b43 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode.proto.bin index fc50f5f4c85b759c7767b3297cd5943c3d57814b..9b22124951e8088054856ecf1803adadd27d989e 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh`B`NgRf1v&XCsY0wm%v_AA8Vmrua0vYX delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzSh`NgRf1v&XCsY0wm%v_AA0I9+V%>V!Z diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json index 0e8cd4c1509e1..71d11fd72cd8e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_posexplode_outer.proto.bin index 19d700665e7f5be995e1466a54e801c4a92b795b..d771cb8c3373928be46661bd66a9ef1ed27227ce 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_position.json b/sql/connect/common/src/test/resources/query-tests/queries/function_position.json index 7b005e2bb8213..e71a363461f8d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_position.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_position.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_position.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_position.proto.bin index 34b7e301fe943f6fd6c0f3ad6dc3c5768d306bfe..db2530e0625ba278182c31930e12a9774c3c0b94 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.json b/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.json index 2cd04992d1da8..0a1f3fc42a29a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_position_with_start.proto.bin index b34eaf80f88666ff0cf13bada2467553b892f563..4f092eae4057ab73ba2673131db6470daac82fa0 100644 GIT binary patch delta 24 gcmdnbc#x5ei%Eb{YS%=z>CDPfiW3*eGifjY07-ZSg8%>k delta 21 dcmX@exSx@Yi%Eb{YR5#j>CB2!@)H-x0{}Sj1ycY3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_positive.json b/sql/connect/common/src/test/resources/query-tests/queries/function_positive.json index a8b3a2d6244bb..26f8ae17bbd19 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_positive.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_positive.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_positive.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_positive.proto.bin index 5507abce8caac273bba40275db695a556172e9ed..32e7859676cfeca79ffb2b38e7f5be13bfab0e74 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx5$`Nf$fnPsU$tU}COjENcy0Hc%$8vpCBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_power.json b/sql/connect/common/src/test/resources/query-tests/queries/function_power.json index 187636fb360c6..b9b24218fd99e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_power.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_power.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_power.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_power.proto.bin index 6e1d3b06fe87a8d2e63181debdf9bc92ba0f24c0..52a249ada18f63b1cec667a609b13e35661a0968 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_printf.json b/sql/connect/common/src/test/resources/query-tests/queries/function_printf.json index 73ca595e8650b..c3ead5f1388b5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_printf.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_printf.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_printf.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_printf.proto.bin index 3fb3862f44d91262510d0c07615e0256348e4a8e..b78d5046c98dc5f0d5d42cd4bb720f8008323b5d 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_product.json b/sql/connect/common/src/test/resources/query-tests/queries/function_product.json index 1dfb7f81912d3..802f3e77e8cd1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_product.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_product.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": true } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_product.proto.bin index 8c3fbd31eb6b37ffa0ddc6cabd5d165ee84ceedf..a7ff0061481b44884b34bb2a22da61f82ee21da7 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{>B2%%0$l6`MfoYE$t6OpLd;x@i2#R(2XX)a diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.json b/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.json index b95867e0be963..eb8e75f914318 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_quarter.proto.bin index fdc2d96fb08ca1c4405e99f2823f512705eb44b8..d9ad4e20a3aedceb08a608c4d555f26f5b2f277c 100644 GIT binary patch delta 34 pcmb=dX5(TKV3aDI$d(`~A|=GdURat~RFYaG#45zh#h9YO004Yz2Fm~d delta 32 ncmb=fWaDBIV3aDJ$d({1BqhMbURat~RFYaG#45zh#h3yBZZ`&- diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_radians.json b/sql/connect/common/src/test/resources/query-tests/queries/function_radians.json index 837960dedc653..83f211272c123 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_radians.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_radians.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_radians.proto.bin index 33a2521b22ac9351987e8ee8f0ffd4cbdd165f72..e371fb5947a5b0d6f32c76c711278700a1482ee4 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=ci7A>B2%%0$l7xi7ACCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json index 453ea54bd0ef3..67c1250a72ff8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.json @@ -18,7 +18,8 @@ "literal": { "long": "133" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rand_with_seed.proto.bin index 566a49d641293220741f81b905f62bafebd4006b..cb15624497821e77f8241b2b5d5b30b82fa5cfb5 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(WiFqkPtX#|%t&AEB0C#f+djJ3c delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0riFqkPtX#|%t&9L^`36e> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json b/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json index ef84f05c3e193..f043d22159b20 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.json @@ -18,7 +18,8 @@ "literal": { "long": "133" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_randn_with_seed.proto.bin index b0064842bf3082b12656b7686602fc3db202ae8a..6f601e62c262bfbb845d9611ca7483e6b9ae7a7e 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i*`iFqk`LabcO7OjjL3;==A2Jip? delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a$WiFqk`LabcO7Oji`b7cm= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.json b/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.json index 11238a43ec1a3..fc0c945b825ae 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.json @@ -18,7 +18,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_random_with_seed.proto.bin index aa4208afedb88a32d5be848b15aeab81cfed2b73..918de15afa6b57d71e620d78c2a4e0c6d192c26b 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$glGiFqmcxk4;lOa_b^3;=^82G9Ti delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|YfriFqmcxk4;lOa_bqboU0J diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rank.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rank.json index 93c8dc38d668a..905af83a134f2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rank.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rank.json @@ -15,7 +15,8 @@ "window": { "windowFunction": { "unresolvedFunction": { - "functionName": "rank" + "functionName": "rank", + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rank.proto.bin index 3aef331fb1739f48edc3ac768addb173412eaba2..8d72dd7fddafa85d1ef3662d4ee05556ba90f958 100644 GIT binary patch delta 51 zcmdnbc#x5ei%Eb{YS%=z=_<-mid!tTuhlM0*n%j E0Q21lH2?qr diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.json b/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.json index 4928145bda572..cad612fbc66d2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.json @@ -35,7 +35,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -55,7 +56,8 @@ "nameParts": ["x_3"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_reduce.proto.bin index 2532c111e38741aad22b2a64a343b5f7ead0777c..89868c4ea5ef60be4b36dc8876717a96b9f08c39 100644 GIT binary patch delta 58 zcmeyu_=Ay+i%Eb{>f=PV>C9155fc}P$!P~^a!E>wb1`ZQu@|v&F;~PJLKv0tMj8wg KFDWr;FaQ9ZOAOKg delta 53 zcmeyt_=S;;i%Eb{>g`0f>C9nLArlvf$!G?tbBRldaxrQPu@|v&F;~PJLKv0tMiZ|n F0RUZB3~T@Z diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.json b/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.json index 2b0fe7911150c..7ae607ccbeb92 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_reflect.proto.bin index 31c6c9bf131502a4f8bf0605071cc4bb73ff18d9..be0973b5020b94141c2aec6d63ddb83ee7a08ab6 100644 GIT binary patch delta 24 gcmcb@c#V;bi%Eb{>ik5u>C6^VW)l~fFljIV08j`90RR91 delta 21 dcmcb{c!iOTi%Eb{>g+_e>C9$QCKDH!002G*1C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.json index 540f1821f50e4..23b90b66e6115 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.json @@ -22,7 +22,8 @@ "literal": { "string": "\\d+" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_count.proto.bin index 3afcfd8c21e7c2a0a5e9079fa846ca8f536d40b4..ec3970a0434ef371da4acdeab473678b50e88bbe 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json index 5d9c7a5b4a5ab..e3b3650c16ba5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.json @@ -26,7 +26,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract.proto.bin index 32ba8b6dcb5e9acd821b92bc5caf541914b08736..9fa17177b5be4ba2f89656afd170af2b1914da8f 100644 GIT binary patch delta 24 gcmX@dc!80Pi%Eb{>f}VW>C8q_1``+PF=;RW08S_c-2eap delta 21 dcmcb>c#e^ci%Eb{>i9&q>C6UFdJ`Au0RTLc1+4%8 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.json index ebe2f581e3de2..04186ace547a0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.json @@ -26,7 +26,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_extract_all_with_regex_group_index.proto.bin index 2cf31e5f75f4f095e6e5bd14d84e1eda4e0e83fb..12d2feefb602c78565a13e4a42c751623cecb4d6 100644 GIT binary patch delta 24 gcmcb@c#V;bi%Eb{>ik5u>C6^VW)l~fFljIV08j`90RR91 delta 21 dcmcb{c!iOTi%Eb{>g+_e>C9$QCKDH!002G*1f}VW>C8q_1``+PF=;RW08S_c-2eap delta 21 dcmcb>c#e^ci%Eb{>i9&q>C6UFdJ`Au0RTLc1+4%8 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.json index cb44dda5ba2c2..69d171eca7f55 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.json @@ -26,7 +26,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_with_regex_group_index.proto.bin index 55cc77eb3cd1f58e24bd4012bd40075e95e73ea6..32649c83ecc75268ef63761b3238c1974c974b3a 100644 GIT binary patch delta 24 gcmX@jc$Sfki%Eb{>exiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.json index eeab13abaa6da..e5ee8f177efdd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.json @@ -22,7 +22,8 @@ "literal": { "string": "\\d+(a|b|m)" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_instr_without_regex_group_index.proto.bin index 3aee655d92c652950b0a924aa26e70728d57e79b..791af5111d72d4fcd77d3cdc747a9b8110ca16f8 100644 GIT binary patch delta 24 gcmX@gc$|@qi%Eb{>cB*{>CBo^>Jt~JGHEaX083Z}rvLx| delta 21 dcmX@kc$ATii%Eb{YTrb*>CEaCCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json index 83dd7a8569fd4..b6a237881c400 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.json @@ -26,7 +26,8 @@ "literal": { "string": "XXX" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_replace.proto.bin index b7d3fde25cf85f9c218f6bdb998a62d239b8c359..6d7dd2cb762f6cf97f4652b19c2740023f935ffa 100644 GIT binary patch delta 24 gcmX@jc$Sfki%Eb{>exiK>CF04x)T>@GifjY08Kar%K!iX delta 21 dcmX@hc$$%oi%Eb{>c~X4>CC!P+7lOO0{}bO1)Bf> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.json index 067652959a94f..0e0c6ea3c4ff1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.json @@ -22,7 +22,8 @@ "literal": { "string": "\\d{2}(a|b|m)" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regexp_substr.proto.bin index 43b987c612cd9d66bc1ae3f96c9f7b9ce6a92ac9..5c629e755a99bcb5b75bb4cd95e07a890a52c96d 100644 GIT binary patch delta 24 gcmX@Xc#4sYi%Eb{>hMIi>C8G(S`!y&FljIV08B^)xc~qF delta 21 dcmX@bc!H6Qi%Eb{>fl7S>C9SE8WR_2002851&IIv diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.json index 4fdc9b035d764..0c220a9401193 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgx.proto.bin index 5771d141728ad0867b80f648a4ceb5cebe13a1a2..e71110d6a35115edf7ff9218470ccf66aa6e135c 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.json index af225fdf5a895..a3ce82193c43a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_avgy.proto.bin index 0a6dcf0106ac7792b74aab9cad992e27a333b808..8c9084a77a0fb210c19e18d145095fe65b2fd65c 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.json index 510fc78140a6e..cc51b8bd0a10e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_count.proto.bin index b1eff9f4d0329dfbdf246ee863af64ec9526f6f2..325137df60e1b5fc2f57d467628d19fc2f272ce9 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.json index 9f88c6ad41268..7d6d482dd2430 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_r2.proto.bin index 0011348d3880ab35f67644ee0face36ab8c27b2c..aca6143f423a6367218237502c2aaace669d6300 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.json index 9503b2c6feff3..7d93ecaf46afb 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_slope.proto.bin index 69c918a7861f283b1eaa254a53c5cf8eacc4c04b..ccfe35dbe4485a3e11138f7e95408e57106bfaf1 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.json index 459deaa391e8d..4c35e57128935 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_sxy.proto.bin index db51c0bc32a79914b0c8284e86e0ea0c88e4a0c3..8d7b682aef3a2ff24f0a16d7a64e1ca9cd52389b 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.json b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.json index 877fbc3aa7c51..624dfa2bf855d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_regr_syy.proto.bin index 6452b277a6e27952675b431114646c309a4efd00..0bf9aa5bcf2630641dcd856d13dfe631389c3963 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_replace.json b/sql/connect/common/src/test/resources/query-tests/queries/function_replace.json index 2f6df6833f368..730207dc6e7d3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_replace.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_replace.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_replace.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_replace.proto.bin index 0564f7ed575838463bdc519dc97e6f5d7d0af71d..a1a579201359513a6510edff39328f2d13e2f831 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.json b/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.json index 2e91450552c19..4b4039e16e5b1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_replace_with_specified_string.proto.bin index 136a6b31821afadee94ded0fa02047edfd8e60c1..8aea29852927204551aceac710aef13ed8d7366c 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.json b/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.json index 93869adfbedca..3f028731c409e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "e" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_reverse.proto.bin index dd7f2d5de513d2214b4af8a3da60d3bde5e13425..f03f20dbba7f0426fd0acd40f2f29e5f683d5f9f 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=csb#4}#i>H9Ld;x@sTvFbmo*2W delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l7xsb#4}#i>H9Ld;x@sQ`yp2XO!Z diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_right.json b/sql/connect/common/src/test/resources/query-tests/queries/function_right.json index 843f5be44a650..966e92157ed94 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_right.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_right.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_right.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_right.proto.bin index b8d0156c981328665801eba345d8397cdc7bdc63..d48b49b52e79e3b5bfe270d66f8562fd4fd172cf 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rint.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rint.json index ea5bcebf81d72..af368a5694875 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rint.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rint.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rint.proto.bin index bd47adc8476fa4112fbd551a60c7bc87a8f627a8..3f8c7c35ceec57d6f47435748bb4c5ff3b4b268d 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(WnRz8btU}COj7b^{0Cw00N&o-= delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0rnRz8btU}COj7b1#zy=xs diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.json index fe8480a0800d1..ec6188eb31ac6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.json @@ -22,7 +22,8 @@ "literal": { "string": "[a-z]+b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rlike.proto.bin index 79bbbe92c7fdbeae0cdd417664f788392dfa22d8..028162c3ddcab0a238cb2141c11c4fa01259e035 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_round.json b/sql/connect/common/src/test/resources/query-tests/queries/function_round.json index 585a0befb224d..d42711c424c46 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_round.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_round.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_round.proto.bin index 8625ccb1a58f147413a74351703ba80943b54f20..40e173d9df4a030715e5cc92cfdc555359de2301 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.json b/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.json index 3d5ac8afe3db3..9972a7e942c96 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.json @@ -15,7 +15,8 @@ "window": { "windowFunction": { "unresolvedFunction": { - "functionName": "row_number" + "functionName": "row_number", + "isInternal": false } }, "partitionSpec": [{ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_row_number.proto.bin index 90b4fcb27d3f1f88a748356537315a2ca4b01b9b..4368883ca2e36355c6fbe9a22cd0e22988db7c6e 100644 GIT binary patch delta 39 vcmX@gc$|@qi%Eb{>cB*{>7trZ>RbX+d|X^b`Q`C>rMXF|MH&nfH;4iNz{m=X delta 55 zcmX@kc$ATii%Eb{YTrb*=^E-$s$6_hJX~Bw`Q`C>rMXF|MMA7X%v_9#KtVPk7A~gD J6ahvFMgU?B3jY8A diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.json index d9b78a0cfd7a9..8c9aaf8242e81 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.json @@ -26,7 +26,8 @@ "literal": { "string": "-" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rpad.proto.bin index d4c355afee0b76805761462a2fdf3c35b06f1a9a..8f945f6329135af7848b65ca5eb3ff02d899d6b4 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWF>B2%%0$i*`B}JLJLaai}T#V@&3;=yHi%Eb{YWYOA>4E}Md|a$WB}JLJLaai}T#V@ebAJYx diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json b/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json index 0ac2401f9eacf..a41ad58bb73ba 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_rtrim_with_pattern.proto.bin index 1332f5b330000a3ba6184cdf6fdab2668864d855..6755240b5167206472ec391bffd567cc9f9a5713 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json index 6df6438a1a9ca..0e6f8f1425d6e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.json @@ -29,9 +29,11 @@ "literal": { "string": "|" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_csv.proto.bin index 99475ddf30d117c90f15146f7b028ea57bae342b..6ee33f3a9e98675d6ef2580c11d30e1d8e206fc2 100644 GIT binary patch delta 50 zcmcc2c%6}ri%Eb{>cT{}>CBc=<`Wl4%ScLzb1~;876`F(v1TzBrxpmYaxrHy)@U$j GFaQ89!V0PY delta 46 zcmcc4c$txni%Eb{>dZv8>CC25#uFDvONdK}axv#776`F(v1TzBrxpmYaxrHy)&Kwk CG73fj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json index 06110d326e1ef..5760bbfc038dc 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.json @@ -18,7 +18,8 @@ "literal": { "string": "[{\"col\":01}]" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json.proto.bin index c4ca00e6292620651b770b29151445f6f308602a..f1934d780d8fde7ce8539050084b0423499029cb 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json index ab05ffa940c50..bca6b670d8b5d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.json @@ -29,9 +29,11 @@ "literal": { "string": "true" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_json_with_options.proto.bin index 482485501dd3738db934c2e0c872f6398ee8a021..7b125c550aef499da4bac1a5f9af65ccaec0de8b 100644 GIT binary patch delta 33 pcmey&_??lBi%Eb{>cd2~>CBN*;S(1qGaE|jPdui}qQRiS005(@2)+OS delta 28 kcmey)_?eN7i%Eb{>di#9>CB;0!4nrKGwVy~PCTXy0C(RAS^xk5 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.json b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.json index c4ea467bc1a24..c0ab18fd4b4b4 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.json @@ -21,9 +21,11 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant.proto.bin index 0971460bf411256b4aab41938947f9375ef3f775..ecea29dcd41ace0b6c7769fe67f9cc5b40969ce7 100644 GIT binary patch delta 48 zcmX@bc#e^ci%Eb{>i9&q>C6UFdJ`9jNQp~{a&Z+T78R$)XBFq?39$+>b1|lCFlaCU E04H(_BLDyZ delta 44 zcmX@dc#4sYi%Eb{>hMIi>C8G(S`!zDh>1!Gb8!_U78R$)XBFq?39$+>b1|j^0QnaR AzW@LL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.json b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.json index 19bf62f70b20f..c31f58aa3e320 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.json @@ -21,9 +21,11 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_schema_of_variant_agg.proto.bin index 68c872ef0d4d2a7705dd67bd596f7a69d48d1e71..a6a3f5bccd26b8fc0f527dfb3388ea78448e880e 100644 GIT binary patch delta 48 zcmX@dc!`mXi%Eb{>hwgm>C7flMiUoENQp~{a&Z+T78R$)XBFq?39$+>b1|lCFlaCU E04nkeI{*Lx delta 44 zcmcb_c#e^ci%Eb{>i9&q>C6UFdJ`8&h>1!Gb8!_U78R$)XBFq?39$+>b1|j^0Q@%! A*8l(j diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sec.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sec.json index 1cab2239755ca..b54347f1488ab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sec.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sec.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sec.proto.bin index 8760f57a6d4f014deee31a6e6dd5a56633fa8aa0..f1b4e4f830a0c8a078c30b234d564e3a93cdbbca 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@MsmVgDLd;x@Ng50QY#If@ delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AhsmVgDLd;x@NdRCR1(5&% diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_second.json b/sql/connect/common/src/test/resources/query-tests/queries/function_second.json index c77a572b88aa0..5040147fa8b2b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_second.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_second.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_second.proto.bin index 193c46e917ba211ad72d7d7e2f7ff813ab6b30a4..196f57b276cad229af9511b2550598c917d82da4 100644 GIT binary patch delta 33 ocmb=ZV&h^GV3aDE$d(`?EG5XrR-BrgpO+%UD#XmiSfarI0C3a>OaK4? delta 31 mcmb=bVB=yEV3aDF$d(`^D8EdEiB3zussd*)-dC958Laai}T#V@&3;?dE2z>wm delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_51sd*)-dC958Laai}T#V@eodO6= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.json index 869e074ccd604..ffb3065d74494 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.json @@ -22,7 +22,8 @@ "literal": { "string": "en" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sentences_with_language.proto.bin index 7514b380a1c82227de4fbdae32194d25a5b9deab..bf003eecdcd402c53788685b4ba453170542c579 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.json index b8bd1b68c9a8f..e9bc5b437502d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.json @@ -22,7 +22,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sequence.proto.bin index 36f1980f4ec2bcdb1ce6e588e53b1552d426afb1..190d2a8225059708e37a08fec175ec56c0cfa303 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.json b/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.json index 07afa4a77c1b9..03ed1976a708b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "session_user" + "functionName": "session_user", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_session_user.proto.bin index 948e3eeed60ac2c64d3a60af42f0055440e2fecd..62904540ce677c59ca5b1ba0f0fa682ac01ca696 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$e=Bsl~;a`FZiB#i>Oa3;>X^2_ygj delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|W)msl~;a`FZiB#i>OAeccH3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.json b/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.json index 92995656bd265..6178d6a73ab77 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.json @@ -22,7 +22,8 @@ "literal": { "string": "10 minutes" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_session_window.proto.bin index 364ecdf2aaa2834124540b0695fd6a9aa18187f5..b669a186b2e2b64e5ac85da02276a8cef597eabf 100644 GIT binary patch delta 24 fcmbQpIGK@+i%Eb{s%IiwBD1!X=EN*@CJhDvJMaXh delta 21 ccmbQtIFXT!i%Eb{s%s)!BD1EH`ot`C04@UrbpQYW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sha.json index 57c5cb5bbd270..2385eb652cc15 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sha.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sha.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sha.proto.bin index e99760e49222dab57f510c1123530f1f56d48ade..66d9291a141018b87b8cb3791d895764a9772b9f 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@M8HqxyLd;x@=^6|GY&HeQ delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3Ah8HqxyLd;x@=>T9X1(yH- diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.json index ce5014ac2f7e6..b19270ee3fdcb 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.json @@ -26,7 +26,8 @@ } } } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sha1.proto.bin index 3fdfdb2a072de772dd9cc541eb00c27b32993a2d..004ef664ee8d8a5168b21ac9fd0b526591fab295 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.json index 5278d604e97b9..ed90b3b939ee6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.json @@ -30,7 +30,8 @@ "literal": { "integer": 512 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sha2.proto.bin index 20a0ee1082ae20efeecf6f1a9a3472519e8f4b20..bf16ad9677137ec3cd04fe4bb831aab98f37d29a 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json index 12decd300ab03..c02f85d5d56b1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftleft.proto.bin index 94bfbc99fce2d28deecdb1a14b0cf8f25f7c5f02..5ebdbbc9996eeae89b0d4450b93eb439ecc3bbab 100644 GIT binary patch delta 24 gcmdnNxQmgEi%Eb{YV$<4>C7@xQWFC9465)&6l001{X1t|ak diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.json b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.json index c2295c4abaaa2..eabafb977393a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftright.proto.bin index 910d12f50d6a92af7273af8d8163b5ade7319d02..6a089c2ffa344e2a1ac728031259d2ef8ee86088 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json index 875e26a5a5652..4b32899df264a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.json @@ -22,7 +22,8 @@ "literal": { "integer": 2 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_shiftrightunsigned.proto.bin index aba9c425dca96ecb8321a9560f1fefbe424f0767..d732f7244aa0e4d5327789c1946bc2706bf0a0e6 100644 GIT binary patch delta 24 gcmX@ec$krmi%Eb{YR^Qr>CCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sign.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sign.json index 34451969078b0..3491a453f6b68 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sign.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sign.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sign.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sign.proto.bin index ff866c97303edefe0216166590c81a175aa505f1..35083f8b9a89a231b7cab25fc2fd58cff89d1849 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)>ndx~#tU}COj7b^{0CrpkJ^%m! delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2Bndx~#tU}COj7b1#Tm}vR diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_signum.json b/sql/connect/common/src/test/resources/query-tests/queries/function_signum.json index bcf6ad7eb174d..02ab0e364fd10 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_signum.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_signum.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_signum.proto.bin index af52abfb7f25b34a0a6afd5283a80270576c357d..65c838e4085401925303a2433fc481e8fe0e0258 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RCHndy0@xk9W$%v_8~8Vmr7?*}db delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J6sndy0@xk9W$%v_8~0DS)j{r~^~ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sin.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sin.json index cb5b0da073456..a6be1adb3249d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sin.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sin.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sin.proto.bin index a63f574fa59cbd6d1b3bfae7bdaf2f93fb552e49..1f746a3ab76f99957a5551aee00e42ee0506d07b 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@MnR!C2Ld;x@Ng50QY_tW? delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AhnR!C2Ld;x@NdRD&1)u-` diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.json index e0f46b428611e..c84ac26b64222 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sinh.proto.bin index 2f17ab02a6d9461a0da9809d9267408612e89ead..545a0749d7973d11255708bc827adb8637cc15aa 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)>nRyvPtU}COj7b^{0CsH#KL7v# delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2BnRyvPtU}COj7b1#YX%Ph diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_size.json b/sql/connect/common/src/test/resources/query-tests/queries/function_size.json index 37c9cd1ac1ba7..97a996e23a790 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_size.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_size.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "f" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_size.proto.bin index a8ae600a3dd7a33d02b004f5983ed18a1bd5173f..4f80765b81564c35c95c83da31e6a2e6e0b66f7d 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)>nN_JmtU}COjAq&xyj(2BnN_JmtU}COjA;OA!v-7x diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.json b/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.json index 4b14c8d5ca79c..4ed304e15c67e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_skewness.proto.bin index 889f96b2d2a399508b8527d7dd5c60365c053eef..73a8e1e4d19980494d093497228604acbe821a39 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx81*{S7usl~-YtU}COjENcy0HbCI8vpCAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFAVdG*FV3e|($fhYOA|=GdQkJABlbEi-003Lr1=|1s delta 32 ncmZ>9V&h^GV3e|%$fhYQBqhMbQkJABlb8+wPlW}m diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.json index b42bede5cd172..24ffc7c2d387e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.json @@ -22,7 +22,8 @@ "literal": { "boolean": true } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sort_array.proto.bin index 994048af2afc4d4fc8461c1ef01ad7228b266b01..0414c104be55632b5bb010550ee945babd97963c 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json b/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json index 851745b32ebe0..0e428e3e199d2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "spark_partition_id" + "functionName": "spark_partition_id", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_spark_partition_id.proto.bin index df99cd64e72032a1ac85612b9ece9a3cc6fed5fb..843ca4273b7d8640e259be620ce85b8dffc0230e 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split.json b/sql/connect/common/src/test/resources/query-tests/queries/function_split.json index 001d44dcaaf6e..a00e18d77628c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_split.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_split.json @@ -22,7 +22,8 @@ "literal": { "string": ";" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_split.proto.bin index cab0bde7b6da20c0d8238f1cb9e217393ead5e77..d0da01bba86dd61a75315274a0e0963ee3225e35 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.json b/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.json index 81ced1555d3e4..a9c6a3ec9e2d0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_split_part.proto.bin index 2c1948f20dc223653a90433e7b0a7f0bd20ef39e..fde88be6654b0c237bc50cfd2646fd4fb451b2ea 100644 GIT binary patch delta 24 gcmX@ec$krmi%Eb{YR^Qr>CCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.json b/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.json index 98ef0e54e6211..5cdf413daf98c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_split_using_columns.proto.bin index a87702f83d1bdcf3a62e9e94527f9518ea90fc16..e5a712f11a74eaf15f3e739f6f3633ddb00224d0 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%Eb{YQsdf>CBQ+;u9B$GHEaX07h5^M*si- delta 21 dcmdnaxRsHOi%Eb{YTZP(>CEC%q7xU00suCJ1s4DS diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json index 45a7588838ff8..0e2f5ac83e77c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.json @@ -26,7 +26,8 @@ "literal": { "integer": 10 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit.proto.bin index 497297fad8715dc9d59821fdb14a76fb3cfb9a59..adc0a101eaf1912f03eca1c10da06e0c80c4b183 100644 GIT binary patch delta 24 gcmdnZxR;TQi%Eb{YTHD%>CEy{vJ)3bGifjY07y6mYXATM delta 21 dcmdnXxSNrUi%Eb{YRg2n>CCcH(i0a*0{}Pq1v>x$ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.json b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.json index 138f9d70b2c85..73b7355de15e7 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_split_with_limit_using_columns.proto.bin index 04e24be40e9d8edf9cb520948c3d563273552fe3..a25da7d145181f06d7d57cfadfbb91e96e5f2b8a 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.json index f9a2b76520c13..6ca6327142558 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sqrt.proto.bin index e98e3bdfdb665b4b57c9920605887149b127d9d1..8b2ed11bd0e2827b6ee5a946105168d980a4ef63 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G)>g+(PotU}COj7b^{0C#Z)R{#J2 delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(2Bg+(PotU}COj7b1$KL#fN diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stack.json b/sql/connect/common/src/test/resources/query-tests/queries/function_stack.json index 14865c72df228..f714739b185cf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_stack.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_stack.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stack.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_stack.proto.bin index 5e5e12478d682ef9ac878300e2f5cb6da71b33da..f578d0d2bb9523e7243dfff3b39a6baec44cee09 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.json b/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.json index ce2b0ac658c4a..4fb08d9de4760 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_startswith.proto.bin index 2f09e8095f5a020ad51065c2f49f74070d884de6..f46d166a7c554a382e3fea239e9457bb79c98f8e 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFyHi%Eb{YWYOA>4E}Md|b@MB`HFzLd;x@i5d(5Y{mu9 delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AhB`HFzLd;x@i2z`#1)u-` diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.json b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.json index 1403817886ca0..d27469b26a7fe 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev.proto.bin index 8d214eea8e74e4e8a8bc6674175c755ce456c838..b24c06af4fd0421112631d07dafd035197bb7eed 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RCHB`GPXWkRe%%v_9#8Vmr7kq0CI delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J6sB`GPXWkRe%%v_9#0DQFu^Z)<= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json index 35e3a08b219f8..038b6e1fbb70a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_pop.proto.bin index b679f55014f973019dc67de46232811ebd1b8f26..e311b7d0311e898d12f8416101616666a12f289b 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh|XB`GPXW$^|11wyPs%v_9#8VmrtX$bBB delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzU%B`GPXW$^|11wyPs%v_9#0I1mrz5oCK diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json index 17cd0fd5e5976..3f7b829e4821f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_stddev_samp.proto.bin index 9f22eba5e39aac2ed26e3c6bc3518538dc24e1b8..7f888c1c07d77239e6b1dc46cc39cb8327c0cfd8 100644 GIT binary patch delta 24 gcmdnOxQ&sGi%Eb{YW+mE>C6&RViOmLFljIV07eJ}K>z>% delta 21 dcmdnSxP_68i%Eb{YVAa}>C9qMA`=&g001^e1rY!M diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.json b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.json index 2cfd095f8fe62..3c640499aba54 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map.proto.bin index 9732a829513a81c9c4b1df033380643046df181a..7c125c71e515003ed82fcaf7d4a3bd18c96a4d7e 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh|XB}MTi`SH1l1wyPs%v_A=8Vmrt+6eRj delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzU%B}MTi`SH1l1wyPs%v_A=0I5C*!~g&Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.json b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.json index 228c939a43ef2..c3b52657efd2e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.json @@ -22,7 +22,8 @@ "literal": { "string": "," } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_str_to_map_with_pair_and_keyValue_delimiter.proto.bin index 069c15db9af76092c6fb6a1ec88c0fe119a4ca4d..e65abe3472b91df2e8736559a859446410ffbd7f 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFCCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_struct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_struct.json index ba950215a2591..f88910dc3f494 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_struct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_struct.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_struct.proto.bin index 079c2be3c52e505693338bad9f46cca81c24a101..90e12eb597175f988fcc19dded1954c5f9967834 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substr.json b/sql/connect/common/src/test/resources/query-tests/queries/function_substr.json index ef6d225821c37..510d501b5c9b0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_substr.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_substr.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substr.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_substr.proto.bin index 934201c4333816d639b30ac195bcd0840fb18c11..6b0871916a8a24c7c44c4946f6afed3afd354639 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.json b/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.json index d8492899d69bc..b09ef5ed2723e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_substr_with_len.proto.bin index 0fab03c0250615f0a4d1ff36165e01ae63344e26..e5a3de2dc6c58b859ea7ccb37e6c37d5c98ac8f4 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substring.json b/sql/connect/common/src/test/resources/query-tests/queries/function_substring.json index 84a70cf1c0236..5590cd2660922 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_substring.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_substring.json @@ -26,7 +26,8 @@ "literal": { "integer": 5 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_substring.proto.bin index d302cd95c7434095bd8ebcf2511786058de52274..bdb3a13e5d9c3657c6ba43b1c27c5898f42418b0 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.json b/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.json index dc81d925957cd..a5396bb478197 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.json @@ -26,7 +26,8 @@ "literal": { "integer": 5 } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_substring_index.proto.bin index 192bb2e300dc3ce1057df483d260d8891fc6bfc2..9ac474c32e1bd7e4b6de2ffe67728126f6266a92 100644 GIT binary patch delta 24 gcmX@gc$|@qi%Eb{>cB*{>CBo^>Jt~JGHEaX083Z}rvLx| delta 21 dcmX@kc$ATii%Eb{YTrb*>CEaC7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sum.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sum.json index e9526a20b67fb..28b4ea5bbe856 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sum.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sum.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sum.proto.bin index 0e347bbc0a167348c7a86bd0afea032590632b94..05c69d6f9402965ca3578a922581c31189af03c9 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b@MrMW_^Ld;x@i5d(5Z7T)Z delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY3AhrMW_^Ld;x@i2z{x1*!l5 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json b/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json index 4614cf99ad3a6..0de8f3d36c22f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.json @@ -19,7 +19,8 @@ "unparsedIdentifier": "a" } }], - "isDistinct": true + "isDistinct": true, + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_sum_distinct.proto.bin index b4cf704391a4d3bba370b8b4388164b640c4a895..6c345201d8eea3093b39bbe833f30ec1302bb810 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$j|+rMW_^Ld;x@i4u$&3;=o-1^)m5 delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|b@MrMW_^Ld;x@i4u$eZHxua diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_tan.json b/sql/connect/common/src/test/resources/query-tests/queries/function_tan.json index ead160a7e3ac2..38ca851765599 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_tan.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_tan.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_tan.proto.bin index d674dc033b2cd91aa773bd4ea51004eb4d750980..ba28964c9befbd4bc22aee2f10d9dd65b2199cdd 100644 GIT binary patch delta 32 ncmZ3(xQ>yHi%Eb{YWYOA>4E}Md|b>WiFrb-Ld;x@Ng50QY-k0^ delta 30 lcmZ3-xQ3C9i%Eb{YUxC_>HK_BJY38riFrb-Ld;x@NdRD41(^T< diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.json b/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.json index bcd12c664427e..e9e3996bc5aaa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_tanh.proto.bin index 21c28c3ef88e6f16fd50fdd90073b53f6f170169..d635c5020a53fccd66cd51b50d7455a256b7df44 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(0iFp}9tU}COj7b^{0CpM%H~;_u delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0LiFp}9tU}COj7b1#CI$!q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.json b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.json index 8fd71bb36d85e..c23a5c3bfa129 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": true } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_add.proto.bin index 5ab8ec531e07319ce2f27de2b71dbbe576133650..142672a0929e80b15ac2ff83e824983cfd5ce61e 100644 GIT binary patch delta 24 fcmbQhIEj&si%Eb{s(T_^BD0p1#>6Z&CJjaaJD~)i delta 21 ccmbQlIDwIki%Eb{s&gV+BD02++Qck104>D?ZvX%Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.json b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.json index 635cbb45460e6..c779b0936dc63 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "t" } - }] + }], + "isInternal": true } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_diff.proto.bin index 3a81fd8b318c012b20929cee4b25ee4c558d3784..c2053f46f6a559ffc13555d0a22affccd0018e26 100644 GIT binary patch delta 24 fcmbQpIGK@+i%Eb{s%IiwBD1!X=EN*@CJjaaJMjdj delta 21 ccmbQtIFXT!i%Eb{s%s)!BD1EH`ot`C04@UrbpQYW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.json b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.json index e43aa6d7115bd..985a23c536e1c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "x" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_micros.proto.bin index c8ca8eedef3c00065c9dd3e880b10d416c5aabcd..f17fc48d4418bd179895609c2b1c6ba6bade3b74 100644 GIT binary patch delta 45 zcmZo-Y++>MViI7Ks+-7`ASWv&%_UHhnVVW%l9*c%pPQLnlwT~wD#XmiSfRlH0Pxfb A+yDRo delta 43 ycmZo+Y+_{NViI7Ks+q`^AR{d$$t6&dnVVW%l9*c%pPQLnlwT~wD#XmiSOEaqI0~!) diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.json b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.json index afcdf42d7b3be..52389f7fe5fab 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "x" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_millis.proto.bin index bbe401c39f3d17d5b748edc3dbac41fbe8ba8fd4..f63af6ecb1fb8553c0583f5e293669286518bcb9 100644 GIT binary patch delta 45 zcmZo-Y++>MViI7Ks+-7`ASWv&%_UHhnVVW%l9*c%pPQMJlUXdpD#XmiSfRlH0Pw#G A*#H0l delta 43 ycmZo+Y+_{NViI7Ks+q`^AR{d$$t6&dnVVW%l9*c%pPQMJlUXdpD#XmiSOEaqDGI3o diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json index e6892d17708b3..526d22229facf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "x" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_timestamp_seconds.proto.bin index 102afbdda90217084d077d73887ce249c9cc07d8..0c7647735eed5078f79b76dc427a1a1bdf360699 100644 GIT binary patch delta 46 zcmZo>Y-MEQViI7Ks-MV~ATK8+!zEafnVVW%l9*c%U!0nppO;cB#45zh#aN-i0094( B3q=3` delta 44 zcmZo=Y-VKRViI7Ks-4J|AS)v!#U)shnVVW%l9*c%U!0nppO;cB#45zh#aICV=pqXh diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.json index 156c3a5b3ca65..fcf1d35f42169 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary.proto.bin index a1da0e6e2eda1c6530ab71dd556dff07f2864915..2b02883e03a30ca7de707c8487894b6fb41968c3 100644 GIT binary patch delta 38 tcmdnQxP_68i%Eb{YVAa}>EdEiB3zs$`SD4ad5J}pLaai}T#V@&3;?cC2z&ql delta 36 rcmdnOxQUUCi%Eb{YV}06>7pW1LR_3B`SD4ad5J}pLaai}T#V@eoUI5< diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.json index 8c78cc6f8b99f..325e92437d515 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.json @@ -22,7 +22,8 @@ "literal": { "string": "utf-8" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_binary_with_format.proto.bin index 2f2364e5abab12c1da5cb62a25cd720f4bce4a0e..5be46a049a53551fff034db322bf166bfcf6727d 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.json index 404a89a87ecb2..117955fa60c3a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.json @@ -22,7 +22,8 @@ "literal": { "string": "$99.99" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_char.proto.bin index 087e212c39f4eb7a04937d4580dc7784f2a2e720..6ed0c2cdde8a98bee181227558c7f2d1bed11793 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.json index 6b3856f5ac0af..cfbce992619b8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.json @@ -29,9 +29,11 @@ "literal": { "string": "|" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_csv.proto.bin index a3017643a330a27d5c3076b01f747391bf604431..318966eb2b58b09822b2b1d6798dec4ae2433df0 100644 GIT binary patch delta 62 zcmX@Xc!rUUi%Eb{>gYta>H2z7I$Ufe`SHocWkRe%%v_8qLXuMAT+F$N1w!mxtXa&( RsRcr;T+CUFH5v>W3;?n341xdv delta 58 zcmX@Zc!H6Qi%Eb{>fl7S>DpRS8eD88`SHocWkRe%%v_8qLgG@QT+F$N1w!mxtXa&( NsRcr;T+CUFH2{4h3?2Xg diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.json index 8b9d50aa578b8..0da88a6158438 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "s" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date.proto.bin index 59178487eef586dd4c6f39069ffa104e98e7ea8c..bfd79f65053c8b846586fe6564035d2ccad90e1e 100644 GIT binary patch delta 34 pcmb=dX5(TKV3aDI$d(`~A|=GdUXmZ5l30=|#45zh#aOJt004U12DShI delta 32 ncmb=fWaDBIV3aDJ$d({1BqhMbUXmZ5l30=|#45zh#aIjgY|#dW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json index 48ae80d1e70ed..1f6250ec0656b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.json @@ -22,7 +22,8 @@ "literal": { "string": "yyyy-MM-dd" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_date_with_format.proto.bin index 2641d660ff69ffb5c55e71aaf6bc2d02db0c140d..8118ff7ee4705c965365cf9a7cec98ca7c69166e 100644 GIT binary patch delta 24 fcmeBT>|tc%ViI7KYMscI$gCozG%-toNrM3ZIlu&a delta 21 ccmeBS>|$i&ViI7KYM#iJ$gCu#FfmI304z}iO8@`> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.json index 7ceeb9d113cd3..0482c5cba9500 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.json @@ -29,9 +29,11 @@ "literal": { "string": "dd/MM/yyyy" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_json.proto.bin index c9461c1aa961ce9899a274983001ca1932d92f42..f3d94b476135a0e705ae65005834020a1ea851d4 100644 GIT binary patch delta 46 zcmcb|_=u5>i%Eb{>h?so>EbR@PF(CI`SDrB`FTRDLd;x@DMETuIuo~Ov1l-8FaQ7* CDhjax delta 41 xcmaFFc#n~di%Eb{>iR^s>EaGjc3kWw`SDrB`FTRDLd;x@DMC6@S`)Wu0RZ0i3QYh2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.json index abb71e80a769c..8df8436bf647d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.json @@ -22,7 +22,8 @@ "literal": { "string": "$99.99" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_number.proto.bin index 189c73553c5dbd014ce9b9c312e939cbfb515247..0d160ed3239c147eed3995d8f202d1f0b95e55c2 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json index 323c57e2ef58a..4e9013a1c0ff2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "s" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp.proto.bin index ec6bd64f9818707092948ade9933db2b7ed384b7..1c8f18761c5c637c598e96ea282ff255ee6cd1ff 100644 GIT binary patch delta 40 vcmbPxPS=| delta 38 tcmZo*tYhP15@3|7n8=nOAuc7##Z!_WUy_-dT3nKtTOhCCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.json index 08cb9c153f77f..6ff1dca87de6a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ltz_with_format.proto.bin index 22fd3d07dfc4374b5bb28b8ac69d5808ead70939..6aebdbd4f866782f26ca66b5c075a4ea8177c365 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.json index 6808047ef2094..53678e814da88 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz.proto.bin index 5cd4cfddbd164b85719e33eeb5e670ef6a5c8663..9a0c00065da258abc6d56d122b8076a8943f6023 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.json index 03e38801bfa56..240c53dd5c31c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_ntz_with_format.proto.bin index 3a5d3dd97020012d08dd25b4a639ecaa5087d375..f38ad9460ff52ab18c3b1351167996e8f5890f43 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json index 30f34528319c7..1988d2fb5a863 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.json @@ -22,7 +22,8 @@ "literal": { "string": "yyyy-MM-dd HH:mm:ss.SSSS" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_timestamp_with_format.proto.bin index 9c2d6d354ca737be165298da205b6770d711f4d6..939f9151de2a4b30c16cb8f40b995346c7fec3f5 100644 GIT binary patch delta 24 fcmbQsIG>S?i%Eb{YQ{viL}qg-(}`KeOd1RTKd=Pt delta 21 ccmbQwIG2%)i%Eb{YT87$L}pVdCAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFCCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json index 015fbb5cf534a..fbd86a28a12f3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.json @@ -22,7 +22,8 @@ "literal": { "string": "-04:00" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_utc_timestamp.proto.bin index b2b65089604a2046f06e5aa783fc47f9234f1029..bfaf5d2af8a9c2d38f928d98b1b380c670954cc6 100644 GIT binary patch delta 24 fcmeBYoXE(=#U#Ke)iseVky%qpePWg>lLi9-J5U6f delta 21 ccmbQp*w4tu#U#Ke)iIGRky%|zbz+t(04;|EX#fBK diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.json b/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.json index 3694a68dc8f5c..732eb4b426dde 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.json @@ -22,7 +22,8 @@ "literal": { "string": "$99.99" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_to_varchar.proto.bin index 005c9ab064c9bf797567303b72dc5fa188b3d97b..b912951a755193e776a0d70dedc64693dabcb01e 100644 GIT binary patch delta 24 gcmdnbc#x5ei%Eb{YS%=z>CDPfiW3*eGifjY07-ZSg8%>k delta 21 dcmX@exSx@Yi%Eb{YR5#j>CB2!@)H-x0{}Sj1ycY3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform.json b/sql/connect/common/src/test/resources/query-tests/queries/function_transform.json index 3ad6fe9435644..c8ec0608b13af 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_transform.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_transform.json @@ -31,14 +31,16 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }, "arguments": [{ "nameParts": ["x_1"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_transform.proto.bin index 266b093f7a99b4b8e87f514db293703c14e36da7..e4efe8865b12439877dd882bb73799103444fd00 100644 GIT binary patch delta 73 zcmcc2c%6}ri%Eb{>cT{}=~k9f=3Ja5MTvREY57IDLaai}T#Ttgib3*RVp1YpjM_r% YMXX%R74e2bEL=dZv8>1L)<#$22wMTvREY57IDLaai}T#Ttg@cd2~>CBN*;S(1~u-gP#acN0uOx&*~q`@G>%Eep}ZwR6)J E09}p?)c^nh delta 47 zcmey)_?eN7i%Eb{>di#9>CB;0!4nrquv-OLaA`=XP28`>C&bFdToG>wqAKH!04|;j AZ2$lO diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.json b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.json index 02aeca229ce5d..6e76f8a0554f9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.json @@ -41,7 +41,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_values.proto.bin index b4a653ff77a5d78a7f6ce63192dfc249a9adcfe3..e24a6320c40b312ce694d6518043ee045033dffd 100644 GIT binary patch delta 24 gcmaFH_=1s*i%Eb{>fuDT>C9eI9upV1F=;RW09B|4Z2$lO delta 21 dcmaFC_>7T_i%Eb{>i$Hw>C7HdZW9-{0RTc{20Z`( diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json index df5e15b44fdd3..49dec8db7da73 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.json @@ -31,7 +31,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -40,7 +41,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_transform_with_index.proto.bin index e502c18dcd9e8fcce046bbe33105a6e9a9bca0fe..30b1901f42f58533299c5ce7bcc34970f3019fdf 100644 GIT binary patch delta 83 zcmcc1_>hr}i%Eb{>efWI=`PMvj$E82MTvREY57IDLaai}T#Ttg+CiFJl2YPajM_r% YMXX%R74e1;MrFK_27?e(1W1AT02(L|s{jB1 delta 79 zcmaFJc$blli%Eb{>e@uM=??Z%wp^SgMTvREY57IDLaai}T#TtgnnCJZ;!>hqjM_r% WMXX%R74e1;MrFK_5R?z3fII-z0T4p~ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_translate.json b/sql/connect/common/src/test/resources/query-tests/queries/function_translate.json index 93d155c2857fb..ad5f98152258e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_translate.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_translate.json @@ -26,7 +26,8 @@ "literal": { "string": "bar" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_translate.proto.bin index 1ce32c8d2843e5d6bfdbb0f5705b5a39df061472..ec9c556cfef09e037a1ffa22af5c9e7900fa3194 100644 GIT binary patch delta 24 gcmX@Yc#M&ai%Eb{YX3yG>C75ZY7-ZzFljIV080o3p#T5? delta 21 dcmX@cc!ZISi%Eb{YVSn0>C9?UDiar|0025C1#tiX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trim.json b/sql/connect/common/src/test/resources/query-tests/queries/function_trim.json index d2700174bca3d..a7925c2c7b5d2 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_trim.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_trim.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_trim.proto.bin index d5f4f21510fc601a41201a35bb738c06a4b6e071..0ea9051f338374386496542ae13480a64090ddb8 100644 GIT binary patch delta 33 ocmZ3>xSo-Xi%Eb{YQ;ph=|X~1{9G(0MVYxmtU}COjOiK-0Cw;OP5=M^ delta 31 mcmZ3_xR#NPi%Eb{YS~1#=>q&xyj(0LMVYxmtU}COjOhSr&;}j= diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json b/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json index fc3281c921531..cb566a6b98dfe 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_trim_with_pattern.proto.bin index 2136b55656212e31b809e7cade085e79303bd585..4423ab1a02a3744d67b23315ba6c36cb02154fe5 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.json b/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.json index 4c596cd863261..01e528241eedd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.json @@ -22,7 +22,8 @@ "literal": { "string": "mm" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_trunc.proto.bin index cdcee95af63443c9fa4d4783d2d4339c086e931f..acfd2bb94483b3cf671dcebfb61ff76a7e39cbc6 100644 GIT binary patch delta 41 wcmb<#WMt!F5@3|7p2(IUEhQzv#adESnwKoZD#Xmim?Ff+#gfI8o2$V90Hgi~f&c&j delta 39 ucmZoCDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.json index 80e10f4786a81..fd2ba4bd66bdf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt.proto.bin index c2a477e5320c7d8017d858ecf86b149cf27d8552..cc94446a39c8d5ab5573c23707f8a451ee7fb214 100644 GIT binary patch delta 24 gcmdnbc#x5ei%Eb{YS%=z>CDPfiW3*eGifjY07-ZSg8%>k delta 21 dcmX@exSx@Yi%Eb{YR5#j>CB2!@)H-x0{}Sj1ycY3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.json index cc4ea4bfe5fb9..360742eaf0b69 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode.proto.bin index 22919795e3e6a9f2b29899a6159e4fa091e650b1..b1fde39e2b632f4a7e74ce3c109fde0aad8f6a38 100644 GIT binary patch delta 24 gcmX@cc!H6Qi%Eb{>fl7S>C9SE8WR_&F=;RW086L^tpET3 delta 21 dcmX@Xc#M&ai%Eb{YX3yG>C75ZY7-Zz0RTFq1$_Vj diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.json index 1f1fc777959a2..f02b625bd9e6a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding.proto.bin index b16d49e2428a273c6ce58a6066a206bde283f5a2..1774a4dfaba1902b84a1cdf997ebb42994f945a6 100644 GIT binary patch delta 24 gcmX@hc%G4si%Eb{>cm90>CA>w`V$xEGHEaX08Q8h*8l(j delta 21 dcmX@lc$Sfki%Eb{>exiK>CF04x)T@Z0suT!1*ZT2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.json index b7e7cd41bda8f..b0c434231378b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.json @@ -34,7 +34,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_aes_decrypt_with_mode_padding_aad.proto.bin index d406961d5ccfc8bc3ec65e4632bc24b9a46e714d..6f75571c79ff3d869da5b5c73f8b7d951673bbac 100644 GIT binary patch delta 24 gcmcb@c#V;bi%Eb{>ik5u>C6^VW)l~fFljIV08j`90RR91 delta 21 dcmcb{c!iOTi%Eb{>g+_e>C9$QCKDH!002G*17pW1LR{=6MV0Z1W$8k!Ld;x@i5d(5mr)0u delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l7RMV0Z1W$8k!Ld;x@i2#Rt2WCAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFC8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.json index c4e5bc2f415ee..e200f3fa9d278 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_element_at_map.proto.bin index 2f6c54f2fa5ec56e784fb125c91066dbb20a1686..1f8fdb10899a629ced4928bda7294552cf779f40 100644 GIT binary patch delta 24 gcmdnTcz}_Oi%Eb{YUf0@>C8$}3KJK|F=;RW07)nXeEC6gJauXNG0RT9&1x)|| diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.json index a7a2348496040..4c3bafee572d1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years.proto.bin index d459b6e8ec6773a13ba1858bf9dc1d3ad796a7cc..acb1952d621d78fd1b482704a485b71a04b70355 100644 GIT binary patch delta 24 gcmdnRxQCIAi%Eb{YU@O{>CAFcG7}d_F=;RW07vKrWdHyG delta 21 dcmdnPxQmgEi%Eb{YV$<4>C7@xQWFCCE9$`cnTGHEaX07@7Ij{pDw delta 21 dcmX@ic#x5ei%Eb{YS%=z>CDPfiW3(o0suK}1z!LF diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.json index a6ac2f27e3dc5..672acdf6b2d3d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks.proto.bin index cecfca97f7e2077f087578d64820710d534a0ddb..555eb56ff043853d1893c120dcb413fbee6ed352 100644 GIT binary patch delta 24 gcmX@Xc#4sYi%Eb{>hMIi>C8G(S`!y&FljIV08B^)xc~qF delta 21 dcmX@bc!H6Qi%Eb{>fl7S>C9SE8WR_2002851&IIv diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.json index c9d4f1d4d2f1f..70782ac30ea49 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days.proto.bin index 423172405c397b95ce801753e6133e856e3f1fa2..b9d6de2d00662aa6d00a6a551f155415f97f5e53 100644 GIT binary patch delta 24 gcmX@lc#)Bfi%Eb{>eNKG>CDDbh7%X)GifjY08V%X;{X5v delta 21 dcmcb}c%G4si%Eb{>cm90>CA>w`V$xE0{}eH1+xGE diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.json index 7f2a42f01db45..fda1b3ea21a36 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.json @@ -34,7 +34,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours.proto.bin index 71259b402aa5106801e8f051b96221315ab37910..3f477ddeab7dbf0284e9a2e040afef351b7a18ee 100644 GIT binary patch delta 24 gcmcb{c!QCRi%Eb{>f%JU>C9GA784hkF=;RW08pp~4FCWD delta 21 dcmcb?c#V;bi%Eb{>ik5u>C6^VW)l~f0RTRO1>FDu diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.json index 35ab05a90b3cd..b2db53de2c58e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins.proto.bin index f8cf29d15aabfd6a66aca680522b72e5ed675850..72dd0ffd9d539d5eff975529f3ed4d601f909028 100644 GIT binary patch delta 24 gcmcc1c%PAti%Eb{>c&L2>CBE&_7fM_GHEaX08-cnHvj+t delta 21 dcmcc5c$blli%Eb{>e@uM>CE<0wi6fF0sufY1_uBD diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.json index 2f9c1d019359b..5a1389ff2665e 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.json @@ -42,7 +42,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_interval_years_months_weeks_days_hours_mins_secs.proto.bin index d7343a059b53d4f5c9241fb0c693187be0de57fc..90bf07eb4e9e3245b482e372111560dfcac5e7dc 100644 GIT binary patch delta 24 gcmaFD_>7T_i%Eb{>i$Hw>C7HdZW9-{FljIV096PEVE_OC delta 21 dcmaFH_=J&-i%Eb{>h46g>CA3YE)y5H002Sf1~C8t diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.json index 179f6e06988fc..36559a50d7aa4 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.json @@ -42,7 +42,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_with_timezone.proto.bin index d0c60ba1c7bf8433f58e3c7372cd923b2ce801d6..34dcc6bf460923e41ca2578270225d95c5f59757 100644 GIT binary patch delta 24 gcmaFK_?nT8i%Eb{>d8d5>CC=T-V+yiGHEaX09Kd=e*gdg delta 21 dcmaFO_>z&0i%Eb{>d{2D>CE0zo)Z^%0suoD22TJ0 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.json index 29aa2096c2273..36e121b976dbf 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_ltz_without_timezone.proto.bin index 9caf6f6ba5285aea06bda2ed3462350672e4fe93..c0eaf743e46fedee8106f7d65913495dc4b1de15 100644 GIT binary patch delta 24 gcmaFF_=J&-i%Eb{>h46g>CA3YE)y3xF=;RW090rORR910 delta 21 dcmaFD_=u5>i%Eb{>h?so>C7%tP7@b60RTa31|h46g>CA3YE)y3xF=;RW090rORR910 delta 21 dcmaFD_=u5>i%Eb{>h?so>C7%tP7@b60RTa31|cK>|>CB!|?h_ZdGHEaX099B9X8-^I delta 21 dcmaFP_>_^2i%Eb{>fS`Q>CEm@t`iry0sulK1~&iz diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.json index 39ce728a38862..549071a8a964c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.json @@ -38,7 +38,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_make_timestamp_without_timezone.proto.bin index 74918d42f89c66313832a6e42c4ea00261fd071d..2d8c06fb5a2a5c2882eb03e9c1116c828d8627fe 100644 GIT binary patch delta 24 gcmcb|_<)g(i%Eb{>gGhY>C8@24igvHF=;RW08=OiJpcdz delta 21 dcmaFBc#n~di%Eb{>iR^s>C6sNb`uxa0RTXA1`PlJ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.json index df22654c82031..0838efb2c1eb3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_multiply.proto.bin index 8912423235e0b2e8644e2bf856d34bdac8a764e2..703d897792bcb3acf81734093cc7ff681b760c35 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.json index 91177eb4a5857..fca2a95a83ab8 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_json.proto.bin index cc1f159cfd78c933dd36463f9013c6f651bd033c..a97f0801944be8be57765aa07b3ea9b64737a9ab 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.json index b9603d5af2634..0b4e011421d6a 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url.proto.bin index 696c4ddde519c115ddeae5d780a96fd196c66150..080be30a46d23f47cd9d2ede0bf485863b2e5aa3 100644 GIT binary patch delta 24 gcmdnXxSx@Yi%Eb{YR5#j>CB2!@)H-xGHEaX07%#ccK`qY delta 21 dcmdnbxR;TQi%Eb{YTHD%>CEy{vJ)4`0suI51xEk? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.json index 137ed4bd9bc80..d173704b73354 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_parse_url_with_key.proto.bin index f4a13872e3c8f113b5c362cf5bdf13b8ec00fc67..b39bd2cb28cbb953ac854635ab7ffb8549d3f1ef 100644 GIT binary patch delta 24 gcmX@Yc#M&ai%Eb{YX3yG>C75ZY7-ZzFljIV080o3p#T5? delta 21 dcmX@cc!ZISi%Eb{YVSn0>C9?UDiar|0025C1#tiX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.json index de3fae90c2c4b..0d787f37493b6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.json @@ -26,7 +26,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_reflect.proto.bin index e38e0e5c06548a330ca2fe420d8b80f67287b65c..5971e17d1041bbca301a7050c9f82a59d6cb8897 100644 GIT binary patch delta 24 gcmcb?c#DyZi%Eb{>heUk>C84#RudOkFljIV08vN=82|tP delta 21 dcmcb`c!QCRi%Eb{>f%JU>C9GA784g(002J!1?d0) diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.json index f3a5df24cce88..d1bae052a945c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_subtract.proto.bin index f0cb5f502787389fe36a304417a66b453635eda1..4951a8e3c5fd69ceb8837576de7bf5dff40093ba 100644 GIT binary patch delta 24 gcmdnPxQ~&Ii%Eb{YWqaC>C6gJauXNGFljIV07!@haR2}S delta 21 dcmdnTxQCIAi%Eb{YU@O{>CAFcG7}fb001~Q1wjA+ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.json index 41e93d1fcf956..7961eb8eb5596 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_sum.proto.bin index dce7d9df359c9528abd83ce886fae5d902d4365f..6e4cbe5a01090d70732b9e4daa8c8558409ea5af 100644 GIT binary patch delta 36 rcmdnMxQUUCi%Eb{YV}06>7pW1LR{=6MV0ZzrMW_^Ld;x@i5d(5m|6$4 delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l7RMV0ZzrMW_^Ld;x@i2#UJ2ZR6s diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.json index 9b57b6b26b562..164304ef4b0ae 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.json @@ -22,7 +22,8 @@ "literal": { "string": "format" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary.proto.bin index 28b7059160757aa9bbe2b5548bdcea3f31026c12..3a1b90ca42f70f5faf635436d9e8a60f247efc29 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.json index 2498ff9a7872f..6c676dc702a35 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_binary_without_format.proto.bin index 682eb1821a3a1ba535b5073052e7753e76da618f..4a7ed3da5738c9ce90539b967d03000747df31e3 100644 GIT binary patch delta 24 gcmdnSxPy_6i%Eb{YU4z<>C9465)&7QF=;RW07j?C6&RViOmL0RT3`1swnY diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.json index 44e894743dfc8..f206393079de7 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.json @@ -22,7 +22,8 @@ "literal": { "string": "99,999" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_number.proto.bin index c2eba8a19d5df1b543508114eec27b2a4e70c16b..2eb9ff68df8fea7f66f2c2682423b5eb57115c11 100644 GIT binary patch delta 24 gcmX@ac!ZISi%Eb{YVSn0>C9?UDiaqdF=;RW07_^Dl>h($ delta 21 dcmX@Yc!-gWi%Eb{YWGC8>C7rpN)s0-0RTCx1!VvL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.json index d00967823a33c..e0a532043e00c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp.proto.bin index 4f0300d48a6fc2841a907a13835e4038ff20dce4..ff61c6147cd84861aec3664f43a79e8e5cc5e27d 100644 GIT binary patch delta 24 gcmX@Wc!-gWi%Eb{YWGC8>C7rpN)s0-FljIV07=LNi2wiq delta 21 dcmX@acz}_Oi%Eb{YUf0@>C8$}3KJJ70022J1z7+9 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.json index 4fdfc38ca539b..8589656d5ed52 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_to_timestamp_without_format.proto.bin index 91a4156e305f6f601ac9a328b0e3fe28cc034ec8..72a686887048764494301d45d8b764f518a2738e 100644 GIT binary patch delta 24 gcmdnVxSNrUi%Eb{YRg2n>CCcH(i0a*GHEaX07sYwUjP6A delta 21 dcmdnZxRa5Mi%Eb{YSTov>CDnnk`ot50suFC1up;q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.json index d51704c8f62e2..e73de5e669362 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_url_decode.proto.bin index 3e84921b12206b61d354a70ea0fc776dd8437a1d..e15e5e2d902daaef5d7d0f53387ce5a5b7cb3443 100644 GIT binary patch delta 24 gcmdnaxRa5Mi%Eb{YSTov>CDnnk`ot*GifjY07m!)Qvd(} delta 21 dcmdnVxSf%Wi%Eb{YQsdf>CBQ+;u9B$0{}Mx1tS0e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.json b/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.json index 9a4a4e25f19e6..b8ce69c4bcf39 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.json @@ -21,7 +21,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }, { "literal": { @@ -31,7 +32,8 @@ "literal": { "string": "int" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_try_variant_get.proto.bin index b16bbf4c7a4e908ec17ad8f818eb14303da7e973..82584c937aaa7bd82b5656cb01b2e977cc9a0057 100644 GIT binary patch delta 78 zcmcb?c!!aVi%Eb{>gq(c>GpO~HeCEAMV0YoiA9--c_s1bsUf%JU>DE?K7F_%#MV0YoiA9--c_s1bsUwZ1&Kw)sqtCG d`FTRDLd;x@=|Zer%vp>oLhM|uS)H367m25 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.json b/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.json index 7a6fcfcbcf898..7f9808d9fd947 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_typeof.proto.bin index a042a6e8d7607367eba4e5a56cb1f25e7fdd0642..585d98f767904387fdeb92eddf308ed91b27c49a 100644 GIT binary patch delta 35 qcmZ3_xRH^Ki%Eb{YSl!x=_0~Xf?RARl?AE!X+o?*%v_A=8Vmr8ZU;92 delta 33 ocmdnUxSo-Xi%Eb{YQ;ph=|X~1{9J4$l?AE!X+o?*%v_A=0DWu+2LJ#7 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.json b/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.json index 7193142acdb6f..5580e31e26ffa 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_ucase.proto.bin index 3e17a01d4b1f5b375b47f2cec4814e0b7603d57f..8a2b70936e0f92e7ab7745472c82557e4371c7fe 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i-6$%)0OLaai}T#V@&3;=*Q2C4u6 delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%h$%)0OLaai}T#V@ea##j> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.json index 6af2a00ed160e..af85c7b64779c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unbase64.proto.bin index f37ceb91bf42b72adb12116b97f5adae87a6e0fb..f446e0ad73f45a7ce13649db245aa786cb1ca437 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx7sc}a=Isb(fZtU}COjOiK-0G*WwmH+?% delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OP>c}a=Isb(fZtU}COjOhT4#Rp~p diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.json index 7c409d023f76a..1cea642cc9c68 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unhex.proto.bin index fbac2821fdb07642a568407b15a43d00c0e44711..757eca2dc04d59e7417daf2aa6edd6eb1fdedfbe 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i-6c^Rn{Laai}T#Sht3;=;G2D<yHi%Eb{YWYOA>4E}Md|a%hc^Rn{Laai}T#ShTb1Mdm diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.json index 1a7ae09f46dad..2e617c4e6b8b1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.json @@ -25,9 +25,11 @@ "literal": { "string": "yyyy-MM-dd" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_date.proto.bin index 9c05e42bfad30c0594bbb0bc77fbfa06d833362b..6ed08ec71d76afd6b3ed163ce015a96bf0f3bf8d 100644 GIT binary patch delta 41 wcmbQqIG2%)i%Eb{YT87$L?KftV=m6pyv&ODl*E!$Ar&d5iIoa08Vni?0LB0bTmS$7 delta 36 rcmbQsIFpf$i%Eb{YSKiuL?J^deJ;+@yv&ODl*E!$Atfn=iIoZfmcR)3 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.json index 07f5cd1d53dbd..f7bb6d9ba6264 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.json @@ -25,9 +25,11 @@ "literal": { "string": "yyyy-MM-dd HH:mm:ss.SSSS" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_micros.proto.bin index c3f44d766f8b1daccdb0b7f922d93ac4490528cb..2574acb3c8d95d0f5889c669c113767d0f6ba59e 100644 GIT binary patch delta 43 ycmZ3-xQUUCi%Eb{YV}06L=i734=(P~yv&OD+|1;n{9++yHi%Eb{YWYOAL=iVB7cTD7yv&OD+|1;n{9++fDdUON#sIp&3XcE) diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.json index aac02cc807aa0..4a7c077e88bc1 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.json @@ -25,9 +25,11 @@ "literal": { "string": "yyyy-MM-dd HH:mm:ss.SSSS" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_millis.proto.bin index f0456e03e3fc11d2a93dcde70dd5d9ac68d44ca2..1865aac8a7340f7c026d19ac7e8ef4bf37abe73a 100644 GIT binary patch delta 43 ycmZ3-xQUUCi%Eb{YV}06L=i734=(P~yv&OD+{~Pu%wi#PDbtD7#w;2P8VmsO?F#S! delta 38 tcmdnQxQ>yHi%Eb{YWYOAL=iVB7cTD7yv&OD+{~Pu%wi!^DdUON#sIpp3XA{% diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.json index 428cb26cd9c86..dc3fc3fae0c05 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.json @@ -25,9 +25,11 @@ "literal": { "string": "yyyy-MM-dd HH:mm:ss.SSSS" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_seconds.proto.bin index fdaf50e7322bb8c8d9278b581bc472a06acb6bc1..2ca04f640cda4dd2938e0587c4c07358f9211ea5 100644 GIT binary patch delta 44 zcmZ3_xS5fSi%Eb{YRyEpL{V=kPcEL)yv&OD;?(5)yp&=gb1Bn_HO4F&3>pjo0Kf}f delta 39 ucmdnYxSo-Xi%Eb{YQ;phL{WDsS1z8?yv&OD;?(5)yp&=gQz_$#HO2tPNDBD? diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json index e590f7778f2ea..0780a83d74088 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.json @@ -16,9 +16,11 @@ "functionName": "unix_timestamp", "arguments": [{ "unresolvedFunction": { - "functionName": "current_timestamp" + "functionName": "current_timestamp", + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_unix_timestamp.proto.bin index cb3d967ae0123098b93f8e075aa49425676783be..6f8cf9115629fac4b391206d1ef77e044271d8c7 100644 GIT binary patch delta 61 zcmbQpIF*r&i%Eb{s&67&qMojlHWy!MUS>snNoHsnNoHS?i%Eb{YQ{viL}qg-(}`K805N9-$p8QV diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_upper.json b/sql/connect/common/src/test/resources/query-tests/queries/function_upper.json index 208ee9231a13c..36f1f0258ca82 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_upper.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_upper.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_upper.proto.bin index 5ddbfce96e71bf5062006df57323aa7552f54d32..a7d6be43571a0a37baa8417dca5565d947cde26d 100644 GIT binary patch delta 34 pcmZ3-xPg(4i%Eb{YUM<>>B2%%0$i-61qG=^Laai}T#V@&3;=yHi%Eb{YWYOA>4E}Md|a%h1qG=^Laai}T#V@eb7uyX diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.json index d4cdeeb6c48c5..2e8003e4e9a9f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_url_decode.proto.bin index e347e73c3aef194812cca5311c1260ff8e4d17f6..ff6fb3793e6714146d2c62aead826afb2ba4c4d8 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh|1MLF>)smb{%sY0wm%v_A=8Vmrr`3TYg delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzUXMLF>)smb{%sY0wm%v_A=0H>h{q5uE@ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.json b/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.json index 5d221e0fea6f4..9df8a4683ea65 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_url_encode.proto.bin index 9313fb8249859686262567e9c8f2b2541ba922e7..eb9e31e63697bed74eb0f4338a572ea4b7534d45 100644 GIT binary patch delta 39 ucmdnYxRsHOi%Eb{YTZP(=@Q~nqFh|1MLF@QdCB=HsY0wm%v_A=8Vmrsh6vmM delta 37 scmdnWxS5fSi%Eb{YRyEp>0+W%!dzUXMLF@QdCB=HsY0wm%v_A=0H_)WtN;K2 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_user.json b/sql/connect/common/src/test/resources/query-tests/queries/function_user.json index aaf3de9ba034e..c3ab4ae4be94c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_user.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_user.json @@ -13,7 +13,8 @@ }, "expressions": [{ "unresolvedFunction": { - "functionName": "user" + "functionName": "user", + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_user.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_user.proto.bin index dbd64cae9f3608cfbcaeaa0ab81bce88844a1ab6..17b0d6a4ecc90b1749d4e4154e4fa9624fcb09e8 100644 GIT binary patch delta 26 hcmZ3+xPp<5i%Eb{YT-n->D*jW99%4=#i>Oa3;;^c1&IIv delta 24 fcmZ3%xQvmFi%Eb{YW_sF>6{!=Y+Nj*#i>OAKXC7pW1LR{=+iAC`R`2|9(Ld;x@i5d(5meB{8 delta 34 pcmdnQxPg(4i%Eb{YUM<>>B2%%0$l86iAC`R`2|9(Ld;x@i2#QM2WS8Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.json b/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.json index 979313dd0510d..8132510e61129 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_var_samp.proto.bin index 9bd042ad339e74a1b6de6d4957fa2cc875311b80..99a8d28ec0e72d0ac315cc79021e3888253149ca 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx6>iAC|niMa(rtU}COjENcy0HPfT0RR91 delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OPBiAC|niMa(rtU}COjEMk|cL&h` diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_variance.json b/sql/connect/common/src/test/resources/query-tests/queries/function_variance.json index 90a97c3becf4d..4bfce573c50c5 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_variance.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_variance.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_variance.proto.bin index fd494fc496391f03b66c627d181789df4ed46ac1..e3c0f8512c0c6b0f9755b794f8031ca4a63b4d7f 100644 GIT binary patch delta 37 scmdnUxS5fSi%Eb{YRyEp>0+W%!dx6>iA9--dC93ltU}COjENcy0HJ>e@c;k- delta 35 qcmdnYxRH^Ki%Eb{YSl!x=_0~Xf?OPBiA9--dC93ltU}COjEMk{`3J!O diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.json b/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.json index ab0acd29d505b..d44a94f365b56 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.json @@ -21,7 +21,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }, { "literal": { @@ -31,7 +32,8 @@ "literal": { "string": "int" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_variant_get.proto.bin index fe9b76bb97c4a4946015792251d95eabcc144a36..f1ac3c8c0ad638fbfa5ca88f337223f51cbbcca7 100644 GIT binary patch delta 59 zcmcb@c!QCRi%Eb{>f%JU=^|EA7F^tAiA9--c_s1bsU Pv2(FzF=ytLXfOZ($aM|a delta 55 zcmcb?c!iOTi%Eb{>g+_e=^|!QCS2TQiA9--c_s1bsUE+xvvRi2ufou5{jnph;nD#Xmin4-Y|0G(wB4gdfE delta 35 qcmb=cVdG*FV3aDG$d(``DkaRtRi2ufou5{jnph;nD#Xmim;wNdlLy`a diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_window.json b/sql/connect/common/src/test/resources/query-tests/queries/function_window.json index bdcb6a398800f..95bde679468e0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_window.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_window.json @@ -30,7 +30,8 @@ "literal": { "string": "0 second" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_window.proto.bin index 8cffcc1e9f673b5c184e43919e4e3f16d354d2f6..303b9673c8ab563bbc764039f66f1a3edc53351d 100644 GIT binary patch delta 24 fcmZ3?xRjBNi%Eb{YTiV)L}ptl>xo&GOd1RTL68Lz delta 21 ccmZ3=xR{ZRi%Eb{YR*KqL}qI#%ZXW*05V?$;Q#;t diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.json b/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.json index 4809ea21261c4..2107ac2f12ecd 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.json @@ -35,7 +35,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "wt" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_window_time.proto.bin index c143520df08ce1103a056cbb65b619f66a93b605..3de3a3e156a2cc6729980fc18f306216ae58ee5f 100644 GIT binary patch delta 24 gcmdnWxSf%Wi%EbCBQ+;u9B$GHEaX07hs9NB{r; delta 21 dcmdnaxRsHOi%EbCEC%q7xU00suCW1sDJT diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.json index 3dea90a13653d..fd36b378137c6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b/text()" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath.proto.bin index aabfc76f8a7e1e76d4eb5d79faee5f7442191afb..d07c5d50fc3a44b1ed4e8568930f195ce13a068b 100644 GIT binary patch delta 24 fcmeBR>|$i&ViI7KYM#iJ$gCu#FfmJxNrM3ZIUodY delta 21 ccmeBT>|kW$ViI7KYMjWH$gChGH!({N04vo5KL7v# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.json index 793d459ec165b..0749c2d422314 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_boolean.proto.bin index 544caab4ecc5b012ab8e9465e2b68b23d3fa4d21..e70d2e5c3edd60dfd03735360c808abdb223aacf 100644 GIT binary patch delta 24 fcmeBV>}F)+ViI7KYMIEE$gC`-I5A6}NrM3ZIdBAZ delta 21 ccmeBX>||u)ViI7KYMRKF$gC(OKQT)l04x&(MF0Q* diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.json index f88a06641b8f4..d1c77ae96a86b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_double.proto.bin index 9c4ea317120215b80fc6c2039015d5373fe14625..de580971683b30bffe3e5aeb77df1eb6afddcfaf 100644 GIT binary patch delta 24 fcmeBR>|$i&ViI7KYM#iJ$gCu#FfmJxNrM3ZIUodY delta 21 ccmeBT>|kW$ViI7KYMjWH$gChGH!({N04vo5KL7v# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.json index 94932891225d7..b13e12a2d7e9f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_float.proto.bin index 32dfbc00cfa44ff9e462fc4898d6f30b5eca3f36..4285deba56d7ca9a820303a57efc1f9d09ea554e 100644 GIT binary patch delta 24 fcmZo?>||u)ViI7KYMRKF$gC(OKQT*|NrM3ZIM4)X delta 21 ccmeBVY-eQSViI7KYM98D$Sf}MViI7Ks+-83z$_~zJuyof04oUuEC2ui diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.json index c740d2bad4f5f..3d4d9267a6a50 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_long.proto.bin index d240600eabbae5b72b07796cb51722330aae37e6..7cb6efd6ab2d07d52e4ddf371e172ce197b41ede 100644 GIT binary patch delta 25 gcmZo=>|kW)ViI7KYGj|$i&ViI7KYM#iJ$gCu#FfmJxNrM3ZIUodY delta 21 ccmeBT>|kW$ViI7KYMjWH$gChGH!({N04vo5KL7v# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.json index 5d3a3e9983707..4aa5e3aae7fc9 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.json @@ -22,7 +22,8 @@ "literal": { "string": "a/b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xpath_short.proto.bin index 9ae27bd973853ebfa33e67346c342d9e75d84e64..f407b525cdfa7cf1e189e36afc9297194f4b9bc3 100644 GIT binary patch delta 24 fcmZo?>||u)ViI7KYMRKF$gC(OKQT*|NrM3ZIM4)X delta 21 ccmeBVY-eQSViI7KYM98D$Sf}|$i&ViI7KYM#iJ$gCu#FfmJxNrM3ZIUodY delta 21 ccmeBT>|kW$ViI7KYMjWH$gChGH!({N04vo5KL7v# diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json b/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json index c20739d09ff10..5000f3b164766 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "g" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_xxhash64.proto.bin index 414c76fc5ce7f0db71df3998a8787b088cc32814..de84e70acef5e40860bc0c38554cc0f8d13a72ed 100644 GIT binary patch delta 24 gcmX@kc#@Hgi%Eb{>d-{C>CDcB*{>CBo^>Jt~J0{}YV1%m(p diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_year.json b/sql/connect/common/src/test/resources/query-tests/queries/function_year.json index b8a4ee5a16525..9fadb5c411b9b 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_year.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_year.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "d" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_year.proto.bin index 623bc9ac6d81f7683f017f5ade7de4a2bb7c0d76..91bbfdc180efcd4679973095bb0dc179a2f229a3 100644 GIT binary patch delta 31 mcmXS`VB=yEV3aDF$d(`^D8>B2%%0$i+>sfk6!Laai}T#Shti~xZ82DktK delta 32 ncmdnMxQ>yHi%Eb{YWYOA>4E}Md|a%Rsfk6!Laai}T#ShTa{mU0 diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.json b/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.json index 660ca1931137e..66d53c39742ba 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.json @@ -35,7 +35,8 @@ "unresolvedNamedLambdaVariable": { "nameParts": ["y_2"] } - }] + }], + "isInternal": false } }, "arguments": [{ @@ -44,7 +45,8 @@ "nameParts": ["y_2"] }] } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/function_zip_with.proto.bin index edbfe197af4dc20bc411309e5e6b3cc87b2fbe96..1bf478358f35790ee040a46d8278016c9e68ce3d 100644 GIT binary patch delta 51 zcmaFN_?(fAi%Eb{>cK>|>CB!|?h_YCvTFxva!E>wPdp$hq`@G>%Eep}ZwR6)J E07zB~G5`Po delta 47 zcmaFP_?VH6i%Eb{>dr*A>CCQD&J!0%vTFvZbBRldPCOvWC&bFdToG>wqAKH!02}fO A$^ZZW diff --git a/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.json b/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.json index b7b4c98518e6b..e36ad1de4960d 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.json @@ -25,7 +25,8 @@ "unparsedIdentifier": "a", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -35,7 +36,8 @@ "unparsedIdentifier": "b", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -45,7 +47,8 @@ "unparsedIdentifier": "b", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -55,7 +58,8 @@ "unparsedIdentifier": "b", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -65,7 +69,8 @@ "unparsedIdentifier": "b", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -75,7 +80,8 @@ "unparsedIdentifier": "b", "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -84,7 +90,8 @@ "unresolvedStar": { "planId": "0" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -94,7 +101,8 @@ "unparsedIdentifier": "a", "planId": "0" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/groupby_agg.proto.bin index d7b1b94ed04a251a30d4deb64ea77bcb6d68ab51..22eacb3a01b03d8501342741cf2142400978b1e8 100644 GIT binary patch literal 226 zcmd;L5@7VY&B&#}#U#KGr6iP(8dZ< delta 42 xcmebCVdG*FV3cy3$fhYPBPGSfn~{?fUz(ShpBJB)o-V{G#LmTP(8dZ< delta 42 xcmebCVdG*FV3cy3$fhYPBPGSfn~{?fUz(ShpBJB)o-V{G#LmT_{3UqCIwaiJ?R9r diff --git a/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json b/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json index 527338c56ae60..9308d6babdb25 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.json @@ -46,7 +46,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "r.a" } - }] + }], + "isInternal": false } }, "joinType": "JOIN_TYPE_INNER" diff --git a/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/join_inner_condition.proto.bin index 5d3de55da9cf8d8bc1beda74a80a6d30047bfb98..a49cc6ef30806deb2ea7c5ab5b61bf2779273264 100644 GIT binary patch delta 43 ucmbQjIE|5wi%Eb*tA8R}iHxL_I2WU>5W5g77juqYB7{+-m#D#@zz6`I1_p!x delta 41 scmbQnIE9goi%Eb*t9K$>iIljMC>Nux5W5g77juqYB7{+-m#Dx90FWF8Q~&?~ diff --git a/sql/connect/common/src/test/resources/query-tests/queries/pivot.json b/sql/connect/common/src/test/resources/query-tests/queries/pivot.json index 2af86606b9fcb..f085d1a43b678 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/pivot.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/pivot.json @@ -24,7 +24,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }], "pivot": { diff --git a/sql/connect/common/src/test/resources/query-tests/queries/pivot.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/pivot.proto.bin index f545179e84968884a3fcdc5b3ca0b3bd259027c7..73c88bf97535ea4fcf2b4b1d1a34736eb2386302 100644 GIT binary patch delta 55 zcmYdJW#eKJVDyTg$Y!b~BqhMbnw(#nS0cnJ#LUH*q`{yi%Ed0k%Eg!{z#znAzzD=l HK+FsP{Za>Y delta 33 ocmYdIX5(TKVDyTe$Yv@iAjQYUnw(#nS0cnJ#LUH*G|^8S0BHOMZvX%Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.json b/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.json index aa043613795c4..9d5b22bce6e89 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.json @@ -24,7 +24,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }], "pivot": { diff --git a/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/pivot_without_column_values.proto.bin index 588b56f247e070ead87d0c67a0733adeb768054a..d722db0e17ea9cd80d1a6bf4ebf311b527d0960c 100644 GIT binary patch delta 43 ycmWHKWaDBIVDt)}$Yv@hBqhMbnw(#nS0cnJ#LUH*q`{!Y$;B?j%Eg!{zyJWC(gtS$ delta 41 wcmazHXX9cLVDt){$Yv@dAjQYUnw(#nS0cnJ#LUH*q{Yd_F2u^km?*#i0Fx93Hvj+t diff --git a/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.json b/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.json index 1102db18830bd..146904dc898e3 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.json @@ -30,7 +30,8 @@ "literal": { "integer": 1 } - }] + }], + "isInternal": false } }, "name": ["count"] diff --git a/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/rollup_column.proto.bin index 64dbb597c36501fd6c6cdf2f63c413f0ef3e0e74..8949050821a12ee1d1460ecda269a82d103b53bb 100644 GIT binary patch delta 42 tcma!vW8-2HVDyTZ$Yw1oVCS}6KmYJ86nU^Z0A*IH}npjknSScheCCSBBTvAk;Tq49K U#KOgtnIgn0#LUH*sKEe&0P(2}N&o-= delta 62 zcmc~uV&h^GV3dlP$fj**Af?B}mYJ86nU^Z0BBjK|npjknSSchaCC(O-;7 Gg8=~WkqDsx delta 46 zcmc~yW#eKJV3dlV$fhkPDJ9OuR$Nk4np`5pCd9(Ul$j#LD#Xmim?$M7B{nfo3;@O; B2txn> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.json b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.json index 23850dcb136ef..55b64d26d4904 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -35,7 +36,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -48,7 +50,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_3-arg.proto.bin index b3b56953a858679a37dfc6ccb37fa3d39cd11ac5..d535835523de3be769a165c9db0879bde4ac4622 100644 GIT binary patch delta 83 zcmZo*Y-eQWViI7KYG9nmrmY|?CCSBBTvAk;Tq49K#KOgtnIgn0#LUH*sKFp5B_%P@ LUrZjz!lw)XUeOKw delta 55 zcmZo?Y+z*LViI7Ks+!2I%`7P;J~2;RPD(;bjEk+fq^LBxM2J<0nTs(ILM2H-Q~&@J Ck_pEE diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.json b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.json index 2bbdb60794db5..da0adf605f977 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -35,7 +36,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -48,7 +50,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -61,7 +64,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_4-arg.proto.bin index bacccff22ae0ab76868163acdd7ba0d919fd92ce..50197b862ad1460a354c1da52e31ab5b0ca0c9a7 100644 GIT binary patch delta 54 lcmbQsxRjBNi%Eb{YTiUP9cF1M$%z5tY#IzwxLHgZ3;+mB2><{9 delta 45 gcmZ3=IG2%)i%Eb{YT86L9cD=>@rn83EK=BM0LMKDyZ`_I diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.json b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.json index 4f57c0ef82145..196a91b9fbb81 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.json @@ -22,7 +22,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -35,7 +36,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -48,7 +50,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -61,7 +64,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }, { "unresolvedFunction": { @@ -74,7 +78,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "b" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/select_typed_5-arg.proto.bin index 2c51e2088885f14d8240df6ba82cb885a093748f..e2ff25edd34cd233b26dda1dac37796bf52cefc5 100644 GIT binary patch literal 192 zcmd;L5@3|t$;hR_#U#KGr6iy5J8As5>O7WM!b@EOa%b4dL?=Q delta 90 zcmX@WxQ&sGi%Eb{YW+ku9cD=>@rilja#9jfVq9#+B}JvlB|@x1%v_9#5GqLuqJjlV IOW{@t01{FVO8@`> diff --git a/sql/connect/common/src/test/resources/query-tests/queries/toJSON.json b/sql/connect/common/src/test/resources/query-tests/queries/toJSON.json index 9a99a18853cf1..9faba08d9792c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/toJSON.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/toJSON.json @@ -21,9 +21,11 @@ "unresolvedStar": { "planId": "0" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/toJSON.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/toJSON.proto.bin index e930ee76aae979a4812dd6007ed5fae758d18582..0cd2c3d35c6b472f237a41c74039576e65704767 100644 GIT binary patch delta 49 zcmdnZxSx@Yi%Eb{YR5#j=}L-H@?7jC`SDrB`FTP@QUY9T#U(|h$t6N8T1)~A8bAmD DC!Pw< delta 45 zcmdnbxSNrUi%Eb{YRg2n>2k7C(p>B%`SDrB`FTPDQhZ!&#U(|h$t6N8T1)~A0Q!Fl AY5)KL diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.json b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.json index 6079e13bbfc93..b632fba4a0192 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.json @@ -22,7 +22,8 @@ "literal": { "string": "{\"type\": \"int\", \"name\": \"id\"}" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_with_schema.proto.bin index 2843fbb67fecf92c6ebbdb504a8e7aedbb363810..6c3907802968c62f372a2ed5b5770085633d1d9f 100644 GIT binary patch delta 22 dcmYe#WaDBIV3bOp$fnI~E@e8=#F$Bg0RS#h1Kt1t delta 19 acmc~yXX9cLV3bOn$fnI~DrG#;#25e~w*#&K diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.json b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.json index fa19d2120b94f..dd289a6abcc16 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.json @@ -18,7 +18,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_avro_without_schema.proto.bin index 4e7251125e4ce31bd0ecfe08bd342c16e18246fb..59bce6aac25c7476faa5a3c5684a1f742fd5cb11 100644 GIT binary patch delta 35 qcmZ>DXX9cLV3cy4$fhkODkaRtUXmZ5SXPuT#3sbT#gv(%!2kejhz4H( delta 33 ocmZ>FW#eKJV3e|-$fhkKEG5XrUXmZ5SXPuT#3sbT#gv%>0A3^pF8}}l diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.json b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.json index 921c1b800a089..e71bddc0b19a0 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.json @@ -22,7 +22,8 @@ "literal": { "string": "org.apache.spark.connect.proto.StorageLevel" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName.proto.bin index 5cc7c49882c039d2d6ffb8b150967f20321dbbfb..f49e6d227ddcd844ec3c4549739247a0e13334f2 100644 GIT binary patch delta 22 dcmb=fW#eKJV3aDK$fn8cE9E`W$dgHf0RS{z1Xut7 delta 19 acmb=eX5(TKV3aDI$fn8cE#*1U$P)l3RRkyi diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.json b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.json index 0843b469384e0..c6ccee6f35c3f 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.json @@ -26,7 +26,8 @@ "literal": { "binary": "CvwBCgxjb21tb24ucHJvdG8SDXNwYXJrLmNvbm5lY3QisAEKDFN0b3JhZ2VMZXZlbBIZCgh1c2VfZGlzaxgBIAEoCFIHdXNlRGlzaxIdCgp1c2VfbWVtb3J5GAIgASgIUgl1c2VNZW1vcnkSIAoMdXNlX29mZl9oZWFwGAMgASgIUgp1c2VPZmZIZWFwEiIKDGRlc2VyaWFsaXplZBgEIAEoCFIMZGVzZXJpYWxpemVkEiAKC3JlcGxpY2F0aW9uGAUgASgFUgtyZXBsaWNhdGlvbkIiCh5vcmcuYXBhY2hlLnNwYXJrLmNvbm5lY3QucHJvdG9QAWIGcHJvdG8z" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath.proto.bin index c3fe14aef47da3bdbbe6fa4aa7861155d4ee9e32..c30bc963ce0ebfb47646afe47172fd0ec8c198e2 100644 GIT binary patch delta 26 icmaFP^pc5d{0tEtc&}Qd>5f7&9_yFaQ8yY6gP< delta 23 fcmaFK^qh%}i%Eb{>cK=dEtaiJQkyoK7&8I@P1y!h diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.json b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.json index 76307b3141f7f..bc676b7aa5b1c 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.json @@ -37,9 +37,11 @@ "literal": { "string": "2" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_descFilePath_options.proto.bin index a387611c1ad558fe7e9b0c12ca3046580d91155a..971d6b358711ced56764355b2ba7957166d60192 100644 GIT binary patch delta 36 scmbQvJe!$~i%Eb{YRW`5EtZ!|QqMM;m@qOMNa;<^X4GNPV9;Oy0Gi$h&Hw-a delta 31 ncmbQuJe`@1i%Eb{YQjV|EtaQDQja#8m@qQyN$E_^X4C-ybD#%m diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.json b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.json index 8787f0fc15d77..fa3d57a251cb6 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.json @@ -33,9 +33,11 @@ "literal": { "string": "2" } - }] + }], + "isInternal": false } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/to_protobuf_messageClassName_options.proto.bin index 9ef8348446ad4eb43a40af06685e2ac1c05af146..7f955b1013fe17d6af1af86d75334286868ad5b8 100644 GIT binary patch delta 33 ocmZ3(xPg(4i%Eb{YUM;WE#`Wu+KI+q%mz|=6B~3`G#E4(0D=(*IRF3v delta 28 jcmdnMxQ3C9i%Eb{YUxBaE#_*e%8ABa%z9Ef6B~2@UEl`Q diff --git a/sql/connect/common/src/test/resources/query-tests/queries/where_column.json b/sql/connect/common/src/test/resources/query-tests/queries/where_column.json index bef80a7e6ed5a..15cca60f2a407 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/where_column.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/where_column.json @@ -22,7 +22,8 @@ "literal": { "long": "1" } - }] + }], + "isInternal": false } } } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/where_column.proto.bin index e472ed0715b6254ce2b930c6d031233c7ef7dbe2..ef377b7044366d482b669349851bcf31dbe29084 100644 GIT binary patch delta 35 ncmZ>DXX9cLU{rFP$fhkODkaRtXe-1f#KOgtnF1m$7&RCGPHF@H delta 33 lcmZ>FW#eKJU{tc7$fhkKEG5XrXe-1f#KOgtnF1m$7y(7b1J3{e diff --git a/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.json b/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.json index 93d3b5297d9e1..3e7c112776655 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.json @@ -30,7 +30,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "a" } - }] + }], + "isInternal": false } }] } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/width_bucket.proto.bin index f212e97bc1c5a32e5e111f165b73327788ef390c..2667912763248f191e799b7255eae7b5fd5fe62c 100644 GIT binary patch delta 22 dcmaz`VB=yEV3dlQ$fnJ#C#5sdM2ks-0RStU1E>H1 delta 19 acmYd@W8-2HV3dlO$fnJ#Bc(ObL<;~Qqyu;W diff --git a/sql/connect/common/src/test/resources/query-tests/queries/window.json b/sql/connect/common/src/test/resources/query-tests/queries/window.json index 23fd5c1556ec5..ad9555f3ae898 100644 --- a/sql/connect/common/src/test/resources/query-tests/queries/window.json +++ b/sql/connect/common/src/test/resources/query-tests/queries/window.json @@ -20,7 +20,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "partitionSpec": [{ @@ -42,7 +43,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "partitionSpec": [{ @@ -64,7 +66,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "orderSpec": [{ @@ -94,7 +97,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "orderSpec": [{ @@ -124,7 +128,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "orderSpec": [{ @@ -163,7 +168,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } }, "orderSpec": [{ @@ -202,7 +208,8 @@ "unresolvedAttribute": { "unparsedIdentifier": "id" } - }] + }], + "isInternal": false } } } diff --git a/sql/connect/common/src/test/resources/query-tests/queries/window.proto.bin b/sql/connect/common/src/test/resources/query-tests/queries/window.proto.bin index a89c0d6a6a3f4ffb737b25b1048cd18cac42688b..01616601af0ead303e99367c1aee96e733214b44 100644 GIT binary patch literal 358 zcmd;L5@3|N&%~v{#U#KGr6i&KGxLPlgjl$kGE+1dgjj``xfl~6R1$HD^rLjKm?Xu`1u;v2QGyZ1Ndj|7 z)?gN8LdXUsAr3|%HYuPt449-KG_w?O`j{a4ETH-QiktjYPMc_pwA0RWA- BFl_(; diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index f4be1d17b0e95..490ae473a6e4c 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -1655,14 +1655,18 @@ class SparkConnectPlanner( fun.getArgumentsList.asScala.map(transformExpression).toSeq, isDistinct = fun.getIsDistinct) } else { - // Spark Connect historically used the global namespace to lookup a couple of internal - // functions (e.g. product, collect_top_k, unwrap_udt, ...). In Spark 4 we moved these - // functions to a dedicated namespace, however in order to stay backwards compatible we still - // need to allow connect to use the global namespace. Here we check if a function is - // registered in the internal function registry, and we reroute the lookup to the internal - // registry. val name = fun.getFunctionName - val internal = FunctionRegistry.internal.functionExists(FunctionIdentifier(name)) + val internal = if (fun.hasIsInternal) { + fun.getIsInternal + } else { + // Spark Connect historically used the global namespace to look up a couple of internal + // functions (e.g. product, collect_top_k, unwrap_udt, ...). In Spark 4 we moved these + // functions to a dedicated namespace, however in order to stay backwards compatible we + // still need to allow Connect to use the global namespace. Here we check if a function is + // registered in the internal function registry, and we reroute the lookup to the internal + // registry. + FunctionRegistry.internal.functionExists(FunctionIdentifier(name)) + } UnresolvedFunction( name :: Nil, fun.getArgumentsList.asScala.map(transformExpression).toSeq, diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala index 84d5fd68d4c73..aaeb5d9fe509a 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectPlannerSuite.scala @@ -26,9 +26,10 @@ import org.apache.spark.connect.proto import org.apache.spark.connect.proto.Expression.{Alias, ExpressionString, UnresolvedStar} import org.apache.spark.sql.{AnalysisException, Dataset, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAlias, UnresolvedFunction, UnresolvedRelation} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.DataTypeUtils import org.apache.spark.sql.connect.SparkConnectTestUtils import org.apache.spark.sql.connect.common.InvalidPlanInput @@ -884,4 +885,36 @@ class SparkConnectPlannerSuite extends SparkFunSuite with SparkConnectPlanTest { intercept[AnalysisException](Dataset.ofRows(spark, logical)) } + + test("Internal functions") { + def getProjectRelationWithFn(name: String, isInternal: Option[Boolean]): proto.Relation = { + val fn = proto.Expression.UnresolvedFunction.newBuilder.setFunctionName(name) + isInternal.foreach(fn.setIsInternal) + val proj = proto.Project.newBuilder + .setInput(readRel) + .addExpressions(proto.Expression.newBuilder.setUnresolvedFunction(fn)) + proto.Relation.newBuilder.setProject(proj).build() + } + + def getUnresolvedFunction(plan: LogicalPlan): UnresolvedFunction = + plan.expressions.head.asInstanceOf[UnresolvedAlias].child.asInstanceOf[UnresolvedFunction] + + // "bloom_filter_agg" is an internal function. + val plan1 = transform(getProjectRelationWithFn("bloom_filter_agg", isInternal = None)) + val fn1 = getUnresolvedFunction(plan1) + assert(fn1.nameParts.head == "bloom_filter_agg") + assert(fn1.isInternal) + + // "abcde" is not an internal function. + val plan2 = transform(getProjectRelationWithFn("abcde", isInternal = None)) + val fn2 = getUnresolvedFunction(plan2) + assert(fn2.nameParts.head == "abcde") + assert(!fn2.isInternal) + + // "abcde" is not an internal function but we could set it to be internal. + val plan3 = transform(getProjectRelationWithFn("abcde", isInternal = Some(true))) + val fn3 = getUnresolvedFunction(plan3) + assert(fn3.nameParts.head == "abcde") + assert(fn3.isInternal) + } } From 580b3c0158e4fd746f7c9eb9f0d63099f34f6b4c Mon Sep 17 00:00:00 2001 From: Milan Cupac Date: Fri, 3 Jan 2025 19:19:20 +0800 Subject: [PATCH 26/60] [SPARK-50636][SQL] Extending CTESubstitution.scala to make it aware of recursion ### What changes were proposed in this pull request? 1. Self-contained changes to CTESubstitution.scala that make CTE substitutions and resolutions aware of the recursion. Also, addition of error messages for the incorrect usage of RECURSIVE keyword 2. Introduction of RECURSIVE keyword to the lexer and parser, and other additions due to the introduction of a new keyword - adding RECURSIVE to keywords.sql tests, and hive-thriftserver. More information about recursive CTEs and the future files to be merged: https://docs.google.com/document/d/1qcEJxqoXcr5cSt6HgIQjWQSqhfkSaVYkoDHsg5oxXp4/edit ### Why are the changes needed? Support for the recursive CTE. ### Does this PR introduce _any_ user-facing change? Yes. RECURSIVE keyword is introduced in this PR. ### How was this patch tested? WIP: additional tests to further test this change should be added soon ### Was this patch authored or co-authored using generative AI tooling? No Closes #49232 from milanisvet/milanrcte2continue. Lead-authored-by: Milan Cupac Co-authored-by: Nemanja Petrovic Signed-off-by: Wenchen Fan --- .../resources/error/error-conditions.json | 12 ++ docs/sql-ref-ansi-compliance.md | 1 + .../spark/sql/catalyst/parser/SqlBaseLexer.g4 | 1 + .../sql/catalyst/parser/SqlBaseParser.g4 | 3 +- .../catalyst/analysis/CTESubstitution.scala | 149 ++++++++++++++---- ...wnPredicatesAndPruneColumnsForCTEDef.scala | 4 +- .../sql/catalyst/parser/AstBuilder.scala | 2 +- .../plans/logical/basicLogicalOperators.scala | 9 +- .../analyzer-results/cte-command.sql.out | 10 +- .../analyzer-results/cte-nested.sql.out | 98 ++++++------ .../analyzer-results/cte-nonlegacy.sql.out | 54 +++---- .../sql-tests/analyzer-results/cte.sql.out | 64 ++++---- .../double-quoted-identifiers-enabled.sql.out | 4 +- .../identifier-clause.sql.out | 10 +- .../analyzer-results/join-lateral.sql.out | 4 +- .../non-excludable-rule.sql.out | 2 +- .../postgreSQL/window_part3.sql.out | 8 +- .../analyzer-results/postgreSQL/with.sql.out | 12 +- .../sql-session-variables.sql.out | 2 +- .../exists-subquery/exists-cte.sql.out | 14 +- .../in-subquery/in-multiple-columns.sql.out | 2 +- .../subquery/in-subquery/in-with-cte.sql.out | 20 +-- .../scalar-subquery-select.sql.out | 18 +-- .../analyzer-results/transform.sql.out | 2 +- .../analyzer-results/using-join.sql.out | 4 +- .../results/keywords-enforced.sql.out | 2 + .../sql-tests/results/keywords.sql.out | 1 + .../results/nonansi/keywords.sql.out | 1 + .../ThriftServerWithSparkContextSuite.scala | 2 +- 29 files changed, 311 insertions(+), 204 deletions(-) diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 733b094f744e2..573e7f3a6a384 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -4146,6 +4146,18 @@ ], "sqlState" : "38000" }, + "RECURSIVE_CTE_IN_LEGACY_MODE" : { + "message" : [ + "Recursive definitions cannot be used in legacy CTE precedence mode (spark.sql.legacy.ctePrecedencePolicy=LEGACY)." + ], + "sqlState" : "42836" + }, + "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED" : { + "message" : [ + "Recursive definitions cannot be used when CTE inlining is forced." + ], + "sqlState" : "42836" + }, "RECURSIVE_PROTOBUF_SCHEMA" : { "message" : [ "Found recursive reference in Protobuf schema, which can not be processed by Spark by default: . try setting the option `recursive.fields.max.depth` 1 to 10. Going beyond 10 levels of recursion is not allowed." diff --git a/docs/sql-ref-ansi-compliance.md b/docs/sql-ref-ansi-compliance.md index 7af54850f5da7..50cdcd6d09790 100644 --- a/docs/sql-ref-ansi-compliance.md +++ b/docs/sql-ref-ansi-compliance.md @@ -651,6 +651,7 @@ Below is a list of all the keywords in Spark SQL. |RECORDREADER|non-reserved|non-reserved|non-reserved| |RECORDWRITER|non-reserved|non-reserved|non-reserved| |RECOVER|non-reserved|non-reserved|non-reserved| +|RECURSIVE|reserved|non-reserved|reserved| |REDUCE|non-reserved|non-reserved|non-reserved| |REFERENCES|reserved|non-reserved|reserved| |REFRESH|non-reserved|non-reserved|non-reserved| diff --git a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 index eeebe89de8ff1..91a267364216c 100644 --- a/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 +++ b/sql/api/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBaseLexer.g4 @@ -365,6 +365,7 @@ REAL: 'REAL'; RECORDREADER: 'RECORDREADER'; RECORDWRITER: 'RECORDWRITER'; RECOVER: 'RECOVER'; +RECURSIVE: 'RECURSIVE'; REDUCE: 'REDUCE'; REFERENCES: 'REFERENCES'; REFRESH: 'REFRESH'; 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 a5d217486bf20..3ca120da98dd4 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 @@ -509,7 +509,7 @@ describeColName ; ctes - : WITH namedQuery (COMMA namedQuery)* + : WITH RECURSIVE? namedQuery (COMMA namedQuery)* ; namedQuery @@ -2118,6 +2118,7 @@ nonReserved | RECORDREADER | RECORDWRITER | RECOVER + | RECURSIVE | REDUCE | REFERENCES | REFRESH 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 84f6d04eeb1a4..50f149bb28064 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 @@ -149,10 +149,15 @@ object CTESubstitution extends Rule[LogicalPlan] { plan: LogicalPlan, cteDefs: ArrayBuffer[CTERelationDef]): LogicalPlan = { plan.resolveOperatorsUp { - case UnresolvedWith(child, relations, _) => - val resolvedCTERelations = - resolveCTERelations(relations, isLegacy = true, forceInline = false, Seq.empty, cteDefs) - substituteCTE(child, alwaysInline = true, resolvedCTERelations) + case cte @ UnresolvedWith(child, relations, allowRecursion) => + if (allowRecursion) { + cte.failAnalysis( + errorClass = "RECURSIVE_CTE_IN_LEGACY_MODE", + messageParameters = Map.empty) + } + val resolvedCTERelations = resolveCTERelations(relations, isLegacy = true, + forceInline = false, Seq.empty, cteDefs, allowRecursion) + substituteCTE(child, alwaysInline = true, resolvedCTERelations, None) } } @@ -202,14 +207,21 @@ object CTESubstitution extends Rule[LogicalPlan] { var firstSubstituted: Option[LogicalPlan] = None val newPlan = plan.resolveOperatorsDownWithPruning( _.containsAnyPattern(UNRESOLVED_WITH, PLAN_EXPRESSION)) { - case UnresolvedWith(child: LogicalPlan, relations, _) => + // allowRecursion flag is set to `True` by the parser if the `RECURSIVE` keyword is used. + case cte @ UnresolvedWith(child: LogicalPlan, relations, allowRecursion) => + if (allowRecursion && forceInline) { + cte.failAnalysis( + errorClass = "RECURSIVE_CTE_WHEN_INLINING_IS_FORCED", + messageParameters = Map.empty) + } val resolvedCTERelations = - resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs) ++ - outerCTEDefs + resolveCTERelations(relations, isLegacy = false, forceInline, outerCTEDefs, cteDefs, + allowRecursion) ++ outerCTEDefs val substituted = substituteCTE( traverseAndSubstituteCTE(child, forceInline, resolvedCTERelations, cteDefs)._1, forceInline, - resolvedCTERelations) + resolvedCTERelations, + None) if (firstSubstituted.isEmpty) { firstSubstituted = Some(substituted) } @@ -228,7 +240,8 @@ object CTESubstitution extends Rule[LogicalPlan] { isLegacy: Boolean, forceInline: Boolean, outerCTEDefs: Seq[(String, CTERelationDef)], - cteDefs: ArrayBuffer[CTERelationDef]): Seq[(String, CTERelationDef)] = { + cteDefs: ArrayBuffer[CTERelationDef], + allowRecursion: Boolean): Seq[(String, CTERelationDef)] = { val alwaysInline = isLegacy || forceInline var resolvedCTERelations = if (alwaysInline) { Seq.empty @@ -247,49 +260,116 @@ object CTESubstitution extends Rule[LogicalPlan] { // NOTE: we must call `traverseAndSubstituteCTE` before `substituteCTE`, as the relations // in the inner CTE have higher priority over the relations in the outer CTE when resolving // inner CTE relations. For example: - // WITH t1 AS (SELECT 1) - // t2 AS ( - // WITH t1 AS (SELECT 2) - // WITH t3 AS (SELECT * FROM t1) - // ) - // t3 should resolve the t1 to `SELECT 2` instead of `SELECT 1`. - traverseAndSubstituteCTE(relation, forceInline, resolvedCTERelations, cteDefs)._1 + // WITH + // t1 AS (SELECT 1), + // t2 AS ( + // WITH + // t1 AS (SELECT 2), + // t3 AS (SELECT * FROM t1) + // SELECT * FROM t1 + // ) + // SELECT * FROM t2 + // t3 should resolve the t1 to `SELECT 2` ("inner" t1) instead of `SELECT 1`. + // + // When recursion allowed (RECURSIVE keyword used): + // Consider following example: + // WITH + // t1 AS (SELECT 1), + // t2 AS ( + // WITH RECURSIVE + // t1 AS ( + // SELECT 1 AS level + // UNION ( + // WITH t3 AS (SELECT level + 1 FROM t1 WHERE level < 10) + // SELECT * FROM t3 + // ) + // ) + // SELECT * FROM t1 + // ) + // SELECT * FROM t2 + // t1 reference within t3 would initially resolve to outer `t1` (SELECT 1), as the inner t1 + // is not yet known. Therefore, we need to remove definitions that conflict with current + // relation `name` from the list of `outerCTEDefs` entering `traverseAndSubstituteCTE()`. + // NOTE: It will be recognized later in the code that this is actually a self-reference + // (reference to the inner t1). + val nonConflictingCTERelations = if (allowRecursion) { + resolvedCTERelations.filterNot { + case (cteName, cteDef) => cteDef.conf.resolver(cteName, name) + } + } else { + resolvedCTERelations + } + traverseAndSubstituteCTE(relation, forceInline, nonConflictingCTERelations, cteDefs)._1 } - // CTE definition can reference a previous one - val substituted = substituteCTE(innerCTEResolved, alwaysInline, resolvedCTERelations) + + // If recursion is allowed (RECURSIVE keyword specified) + // then it has higher priority than outer or previous relations. + // Therefore, we construct a `CTERelationDef` for the current relation. + // Later if we encounter unresolved relation which we need to find which CTE Def it is + // referencing to, we first check if it is a reference to this one. If yes, then we set the + // reference as being recursive. + val recursiveCTERelation = if (allowRecursion) { + Some(name -> CTERelationDef(relation)) + } else { + None + } + // CTE definition can reference a previous one or itself if recursion allowed. + val substituted = substituteCTE(innerCTEResolved, alwaysInline, + resolvedCTERelations, recursiveCTERelation) val cteRelation = CTERelationDef(substituted) if (!alwaysInline) { cteDefs += cteRelation } + // Prepending new CTEs makes sure that those have higher priority over outer ones. resolvedCTERelations +:= (name -> cteRelation) } resolvedCTERelations } + /** + * This function is called from `substituteCTE` to actually substitute unresolved relations + * with CTE references. + */ private def resolveWithCTERelations( table: String, alwaysInline: Boolean, cteRelations: Seq[(String, CTERelationDef)], + recursiveCTERelation: Option[(String, CTERelationDef)], unresolvedRelation: UnresolvedRelation): LogicalPlan = { - cteRelations - .find(r => conf.resolver(r._1, table)) - .map { + if (recursiveCTERelation.isDefined && conf.resolver(recursiveCTERelation.get._1, table)) { + // self-reference is found + recursiveCTERelation.map { case (_, d) => - if (alwaysInline) { - d.child - } else { - // Add a `SubqueryAlias` for hint-resolving rules to match relation names. - SubqueryAlias(table, CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) - } - } - .getOrElse(unresolvedRelation) + SubqueryAlias(table, + CTERelationRef(d.id, d.resolved, d.output, d.isStreaming, recursive = true)) + }.get + } else { + cteRelations + .find(r => conf.resolver(r._1, table)) + .map { + case (_, d) => + if (alwaysInline) { + d.child + } else { + // Add a `SubqueryAlias` for hint-resolving rules to match relation names. + // This is a non-recursive reference, recursive parameter is by default set to false + SubqueryAlias(table, + CTERelationRef(d.id, d.resolved, d.output, d.isStreaming)) + } + } + .getOrElse(unresolvedRelation) + } } + /** + * Substitute unresolved relations in the plan with CTE references (CTERelationRef). + */ private def substituteCTE( plan: LogicalPlan, alwaysInline: Boolean, - cteRelations: Seq[(String, CTERelationDef)]): LogicalPlan = { + cteRelations: Seq[(String, CTERelationDef)], + recursiveCTERelation: Option[(String, CTERelationDef)]): LogicalPlan = { plan.resolveOperatorsUpWithPruning( _.containsAnyPattern(RELATION_TIME_TRAVEL, UNRESOLVED_RELATION, PLAN_EXPRESSION, UNRESOLVED_IDENTIFIER)) { @@ -298,7 +378,8 @@ object CTESubstitution extends Rule[LogicalPlan] { throw QueryCompilationErrors.timeTravelUnsupportedError(toSQLId(table)) case u @ UnresolvedRelation(Seq(table), _, _) => - resolveWithCTERelations(table, alwaysInline, cteRelations, u) + resolveWithCTERelations(table, alwaysInline, cteRelations, + recursiveCTERelation, u) case p: PlanWithUnresolvedIdentifier => // We must look up CTE relations first when resolving `UnresolvedRelation`s, @@ -308,7 +389,8 @@ object CTESubstitution extends Rule[LogicalPlan] { p.copy(planBuilder = (nameParts, children) => { p.planBuilder.apply(nameParts, children) match { case u @ UnresolvedRelation(Seq(table), _, _) => - resolveWithCTERelations(table, alwaysInline, cteRelations, u) + resolveWithCTERelations(table, alwaysInline, cteRelations, + recursiveCTERelation, u) case other => other } }) @@ -317,7 +399,8 @@ object CTESubstitution extends Rule[LogicalPlan] { // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. other.transformExpressionsWithPruning(_.containsPattern(PLAN_EXPRESSION)) { case e: SubqueryExpression => - e.withNewPlan(apply(substituteCTE(e.plan, alwaysInline, cteRelations))) + e.withNewPlan( + apply(substituteCTE(e.plan, alwaysInline, cteRelations, None))) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala index 838827a0e2e1e..59b3d83c55162 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushdownPredicatesAndPruneColumnsForCTEDef.scala @@ -122,7 +122,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { private def pushdownPredicatesAndAttributes( plan: LogicalPlan, cteMap: CTEMap): LogicalPlan = plan.transformWithSubqueries { - case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _, _, _) => + case cteDef @ CTERelationDef(child, id, originalPlanWithPredicates, _, _) => val (_, _, newPreds, newAttrSet) = cteMap(id) val originalPlan = originalPlanWithPredicates.map(_._1).getOrElse(child) val preds = originalPlanWithPredicates.map(_._2).getOrElse(Seq.empty) @@ -170,7 +170,7 @@ object PushdownPredicatesAndPruneColumnsForCTEDef extends Rule[LogicalPlan] { object CleanUpTempCTEInfo extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.transformWithPruning(_.containsPattern(CTE)) { - case cteDef @ CTERelationDef(_, _, Some(_), _, _, _) => + case cteDef @ CTERelationDef(_, _, Some(_), _, _) => cteDef.copy(originalPlanWithPredicates = None) } } 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 aa32cc9100515..0a300cea03ffe 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 @@ -551,7 +551,7 @@ class AstBuilder extends DataTypeAstBuilder throw QueryParsingErrors.duplicateCteDefinitionNamesError( duplicates.map(toSQLId).mkString(", "), ctx) } - UnresolvedWith(plan, ctes.toSeq) + UnresolvedWith(plan, ctes.toSeq, ctx.RECURSIVE() != null) } /** 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 b897026a0df46..c1261f2b5fac5 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 @@ -866,7 +866,6 @@ case class UnresolvedWith( * pushdown to help ensure rule idempotency. * @param underSubquery If true, it means we don't need to add a shuffle for this CTE relation as * subquery reuse will be applied to reuse CTE relation output. - * @param recursive If true, then this CTE Definition is recursive - it contains a self-reference. * @param recursionAnchor A helper plan node that temporary stores the anchor term of recursive * definitions. In the beginning of recursive resolution the `ResolveWithCTE` * rule updates this parameter and once it is resolved the same rule resolves @@ -877,7 +876,6 @@ case class CTERelationDef( id: Long = CTERelationDef.newId, originalPlanWithPredicates: Option[(LogicalPlan, Seq[Expression])] = None, underSubquery: Boolean = false, - recursive: Boolean = false, recursionAnchor: Option[LogicalPlan] = None) extends UnaryNode { final override val nodePatterns: Seq[TreePattern] = Seq(CTE) @@ -886,6 +884,13 @@ case class CTERelationDef( copy(child = newChild) override def output: Seq[Attribute] = if (resolved) child.output else Nil + + lazy val recursive: Boolean = child.exists{ + // if the reference is found inside the child, referencing to this CTE definition, + // and already marked as recursive, then this CTE definition is recursive. + case CTERelationRef(this.id, _, _, _, _, true) => true + case _ => false + } } object CTERelationDef { diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out index 926e3cd003046..c12076b85b1df 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-command.sql.out @@ -4,7 +4,7 @@ CREATE TABLE cte_tbl USING csv AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateDataSourceTableAsSelectCommand `spark_catalog`.`default`.`cte_tbl`, ErrorIfExists, [col] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation @@ -26,7 +26,7 @@ CREATE TEMPORARY VIEW cte_view AS WITH s AS (SELECT 42 AS col) SELECT * FROM s -- !query analysis CreateViewCommand `cte_view`, WITH s AS (SELECT 42 AS col) SELECT * FROM s, false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation @@ -43,7 +43,7 @@ Project [col#x] +- View (`cte_view`, [col#x]) +- Project [cast(col#x as int) AS col#x] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias s : +- Project [42 AS col#x] : +- OneRowRelation @@ -58,7 +58,7 @@ INSERT INTO cte_tbl SELECT * FROM S -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias s : +- Project [43 AS col#x] : +- OneRowRelation @@ -80,7 +80,7 @@ INSERT INTO cte_tbl WITH s AS (SELECT 44 AS col) SELECT * FROM s -- !query analysis InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/cte_tbl, false, CSV, [path=file:[not included in comparison]/{warehouse_dir}/cte_tbl], Append, `spark_catalog`.`default`.`cte_tbl`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/cte_tbl), [col] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias s : +- Project [44 AS col#x] : +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out index 0365c4dcd8e09..1b968d0431f4a 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nested.sql.out @@ -7,11 +7,11 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 @@ -30,7 +30,7 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -48,7 +48,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -128,15 +128,15 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2#x] : +- SubqueryAlias t @@ -160,18 +160,18 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false, false +: : :- CTERelationDef xxxx, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] @@ -199,24 +199,24 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [3 AS 3#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t : +- CTERelationRef xxxx, true, [3#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [3#x] : +- SubqueryAlias t2 @@ -234,12 +234,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -261,12 +261,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -291,17 +291,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -323,13 +323,13 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false, false + : :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation @@ -349,14 +349,14 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false, false + : : :- CTERelationDef xxxx, false : : : +- SubqueryAlias t : : : +- Project [2 AS 2#x] : : : +- OneRowRelation @@ -378,19 +378,19 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false, false + : :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [2 AS 2#x] : : +- OneRowRelation : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- WithCTE - : : :- CTERelationDef xxxx, false, false + : : :- CTERelationDef xxxx, false : : : +- SubqueryAlias t : : : +- Project [3 AS 3#x] : : : +- OneRowRelation @@ -410,7 +410,7 @@ WHERE c IN ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -418,7 +418,7 @@ WithCTE +- Project [c#x] +- Filter c#x IN (list#x []) : +- WithCTE - : :- CTERelationDef xxxx, false, false + : :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [2#x AS c#x] : : +- Project [2 AS 2#x] @@ -440,16 +440,16 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation @@ -468,15 +468,15 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias aBc : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x] : +- SubqueryAlias aBC @@ -494,13 +494,13 @@ SELECT ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias abc : +- Project [1 AS 1#x] : +- OneRowRelation +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE - : :- CTERelationDef xxxx, false, false + : :- CTERelationDef xxxx, false : : +- SubqueryAlias aBc : : +- Project [2 AS 2#x] : : +- OneRowRelation @@ -522,16 +522,16 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 @@ -553,11 +553,11 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer @@ -586,16 +586,16 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out index 53dcd46361dc1..fcfe64097069b 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte-nonlegacy.sql.out @@ -7,11 +7,11 @@ WITH t as ( SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 @@ -30,7 +30,7 @@ SELECT max(c) FROM ( Aggregate [max(c#x) AS max(c)#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] @@ -48,7 +48,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -66,7 +66,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias unreferenced : : +- Project [outer(id#xL)] : : +- OneRowRelation @@ -83,7 +83,7 @@ SELECT ( -- !query analysis Project [scalar-subquery#x [id#xL] AS scalarsubquery(id)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias unreferenced : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -153,18 +153,18 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [scalar-subquery#x [] AS scalarsubquery()#x] : : +- Aggregate [max(c#x) AS max(c)#x] : : +- SubqueryAlias __auto_generated_subquery_name : : +- WithCTE -: : :- CTERelationDef xxxx, false, false +: : :- CTERelationDef xxxx, false : : : +- SubqueryAlias t : : : +- Project [2#x AS c#x] : : : +- Project [2 AS 2#x] @@ -211,12 +211,12 @@ SELECT max(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -238,12 +238,12 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] @@ -268,17 +268,17 @@ SELECT sum(c) FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS c#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [2#x AS c#x] : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [3#x AS c#x] : +- Project [3 AS 3#x] @@ -384,16 +384,16 @@ WITH SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x] : +- SubqueryAlias t2 : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- OneRowRelation @@ -454,16 +454,16 @@ WITH SELECT * FROM t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t1 : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t3 : +- Project [1#x] : +- SubqueryAlias t1 : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [1#x] : +- SubqueryAlias t3 @@ -485,11 +485,11 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias cte_outer @@ -518,16 +518,16 @@ SELECT * FROM ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_outer : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner_inner : +- Project [1#x] : +- SubqueryAlias cte_outer : +- CTERelationRef xxxx, true, [1#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte_inner : +- Project [1#x] : +- SubqueryAlias __auto_generated_subquery_name diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index 9a221f6a699cd..ea09573db51a5 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -62,7 +62,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t AS (SELECT 1 FROM t) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS 1#x] : +- SubqueryAlias t @@ -100,7 +100,7 @@ org.apache.spark.sql.catalyst.ExtendedAnalysisException WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t1 : +- Project [id#x] : +- SubqueryAlias t2 @@ -109,7 +109,7 @@ WithCTE : +- Project [id#x] : +- SubqueryAlias t : +- LocalRelation [id#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [2 AS 2#x] : +- SubqueryAlias t1 @@ -134,7 +134,7 @@ FROM CTE1 t1 CROSS JOIN CTE1 t2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias CTE1 : +- Project [id#x AS id#x] : +- Join Cross @@ -168,7 +168,7 @@ WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE x = 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS x#x] : +- Project [1 AS 1#x] @@ -184,7 +184,7 @@ WITH t(x, y) AS (SELECT 1, 2) SELECT * FROM t WHERE x = 1 AND y = 2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -200,7 +200,7 @@ WITH t(x, x) AS (SELECT 1, 2) SELECT * FROM t -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1#x AS x#x, 2#x AS x#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -300,48 +300,48 @@ WITH w1(c1) AS SELECT * FROM w1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w8 : +- Project [1#x AS c8#x] : +- Project [1 AS 1#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w7 : +- Project [c8#x AS c7#x] : +- Project [c8#x] : +- SubqueryAlias w8 : +- CTERelationRef xxxx, true, [c8#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w6 : +- Project [c7#x AS c6#x] : +- Project [c7#x] : +- SubqueryAlias w7 : +- CTERelationRef xxxx, true, [c7#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w5 : +- Project [c6#x AS c5#x] : +- Project [c6#x] : +- SubqueryAlias w6 : +- CTERelationRef xxxx, true, [c6#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w4 : +- Project [c5#x AS c4#x] : +- Project [c5#x] : +- SubqueryAlias w5 : +- CTERelationRef xxxx, true, [c5#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w3 : +- Project [c4#x AS c3#x] : +- Project [c4#x] : +- SubqueryAlias w4 : +- CTERelationRef xxxx, true, [c4#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w2 : +- Project [c3#x AS c2#x] : +- Project [c3#x] : +- SubqueryAlias w3 : +- CTERelationRef xxxx, true, [c3#x], false, false -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias w1 : +- Project [c2#x AS c1#x] : +- Project [c2#x] @@ -379,7 +379,7 @@ WITH same_name AS (SELECT 42) SELECT * FROM same_name, (SELECT 10) AS same_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias same_name : +- Project [42 AS 42#x] : +- OneRowRelation @@ -419,7 +419,7 @@ WITH q AS (SELECT 'foo' AS x) SELECT x, typeof(x) FROM q -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q : +- Project [foo AS x#x] : +- OneRowRelation @@ -479,7 +479,7 @@ SELECT * FROM Project [y#x] +- SubqueryAlias __auto_generated_subquery_name +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias q : +- Project [1 AS x#x] : +- OneRowRelation @@ -493,7 +493,7 @@ select (with q as (select 1 x) select * from q) -- !query analysis Project [scalar-subquery#x [] AS scalarsubquery()#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias q : : +- Project [1 AS x#x] : : +- OneRowRelation @@ -508,7 +508,7 @@ select 1 in (with q as (select 1) select * from q) -- !query analysis Project [1 IN (list#x []) AS (1 IN (listquery()))#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias q : : +- Project [1 AS 1#x] : : +- OneRowRelation @@ -549,11 +549,11 @@ from T1 z -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias T1 : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias T1 : +- Project [2 AS b#x] : +- OneRowRelation @@ -580,15 +580,15 @@ from (WITH TtTt as (select 3 c) select * from ttTT, `tttT_2`) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias TTtt : +- Project [1 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias tTTt_2 : +- Project [2 AS a#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias TtTt : +- Project [3 AS c#x] : +- OneRowRelation @@ -609,7 +609,7 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x] AS scalarsubquery(x)#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias q : : +- Project [outer(x#x)] : : +- OneRowRelation @@ -628,7 +628,7 @@ from (select 1 x, 2 y) T -- !query analysis Project [scalar-subquery#x [x#x && y#x] AS scalarsubquery(x, y)#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias q : : +- Project [3 AS z#x] : : +- OneRowRelation @@ -646,11 +646,11 @@ select * from (with q2 as (select * from q1) select * from q2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q2 : +- Project [x#x] : +- SubqueryAlias q1 @@ -668,11 +668,11 @@ select * from (with q1 as (select x+1 from q1) select * from q1) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q1 : +- Project [1 AS x#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q1 : +- Project [(x#x + 1) AS (x + 1)#x] : +- SubqueryAlias q1 @@ -709,7 +709,7 @@ with cte1 as (Select id as j from t) select * from cte1 where j = (select max(j) from cte1 as cte2) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [id#x AS j#x] : +- SubqueryAlias t diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out index 35713167bc38d..f2b1d91d7e447 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/double-quoted-identifiers-enabled.sql.out @@ -411,7 +411,7 @@ CREATE TEMPORARY VIEW "myview"("c1") AS -- !query analysis CreateViewCommand `myview`, [(c1,None)], WITH "v"("a") AS (SELECT 1) SELECT "a" FROM "v", false, false, LocalTempView, UNSUPPORTED, true +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] @@ -431,7 +431,7 @@ Project [a1#x AS a2#x] +- View (`myview`, [c1#x]) +- Project [cast(a#x as int) AS c1#x] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias v : +- Project [1#x AS a#x] : +- Project [1 AS 1#x] 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 9384f17fa74c1..87d537f949813 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 @@ -1012,11 +1012,11 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER(agg)(IDENTIFIER(col)) FROM IDENTIFIER(tab) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] @@ -1031,11 +1031,11 @@ WITH S(c1, c2) AS (VALUES(1, 2), (2, 3)), SELECT IDENTIFIER('max')(IDENTIFIER('c1')) FROM IDENTIFIER('T') -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias S : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias T : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] @@ -1049,7 +1049,7 @@ 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, false +:- CTERelationDef xxxx, false : +- SubqueryAlias ABC : +- Project [col1#x AS c1#x, col2#x AS c2#x] : +- LocalRelation [col1#x, col2#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out index c93109e428362..390fcf5e30152 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/join-lateral.sql.out @@ -1358,14 +1358,14 @@ WITH cte1 AS ( SELECT * FROM cte2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [c1#x] : +- SubqueryAlias spark_catalog.default.t1 : +- View (`spark_catalog`.`default`.`t1`, [c1#x, c2#x]) : +- Project [cast(col1#x as int) AS c1#x, cast(col2#x as int) AS c2#x] : +- LocalRelation [col1#x, col2#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte2 : +- Project [c1#x, c2#x] : +- LateralJoin lateral-subquery#x [c1#x], Inner diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out index 7f1ca3dda902d..4a717488e0172 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/non-excludable-rule.sql.out @@ -36,7 +36,7 @@ WITH tmp AS ( SELECT id FROM range(3) WHERE id > (SELECT max(id) FROM tmp) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias tmp : +- Intersect false : :- Project [id#xL] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out index 441b3627bc928..2b2d69db348d3 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/window_part3.sql.out @@ -89,7 +89,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -111,7 +111,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte : +- Project [id#xL AS x#xL] : +- Project [id#xL] @@ -134,7 +134,7 @@ FROM cte WINDOW w AS (ORDER BY x rows between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false @@ -167,7 +167,7 @@ FROM cte WINDOW w AS (ORDER BY x range between 1 preceding and 1 following) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte : +- Project [1#xL AS x#xL] : +- Union false, false diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out index b92e70e91a4ba..1219562068dd9 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/postgreSQL/with.sql.out @@ -4,7 +4,7 @@ WITH q1(x,y) AS (SELECT 1,2) SELECT * FROM q1, q1 AS q2 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias q1 : +- Project [1#x AS x#x, 2#x AS y#x] : +- Project [1 AS 1#x, 2 AS 2#x] @@ -185,7 +185,7 @@ InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_d with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte : +- Project [42#x AS foo#x] : +- Project [42 AS 42#x] @@ -207,11 +207,11 @@ WITH outermost(x) AS ( SELECT * FROM outermost ORDER BY 1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias innermost : +- Project [2 AS 2#x] : +- OneRowRelation -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias outermost : +- Project [1#x AS x#x] : +- Distinct @@ -412,7 +412,7 @@ org.apache.spark.sql.catalyst.parser.ParseException with ordinality as (select 1 as x) select * from ordinality -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias ordinality : +- Project [1 AS x#x] : +- OneRowRelation @@ -453,7 +453,7 @@ with test as (select 42) insert into test select * from test InsertIntoHadoopFsRelationCommand file:[not included in comparison]/{warehouse_dir}/test, false, Parquet, [path=file:[not included in comparison]/{warehouse_dir}/test], Append, `spark_catalog`.`default`.`test`, org.apache.spark.sql.execution.datasources.InMemoryFileIndex(file:[not included in comparison]/{warehouse_dir}/test), [i] +- Project [cast(42#x as int) AS i#x] +- WithCTE - :- CTERelationDef xxxx, false, false + :- CTERelationDef xxxx, false : +- SubqueryAlias test : +- Project [42 AS 42#x] : +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out index da4542fe9bb1c..77e7174167eca 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-session-variables.sql.out @@ -2044,7 +2044,7 @@ Project [1 AS 1#x] WITH v1 AS (SELECT var1 AS c1) SELECT c1 AS `1` FROM v1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias v1 : +- Project [variablereference(system.session.var1=1) AS c1#x] : +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out index f210bc49030fc..abaf6a2432251 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/exists-subquery/exists-cte.sql.out @@ -101,7 +101,7 @@ WHERE a.bonus_amt > 30 WHERE a.emp_name = b.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias bonus_cte : +- Project [emp_name#x, bonus_amt#x] : +- Filter exists#x [emp_name#x] @@ -162,7 +162,7 @@ WHERE EXISTS (SELECT * WHERE bonus.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -172,7 +172,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -225,7 +225,7 @@ WHERE e.dept_id = d.dept_id WHERE e.emp_name = a.emp_name) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias emp_cte : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- Filter ((id#x >= 100) AND (id#x <= 300)) @@ -235,7 +235,7 @@ WithCTE : +- Project [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] : +- SubqueryAlias EMP : +- LocalRelation [id#x, emp_name#x, hiredate#x, salary#x, dept_id#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias dept_cte : +- Project [dept_id#x, dept_name#x, state#x] : +- Filter (dept_id#x = 10) @@ -299,7 +299,7 @@ WHERE EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) @@ -352,7 +352,7 @@ WHERE NOT EXISTS (SELECT dept_id, GROUP BY emp_name -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias empdept : +- Project [id#x, salary#x, emp_name#x, dept_id#x] : +- Filter id#x IN (100,200) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out index 85e28de736a0c..230ffc005e90d 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-multiple-columns.sql.out @@ -307,7 +307,7 @@ FROM ( on cte1.t1b = cte2.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter named_struct(t1b, t1b#x, t1d, t1d#xL) IN (list#x [t1c#x]) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out index 03a4c3ffd4fcb..199b876fb9a86 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/in-subquery/in-with-cte.sql.out @@ -123,7 +123,7 @@ WHERE t1b IN (SELECT cte1.t1b WHERE cte1.t1b > 0) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1a) @@ -177,7 +177,7 @@ GROUP BY t1a, t1b, t1c HAVING t1c IS NOT NULL -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- SubqueryAlias t1 @@ -250,7 +250,7 @@ WHERE t1c IN ON cte1.t1d > cte6.t1d) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1e#x] : +- SubqueryAlias t1 @@ -316,7 +316,7 @@ FROM (SELECT * ON cte1.t1b = cte4.t1b) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1b#x IN (list#x []) AND (t1a#x = val1b)) @@ -393,7 +393,7 @@ WHERE t1b IN GROUP BY t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1h#x] : +- Filter t1a#x IN (list#x [t1b#x]) @@ -453,7 +453,7 @@ FROM ( ) s -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x] : +- Filter (t1b#x IN (list#x [t1c#x]) AND (t1a#x = val1b)) @@ -515,7 +515,7 @@ FROM (SELECT cte1.t1a, GROUP BY s.t1b -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -569,7 +569,7 @@ WHERE s.t1b IN ON t1.t1a = cte1.t1a) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter t1b#x IN (list#x [t1c#x]) @@ -627,7 +627,7 @@ WHERE t1b NOT IN (SELECT cte1.t1b t1c > 10 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x] : +- Filter (t1a#x = val1d) @@ -683,7 +683,7 @@ WHERE t1b NOT IN ORDER BY t1c DESC -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias cte1 : +- Project [t1a#x, t1b#x, t1c#x, t1d#xL, t1h#x] : +- Filter NOT t1d#xL IN (list#x []) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out index 3b1ffa94c17f2..2a3a87e5cab81 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/subquery/scalar-subquery/scalar-subquery-select.sql.out @@ -617,7 +617,7 @@ SELECT c1, (WITH t AS (SELECT 1 AS a) SELECT a + c1 FROM t) FROM t1 -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#x] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [1 AS a#x] : : +- OneRowRelation @@ -636,7 +636,7 @@ SELECT c1, (WITH t AS (SELECT * FROM t2 WHERE c1 = t1.c1) SELECT SUM(c2) FROM t) -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- Filter (c1#x = outer(c1#x)) @@ -664,7 +664,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t3 : : +- Project [(c1#x + 1) AS c1#x, (c2#x + 1) AS c2#x] : : +- SubqueryAlias t2 @@ -672,7 +672,7 @@ Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : : +- Project [cast(c1#x as int) AS c1#x, cast(c2#x as int) AS c2#x] : : +- SubqueryAlias t2 : : +- LocalRelation [c1#x, c2#x] -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t4 : : +- Project [c1#x, c2#x] : : +- Filter (outer(c1#x) = c1#x) @@ -697,7 +697,7 @@ SELECT c1, ( -- !query analysis Project [c1#x, scalar-subquery#x [c1#x] AS scalarsubquery(c1)#xL] : +- WithCTE -: :- CTERelationDef xxxx, false, false +: :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -732,7 +732,7 @@ SELECT * FROM t1 WHERE c1 > ( ) -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias v : +- Project [c1#x, c2#x] : +- SubqueryAlias t2 @@ -743,7 +743,7 @@ WithCTE +- Project [c1#x, c2#x] +- Filter (cast(c1#x as bigint) > scalar-subquery#x [c1#x]) : +- WithCTE - : :- CTERelationDef xxxx, false, false + : :- CTERelationDef xxxx, false : : +- SubqueryAlias t : : +- Project [c1#x, c2#x] : : +- SubqueryAlias t2 @@ -771,7 +771,7 @@ WITH t AS (SELECT 1 AS a) SELECT c1, (SELECT a FROM t WHERE a = c1) FROM t1 -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t : +- Project [1 AS a#x] : +- OneRowRelation @@ -1019,7 +1019,7 @@ SELECT (SELECT sum(1) FROM T WHERE a = col OR upper(col)= 'Y') FROM (SELECT null as col) as foo -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias T : +- Project [1 AS a#x] : +- OneRowRelation diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out index 04e019fdbaa41..d5376913f9ae4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/transform.sql.out @@ -874,7 +874,7 @@ WITH temp AS ( SELECT t1.b FROM temp t1 JOIN temp t2 ON t1.b = t2.b -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias temp : +- ScriptTransformation cat, [b#x], ScriptInputOutputSchema(List(),List(),None,None,List(),List(),None,None,false) : +- Project [a#x] diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out index 89f988fe2b616..d26c5ba4430da 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/using-join.sql.out @@ -817,12 +817,12 @@ FROM t1 FULL OUTER JOIN t2 USING (key) WHERE t1.key NOT LIKE 'bb.%' -- !query analysis WithCTE -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t1 : +- Project [key#x] : +- SubqueryAlias t : +- LocalRelation [key#x] -:- CTERelationDef xxxx, false, false +:- CTERelationDef xxxx, false : +- SubqueryAlias t2 : +- Project [key#x] : +- SubqueryAlias t diff --git a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out index 7d96a3e98c832..f9c9df3f9bf5f 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords-enforced.sql.out @@ -252,6 +252,7 @@ REAL false RECORDREADER false RECORDWRITER false RECOVER false +RECURSIVE true REDUCE false REFERENCES true REFRESH false @@ -432,6 +433,7 @@ ORDER OUTER OVERLAPS PRIMARY +RECURSIVE REFERENCES RIGHT SELECT diff --git a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out index 6cbfe519a76f6..67e5e4170d789 100644 --- a/sql/core/src/test/resources/sql-tests/results/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/keywords.sql.out @@ -252,6 +252,7 @@ REAL false RECORDREADER false RECORDWRITER false RECOVER false +RECURSIVE false REDUCE false REFERENCES false REFRESH false diff --git a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out index 6cbfe519a76f6..67e5e4170d789 100644 --- a/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/nonansi/keywords.sql.out @@ -252,6 +252,7 @@ REAL false RECORDREADER false RECORDWRITER false RECOVER false +RECURSIVE false REDUCE false REFERENCES false REFRESH false diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala index 2acf25640ef78..da0ddd3a156f7 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/ThriftServerWithSparkContextSuite.scala @@ -214,7 +214,7 @@ trait ThriftServerWithSparkContextSuite extends SharedThriftServer { val sessionHandle = client.openSession(user, "") val infoValue = client.getInfo(sessionHandle, GetInfoType.CLI_ODBC_KEYWORDS) // scalastyle:off line.size.limit - assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") + assert(infoValue.getStringValue == "ADD,AFTER,AGGREGATE,ALL,ALTER,ALWAYS,ANALYZE,AND,ANTI,ANY,ANY_VALUE,ARCHIVE,ARRAY,AS,ASC,AT,AUTHORIZATION,BEGIN,BETWEEN,BIGINT,BINARY,BINDING,BOOLEAN,BOTH,BUCKET,BUCKETS,BY,BYTE,CACHE,CALL,CALLED,CASCADE,CASE,CAST,CATALOG,CATALOGS,CHANGE,CHAR,CHARACTER,CHECK,CLEAR,CLUSTER,CLUSTERED,CODEGEN,COLLATE,COLLATION,COLLECTION,COLUMN,COLUMNS,COMMENT,COMMIT,COMPACT,COMPACTIONS,COMPENSATION,COMPUTE,CONCATENATE,CONSTRAINT,CONTAINS,COST,CREATE,CROSS,CUBE,CURRENT,CURRENT_DATE,CURRENT_TIME,CURRENT_TIMESTAMP,CURRENT_USER,DATA,DATABASE,DATABASES,DATE,DATEADD,DATEDIFF,DATE_ADD,DATE_DIFF,DAY,DAYOFYEAR,DAYS,DBPROPERTIES,DEC,DECIMAL,DECLARE,DEFAULT,DEFINED,DEFINER,DELETE,DELIMITED,DESC,DESCRIBE,DETERMINISTIC,DFS,DIRECTORIES,DIRECTORY,DISTINCT,DISTRIBUTE,DIV,DO,DOUBLE,DROP,ELSE,END,ESCAPE,ESCAPED,EVOLUTION,EXCEPT,EXCHANGE,EXCLUDE,EXECUTE,EXISTS,EXPLAIN,EXPORT,EXTEND,EXTENDED,EXTERNAL,EXTRACT,FALSE,FETCH,FIELDS,FILEFORMAT,FILTER,FIRST,FLOAT,FOLLOWING,FOR,FOREIGN,FORMAT,FORMATTED,FROM,FULL,FUNCTION,FUNCTIONS,GENERATED,GLOBAL,GRANT,GROUP,GROUPING,HAVING,HOUR,HOURS,IDENTIFIER,IDENTITY,IF,IGNORE,ILIKE,IMMEDIATE,IMPORT,IN,INCLUDE,INCREMENT,INDEX,INDEXES,INNER,INPATH,INPUT,INPUTFORMAT,INSERT,INT,INTEGER,INTERSECT,INTERVAL,INTO,INVOKER,IS,ITEMS,ITERATE,JOIN,KEYS,LANGUAGE,LAST,LATERAL,LAZY,LEADING,LEAVE,LEFT,LIKE,LIMIT,LINES,LIST,LOAD,LOCAL,LOCATION,LOCK,LOCKS,LOGICAL,LONG,LOOP,MACRO,MAP,MATCHED,MERGE,MICROSECOND,MICROSECONDS,MILLISECOND,MILLISECONDS,MINUS,MINUTE,MINUTES,MODIFIES,MONTH,MONTHS,MSCK,NAME,NAMESPACE,NAMESPACES,NANOSECOND,NANOSECONDS,NATURAL,NO,NONE,NOT,NULL,NULLS,NUMERIC,OF,OFFSET,ON,ONLY,OPTION,OPTIONS,OR,ORDER,OUT,OUTER,OUTPUTFORMAT,OVER,OVERLAPS,OVERLAY,OVERWRITE,PARTITION,PARTITIONED,PARTITIONS,PERCENT,PIVOT,PLACING,POSITION,PRECEDING,PRIMARY,PRINCIPALS,PROPERTIES,PURGE,QUARTER,QUERY,RANGE,READS,REAL,RECORDREADER,RECORDWRITER,RECOVER,RECURSIVE,REDUCE,REFERENCES,REFRESH,RENAME,REPAIR,REPEAT,REPEATABLE,REPLACE,RESET,RESPECT,RESTRICT,RETURN,RETURNS,REVOKE,RIGHT,ROLE,ROLES,ROLLBACK,ROLLUP,ROW,ROWS,SCHEMA,SCHEMAS,SECOND,SECONDS,SECURITY,SELECT,SEMI,SEPARATED,SERDE,SERDEPROPERTIES,SESSION_USER,SET,SETS,SHORT,SHOW,SINGLE,SKEWED,SMALLINT,SOME,SORT,SORTED,SOURCE,SPECIFIC,SQL,START,STATISTICS,STORED,STRATIFY,STRING,STRUCT,SUBSTR,SUBSTRING,SYNC,SYSTEM_TIME,SYSTEM_VERSION,TABLE,TABLES,TABLESAMPLE,TARGET,TBLPROPERTIES,TERMINATED,THEN,TIME,TIMEDIFF,TIMESTAMP,TIMESTAMPADD,TIMESTAMPDIFF,TIMESTAMP_LTZ,TIMESTAMP_NTZ,TINYINT,TO,TOUCH,TRAILING,TRANSACTION,TRANSACTIONS,TRANSFORM,TRIM,TRUE,TRUNCATE,TRY_CAST,TYPE,UNARCHIVE,UNBOUNDED,UNCACHE,UNION,UNIQUE,UNKNOWN,UNLOCK,UNPIVOT,UNSET,UNTIL,UPDATE,USE,USER,USING,VALUES,VAR,VARCHAR,VARIABLE,VARIANT,VERSION,VIEW,VIEWS,VOID,WEEK,WEEKS,WHEN,WHERE,WHILE,WINDOW,WITH,WITHIN,X,YEAR,YEARS,ZONE") // scalastyle:on line.size.limit } } From a59941a8ff5372ccd907922ff24ccb66b2419a94 Mon Sep 17 00:00:00 2001 From: YangJie Date: Fri, 3 Jan 2025 08:15:11 -0800 Subject: [PATCH 27/60] [SPARK-50678][BUILD] Clean up outdated style check rules related to Guava ### What changes were proposed in this pull request? The version of Guava being used by Spark is 33.3.1 now, so this pr cleans up the outdated check rules from `dev/checkstyle.xml` and `scalastyle-config.xml` - The current version of Guava no longer has the `Objects.toStringHelper` method, so this check rule is no longer needed - CVE-2020-8908 has been fixed after version 32.0.0, so the related check is no longer needed - CVE-2023-2976 has been fixed after version 32.0.0, so the related check is no longer needed - CVE-2018-10237 has been fixed after version 24.1.1, so the related check is no longer needed ### Why are the changes needed? Clean up outdated style check rules related to Guava ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass Github Actions ### Was this patch authored or co-authored using generative AI tooling? No Closes #49302 from LuciferYang/checkstyle-outdated. Lead-authored-by: YangJie Co-authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- dev/checkstyle.xml | 22 ---------------------- scalastyle-config.xml | 27 --------------------------- 2 files changed, 49 deletions(-) diff --git a/dev/checkstyle.xml b/dev/checkstyle.xml index 4285028109419..6c50718e27fe5 100644 --- a/dev/checkstyle.xml +++ b/dev/checkstyle.xml @@ -172,32 +172,10 @@ - - - - - - - - - - - - - - - - - - - - diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 05b3f6a268985..7e64dc9be6731 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -460,33 +460,6 @@ This file is divided into 3 sections: -1,0,1,2,3 - - Objects.toStringHelper - Avoid using Object.toStringHelper. Use ToStringBuilder instead. - - - - Files\.createTempDir\( - Avoid using com.google.common.io.Files.createTempDir due to CVE-2020-8908. - Use org.apache.spark.util.Utils.createTempDir instead. - - - - - FileBackedOutputStream - Avoid using FileBackedOutputStream due to CVE-2023-2976. - - - - AtomicDoubleArray - Avoid using AtomicDoubleArray due to CVE-2018-10237. - - - - CompoundOrdering - Avoid using CompoundOrdering due to CVE-2018-10237. - - byteCountToDisplaySize Use Utils.bytesToString instead of byteCountToDisplaySize for consistency. From dccb129ea045cfab36d8eb7a6525a6875c46e184 Mon Sep 17 00:00:00 2001 From: panbingkun Date: Fri, 3 Jan 2025 08:20:39 -0800 Subject: [PATCH 28/60] [SPARK-50493][SPARK-37687][BUILD] Migrate kubernetes-client from `6.x` to `7.x` ### What changes were proposed in this pull request? The pr aims to migrate kubernetes-client from `6.x` to `7.x`, includes: - upgrade `kubernetes-client` from `6.13.2` to `7.0.1`. - switch default `HttpClient` implementation from `OkHttp` to `Vert.x`. ### Why are the changes needed? - The full release notes: https://github.com/fabric8io/kubernetes-client/releases/tag/v7.0.1 https://github.com/fabric8io/kubernetes-client/releases/tag/v7.0.0 - [Vert.x as default HttpClient implementation](https://github.com/fabric8io/kubernetes-client/blob/v7.0.0/doc/MIGRATION-v7.md#vertx-as-default-httpclient-implementation-) ### 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 #49159 from panbingkun/k8s_client_7_vert_x. Authored-by: panbingkun Signed-off-by: Dongjoon Hyun --- LICENSE-binary | 4 ++ dev/deps/spark-deps-hadoop-3-hive-2.3 | 61 ++++++++++--------- hadoop-cloud/pom.xml | 12 ++++ pom.xml | 8 +-- resource-managers/kubernetes/core/pom.xml | 7 +-- .../k8s/SparkKubernetesClientFactory.scala | 18 +----- .../k8s/features/VolcanoFeatureStep.scala | 2 +- .../features/VolcanoFeatureStepSuite.scala | 2 +- .../integrationtest/VolcanoTestsSuite.scala | 2 +- 9 files changed, 55 insertions(+), 61 deletions(-) diff --git a/LICENSE-binary b/LICENSE-binary index 40d28fbe71e6b..5cf099cb4d3c4 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -286,6 +286,10 @@ io.netty:netty-transport-classes-kqueue io.netty:netty-transport-native-epoll io.netty:netty-transport-native-kqueue io.netty:netty-transport-native-unix-common +io.vertx:vertx-auth-common +io.vertx:vertx-core +io.vertx:vertx-web-client +io.vertx:vertx-web-common jakarta.inject:jakarta.inject-api jakarta.validation:jakarta.validation-api javax.jdo:jdo-api diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index 8ce2b4d362990..d6db12457eeeb 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -158,31 +158,31 @@ jsr305/3.0.0//jsr305-3.0.0.jar jta/1.1//jta-1.1.jar jul-to-slf4j/2.0.16//jul-to-slf4j-2.0.16.jar kryo-shaded/4.0.2//kryo-shaded-4.0.2.jar -kubernetes-client-api/6.13.4//kubernetes-client-api-6.13.4.jar -kubernetes-client/6.13.4//kubernetes-client-6.13.4.jar -kubernetes-httpclient-okhttp/6.13.4//kubernetes-httpclient-okhttp-6.13.4.jar -kubernetes-model-admissionregistration/6.13.4//kubernetes-model-admissionregistration-6.13.4.jar -kubernetes-model-apiextensions/6.13.4//kubernetes-model-apiextensions-6.13.4.jar -kubernetes-model-apps/6.13.4//kubernetes-model-apps-6.13.4.jar -kubernetes-model-autoscaling/6.13.4//kubernetes-model-autoscaling-6.13.4.jar -kubernetes-model-batch/6.13.4//kubernetes-model-batch-6.13.4.jar -kubernetes-model-certificates/6.13.4//kubernetes-model-certificates-6.13.4.jar -kubernetes-model-common/6.13.4//kubernetes-model-common-6.13.4.jar -kubernetes-model-coordination/6.13.4//kubernetes-model-coordination-6.13.4.jar -kubernetes-model-core/6.13.4//kubernetes-model-core-6.13.4.jar -kubernetes-model-discovery/6.13.4//kubernetes-model-discovery-6.13.4.jar -kubernetes-model-events/6.13.4//kubernetes-model-events-6.13.4.jar -kubernetes-model-extensions/6.13.4//kubernetes-model-extensions-6.13.4.jar -kubernetes-model-flowcontrol/6.13.4//kubernetes-model-flowcontrol-6.13.4.jar -kubernetes-model-gatewayapi/6.13.4//kubernetes-model-gatewayapi-6.13.4.jar -kubernetes-model-metrics/6.13.4//kubernetes-model-metrics-6.13.4.jar -kubernetes-model-networking/6.13.4//kubernetes-model-networking-6.13.4.jar -kubernetes-model-node/6.13.4//kubernetes-model-node-6.13.4.jar -kubernetes-model-policy/6.13.4//kubernetes-model-policy-6.13.4.jar -kubernetes-model-rbac/6.13.4//kubernetes-model-rbac-6.13.4.jar -kubernetes-model-resource/6.13.4//kubernetes-model-resource-6.13.4.jar -kubernetes-model-scheduling/6.13.4//kubernetes-model-scheduling-6.13.4.jar -kubernetes-model-storageclass/6.13.4//kubernetes-model-storageclass-6.13.4.jar +kubernetes-client-api/7.0.1//kubernetes-client-api-7.0.1.jar +kubernetes-client/7.0.1//kubernetes-client-7.0.1.jar +kubernetes-httpclient-vertx/7.0.1//kubernetes-httpclient-vertx-7.0.1.jar +kubernetes-model-admissionregistration/7.0.1//kubernetes-model-admissionregistration-7.0.1.jar +kubernetes-model-apiextensions/7.0.1//kubernetes-model-apiextensions-7.0.1.jar +kubernetes-model-apps/7.0.1//kubernetes-model-apps-7.0.1.jar +kubernetes-model-autoscaling/7.0.1//kubernetes-model-autoscaling-7.0.1.jar +kubernetes-model-batch/7.0.1//kubernetes-model-batch-7.0.1.jar +kubernetes-model-certificates/7.0.1//kubernetes-model-certificates-7.0.1.jar +kubernetes-model-common/7.0.1//kubernetes-model-common-7.0.1.jar +kubernetes-model-coordination/7.0.1//kubernetes-model-coordination-7.0.1.jar +kubernetes-model-core/7.0.1//kubernetes-model-core-7.0.1.jar +kubernetes-model-discovery/7.0.1//kubernetes-model-discovery-7.0.1.jar +kubernetes-model-events/7.0.1//kubernetes-model-events-7.0.1.jar +kubernetes-model-extensions/7.0.1//kubernetes-model-extensions-7.0.1.jar +kubernetes-model-flowcontrol/7.0.1//kubernetes-model-flowcontrol-7.0.1.jar +kubernetes-model-gatewayapi/7.0.1//kubernetes-model-gatewayapi-7.0.1.jar +kubernetes-model-metrics/7.0.1//kubernetes-model-metrics-7.0.1.jar +kubernetes-model-networking/7.0.1//kubernetes-model-networking-7.0.1.jar +kubernetes-model-node/7.0.1//kubernetes-model-node-7.0.1.jar +kubernetes-model-policy/7.0.1//kubernetes-model-policy-7.0.1.jar +kubernetes-model-rbac/7.0.1//kubernetes-model-rbac-7.0.1.jar +kubernetes-model-resource/7.0.1//kubernetes-model-resource-7.0.1.jar +kubernetes-model-scheduling/7.0.1//kubernetes-model-scheduling-7.0.1.jar +kubernetes-model-storageclass/7.0.1//kubernetes-model-storageclass-7.0.1.jar lapack/3.0.3//lapack-3.0.3.jar leveldbjni-all/1.8//leveldbjni-all-1.8.jar libfb303/0.9.3//libfb303-0.9.3.jar @@ -193,7 +193,6 @@ log4j-api/2.24.3//log4j-api-2.24.3.jar log4j-core/2.24.3//log4j-core-2.24.3.jar log4j-layout-template-json/2.24.3//log4j-layout-template-json-2.24.3.jar log4j-slf4j2-impl/2.24.3//log4j-slf4j2-impl-2.24.3.jar -logging-interceptor/3.12.12//logging-interceptor-3.12.12.jar lz4-java/1.8.0//lz4-java-1.8.0.jar metrics-core/4.2.29//metrics-core-4.2.29.jar metrics-graphite/4.2.29//metrics-graphite-4.2.29.jar @@ -203,6 +202,7 @@ metrics-jvm/4.2.29//metrics-jvm-4.2.29.jar minlog/1.3.0//minlog-1.3.0.jar netty-all/4.1.115.Final//netty-all-4.1.115.Final.jar netty-buffer/4.1.115.Final//netty-buffer-4.1.115.Final.jar +netty-codec-dns/4.1.115.Final//netty-codec-dns-4.1.115.Final.jar netty-codec-http/4.1.115.Final//netty-codec-http-4.1.115.Final.jar netty-codec-http2/4.1.115.Final//netty-codec-http2-4.1.115.Final.jar netty-codec-socks/4.1.115.Final//netty-codec-socks-4.1.115.Final.jar @@ -210,6 +210,7 @@ netty-codec/4.1.115.Final//netty-codec-4.1.115.Final.jar netty-common/4.1.115.Final//netty-common-4.1.115.Final.jar netty-handler-proxy/4.1.115.Final//netty-handler-proxy-4.1.115.Final.jar netty-handler/4.1.115.Final//netty-handler-4.1.115.Final.jar +netty-resolver-dns/4.1.115.Final//netty-resolver-dns-4.1.115.Final.jar netty-resolver/4.1.115.Final//netty-resolver-4.1.115.Final.jar netty-tcnative-boringssl-static/2.0.69.Final/linux-aarch_64/netty-tcnative-boringssl-static-2.0.69.Final-linux-aarch_64.jar netty-tcnative-boringssl-static/2.0.69.Final/linux-x86_64/netty-tcnative-boringssl-static-2.0.69.Final-linux-x86_64.jar @@ -258,7 +259,7 @@ scala-parser-combinators_2.13/2.4.0//scala-parser-combinators_2.13-2.4.0.jar scala-reflect/2.13.15//scala-reflect-2.13.15.jar scala-xml_2.13/2.3.0//scala-xml_2.13-2.3.0.jar slf4j-api/2.0.16//slf4j-api-2.0.16.jar -snakeyaml-engine/2.7//snakeyaml-engine-2.7.jar +snakeyaml-engine/2.8//snakeyaml-engine-2.8.jar snakeyaml/2.3//snakeyaml-2.3.jar snappy-java/1.1.10.7//snappy-java-1.1.10.7.jar spire-macros_2.13/0.18.0//spire-macros_2.13-0.18.0.jar @@ -272,11 +273,15 @@ threeten-extra/1.7.1//threeten-extra-1.7.1.jar tink/1.16.0//tink-1.16.0.jar transaction-api/1.1//transaction-api-1.1.jar univocity-parsers/2.9.1//univocity-parsers-2.9.1.jar +vertx-auth-common/4.5.11//vertx-auth-common-4.5.11.jar +vertx-core/4.5.11//vertx-core-4.5.11.jar +vertx-web-client/4.5.11//vertx-web-client-4.5.11.jar +vertx-web-common/4.5.11//vertx-web-common-4.5.11.jar wildfly-openssl/1.1.3.Final//wildfly-openssl-1.1.3.Final.jar xbean-asm9-shaded/4.26//xbean-asm9-shaded-4.26.jar xmlschema-core/2.3.1//xmlschema-core-2.3.1.jar xz/1.10//xz-1.10.jar -zjsonpatch/0.3.0//zjsonpatch-0.3.0.jar +zjsonpatch/7.0.1//zjsonpatch-7.0.1.jar zookeeper-jute/3.9.3//zookeeper-jute-3.9.3.jar zookeeper/3.9.3//zookeeper-3.9.3.jar zstd-jni/1.5.6-8//zstd-jni-1.5.6-8.jar diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 9c2f21e7ab617..3361081549242 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -34,6 +34,8 @@ hadoop-cloud + 3.12.12 + 1.17.6 @@ -191,6 +193,16 @@ ${jetty.version} ${hadoop.deps.scope} + + com.squareup.okhttp3 + okhttp + ${okhttp.version} + + + com.squareup.okio + okio + ${okio.version} + diff --git a/pom.xml b/pom.xml index 05ebba16270de..ba8f42a3b181d 100644 --- a/pom.xml +++ b/pom.xml @@ -235,8 +235,7 @@ org.fusesource.leveldbjni - 6.13.4 - 1.17.6 + 7.0.1 ${java.home} @@ -2594,11 +2593,6 @@ javax.servlet-api ${javaxservlet.version} - - com.squareup.okio - okio - ${okio.version} - diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 211c6c93b9674..814a3e1c595b8 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -37,7 +37,7 @@ io.fabric8 - volcano-model-v1beta1 + volcano-model ${kubernetes-client.version} @@ -105,11 +105,6 @@ test - - io.fabric8 - kubernetes-httpclient-okhttp - ${kubernetes-client.version} - io.fabric8 kubernetes-client diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index 2c28dc380046c..557bf01cbdbae 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -24,10 +24,7 @@ import com.google.common.io.Files import io.fabric8.kubernetes.client.{ConfigBuilder, KubernetesClient, KubernetesClientBuilder} import io.fabric8.kubernetes.client.Config.KUBERNETES_REQUEST_RETRY_BACKOFFLIMIT_SYSTEM_PROPERTY import io.fabric8.kubernetes.client.Config.autoConfigure -import io.fabric8.kubernetes.client.okhttp.OkHttpClientFactory import io.fabric8.kubernetes.client.utils.Utils.getSystemPropertyOrEnvVar -import okhttp3.Dispatcher -import okhttp3.OkHttpClient import org.apache.spark.SparkConf import org.apache.spark.annotation.{DeveloperApi, Since, Stable} @@ -35,7 +32,6 @@ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys.K8S_CONTEXT import org.apache.spark.internal.config.ConfigEntry -import org.apache.spark.util.ThreadUtils /** * :: DeveloperApi :: @@ -78,10 +74,6 @@ object SparkKubernetesClientFactory extends Logging { .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_KEY_FILE_CONF_SUFFIX") val clientCertFile = sparkConf .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") - // TODO(SPARK-37687): clean up direct usage of OkHttpClient, see also: - // https://github.com/fabric8io/kubernetes-client/issues/3547 - val dispatcher = new Dispatcher( - ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) // Allow for specifying a context used to auto-configure from the users K8S config file val kubeContext = sparkConf.get(KUBERNETES_CONTEXT).filter(_.nonEmpty) @@ -117,17 +109,9 @@ object SparkKubernetesClientFactory extends Logging { }.withOption(namespace) { (ns, configBuilder) => configBuilder.withNamespace(ns) }.build() - val factoryWithCustomDispatcher = new OkHttpClientFactory() { - override protected def additionalConfig(builder: OkHttpClient.Builder): Unit = { - builder.dispatcher(dispatcher) - } - } logDebug("Kubernetes client config: " + new ObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(config)) - new KubernetesClientBuilder() - .withHttpClientFactory(factoryWithCustomDispatcher) - .withConfig(config) - .build() + new KubernetesClientBuilder().withConfig(config).build() } private implicit class OptionConfigurableConfigBuilder(val configBuilder: ConfigBuilder) diff --git a/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala b/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala index 314550713ef16..046d268df4e29 100644 --- a/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala +++ b/resource-managers/kubernetes/core/volcano/src/main/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStep.scala @@ -17,8 +17,8 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model._ +import io.fabric8.volcano.api.model.scheduling.v1beta1.{PodGroup, PodGroupSpec} import io.fabric8.volcano.client.DefaultVolcanoClient -import io.fabric8.volcano.scheduling.v1beta1.{PodGroup, PodGroupSpec} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverConf, KubernetesExecutorConf, SparkPod} import org.apache.spark.internal.Logging diff --git a/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala b/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala index dab414e0e19e7..c92164993ef80 100644 --- a/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/volcano/src/test/scala/org/apache/spark/deploy/k8s/features/VolcanoFeatureStepSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.features import java.io.File -import io.fabric8.volcano.scheduling.v1beta1.PodGroup +import io.fabric8.volcano.api.model.scheduling.v1beta1.PodGroup import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ diff --git a/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala b/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala index 2f414b72ee195..4a9dc135ecfdc 100644 --- a/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/volcano/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/VolcanoTestsSuite.scala @@ -28,8 +28,8 @@ import scala.concurrent.Future import scala.jdk.CollectionConverters._ import io.fabric8.kubernetes.api.model.{HasMetadata, Pod, Quantity} +import io.fabric8.volcano.api.model.scheduling.v1beta1.{Queue, QueueBuilder} import io.fabric8.volcano.client.VolcanoClient -import io.fabric8.volcano.scheduling.v1beta1.{Queue, QueueBuilder} import org.scalatest.BeforeAndAfterEach import org.scalatest.concurrent.Eventually From 98dc76364ffab660b52fb8a22e5d2fbce4bd8c47 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sat, 4 Jan 2025 13:28:23 -0800 Subject: [PATCH 29/60] [SPARK-50716][CORE] Fix the cleanup logic for symbolic links in `JavaUtils.deleteRecursivelyUsingJavaIO` method ### What changes were proposed in this pull request? To address the cleanup logic for symbolic links in the `JavaUtils.deleteRecursivelyUsingJavaIO` method, the following changes have been made in this pr: 1. Change to use `Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS)` to read the `BasicFileAttributes` of the file. By specifying `LinkOption.NOFOLLOW_LINKS`, the attributes of the symbolic link itself are read, rather than those of the file it points to. This allows us to use `fileAttributes.isSymbolicLink()` to check if a file is a symbolic link. 2. After the above change, it is no longer possible for `fileAttributes.isDirectory()` and `fileAttributes.isSymbolicLink()` to be true simultaneously. Therefore, when `fileAttributes.isDirectory()` is true, there is no need to check `!fileAttributes.isSymbolicLink()`. 3. When `fileAttributes.isSymbolicLink()` is true, deletion behavior for the symbolic link has been added. 4. When `!file.exists()` is true, an additional check for `!fileAttributes.isSymbolicLink()` has been added. This is because for a broken symbolic link, `file.exists()` will also return false, but in such cases, we should proceed with the cleanup. 5. The previously handwritten `isSymlink` method in JavaUtils has been removed, as it is no longer needed after the above changes. ### Why are the changes needed? Fix the cleanup logic for symbolic links in `JavaUtils.deleteRecursivelyUsingJavaIO` method. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - New test cases have been added - Check with existing test cases which named `PipedRDDSuite`: Run `build/sbt "core/testOnly org.apache.spark.rdd.PipedRDDSuite"` Before ``` git status On branch upmaster Your branch is up to date with 'upstream/master'. Untracked files: (use "git add ..." to include in what will be committed) core/tasks/ ls -l core/tasks total 0 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 099f2492-acef-4556-8a34-1318dccf7ad2 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 47d46196-2f7b-4c7b-acf3-7e1d26584c12 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 5e23fe20-1e3f-49b8-8404-5cd3b1033e37 drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 a2cbf5a9-3ebf-4332-be87-c9501830750e drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 ddf45bf5-d0fa-4970-9094-930f382b675c drwxr-xr-x 5 yangjie01 staff 160 1 3 18:15 e25fe5ad-a0be-48d0-81f6-605542f447b5 ls -l core/tasks/099f2492-acef-4556-8a34-1318dccf7ad2 total 0 lrwxr-xr-x 1 yangjie01 staff 59 1 3 18:15 benchmarks -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./benchmarks lrwxr-xr-x 1 yangjie01 staff 52 1 3 18:15 src -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./src lrwxr-xr-x 1 yangjie01 staff 55 1 3 18:15 target -> /Users/yangjie01/SourceCode/git/spark-sbt/core/./target ``` We noticed that symbolic links are left behind after the tests, even though manual cleanup has been invoked in the test code: https://github.com/apache/spark/blob/b210f422b0078d535eddc696ebba8d92f67b81fb/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala#L214-L232 After ``` git status On branch deleteRecursivelyUsingJavaIO-SymbolicLink Your branch is up to date with 'origin/deleteRecursivelyUsingJavaIO-SymbolicLink'. nothing to commit, working tree clean ``` We observe that there are no residual symbolic links left after the tests. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49347 from LuciferYang/deleteRecursivelyUsingJavaIO-SymbolicLink. Lead-authored-by: yangjie01 Co-authored-by: YangJie Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/JavaUtils.java | 23 ++++------- .../org/apache/spark/util/UtilsSuite.scala | 38 +++++++++++++++++++ 2 files changed, 45 insertions(+), 16 deletions(-) diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java index 90dddc2cb08c1..027a0d5fb402e 100644 --- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -22,6 +22,7 @@ import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.nio.file.LinkOption; import java.nio.file.attribute.BasicFileAttributes; import java.util.*; import java.util.concurrent.TimeUnit; @@ -125,10 +126,11 @@ public static void deleteRecursively(File file, FilenameFilter filter) throws IO private static void deleteRecursivelyUsingJavaIO( File file, FilenameFilter filter) throws IOException { - if (!file.exists()) return; BasicFileAttributes fileAttributes = - Files.readAttributes(file.toPath(), BasicFileAttributes.class); - if (fileAttributes.isDirectory() && !isSymlink(file)) { + Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); + // SPARK-50716: If the file does not exist and not a broken symbolic link, return directly. + if (!file.exists() && !fileAttributes.isSymbolicLink()) return; + if (fileAttributes.isDirectory()) { IOException savedIOException = null; for (File child : listFilesSafely(file, filter)) { try { @@ -143,8 +145,8 @@ private static void deleteRecursivelyUsingJavaIO( } } - // Delete file only when it's a normal file or an empty directory. - if (fileAttributes.isRegularFile() || + // Delete file only when it's a normal file, a symbolic link, or an empty directory. + if (fileAttributes.isRegularFile() || fileAttributes.isSymbolicLink() || (fileAttributes.isDirectory() && listFilesSafely(file, null).length == 0)) { boolean deleted = file.delete(); // Delete can also fail if the file simply did not exist. @@ -192,17 +194,6 @@ private static File[] listFilesSafely(File file, FilenameFilter filter) throws I } } - private static boolean isSymlink(File file) throws IOException { - Objects.requireNonNull(file); - File fileInCanonicalDir = null; - if (file.getParent() == null) { - fileInCanonicalDir = file; - } else { - fileInCanonicalDir = new File(file.getParentFile().getCanonicalFile(), file.getName()); - } - return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); - } - private static final Map timeSuffixes; private static final Map byteSuffixes; diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index febc59e5c5369..baa748573d75b 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -22,6 +22,7 @@ import java.lang.reflect.Field import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.{Files => JFiles} import java.text.DecimalFormatSymbols import java.util.Locale import java.util.concurrent.TimeUnit @@ -731,6 +732,43 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties { assert(!sourceFile2.exists()) } + test("SPARK-50716: deleteRecursively - SymbolicLink To File") { + val tempDir = Utils.createTempDir() + val sourceFile = new File(tempDir, "foo.txt") + JFiles.write(sourceFile.toPath, "Some content".getBytes) + assert(sourceFile.exists()) + + val symlinkFile = new File(tempDir, "bar.txt") + JFiles.createSymbolicLink(symlinkFile.toPath, sourceFile.toPath) + + // Check that the symlink was created successfully + assert(JFiles.isSymbolicLink(symlinkFile.toPath)) + Utils.deleteRecursively(tempDir) + + // Verify that everything is deleted + assert(!tempDir.exists) + } + + test("SPARK-50716: deleteRecursively - SymbolicLink To Dir") { + val tempDir = Utils.createTempDir() + val sourceDir = new File(tempDir, "sourceDir") + assert(sourceDir.mkdir()) + val sourceFile = new File(sourceDir, "file.txt") + JFiles.write(sourceFile.toPath, "Some content".getBytes) + + val symlinkDir = new File(tempDir, "targetDir") + JFiles.createSymbolicLink(symlinkDir.toPath, sourceDir.toPath) + + // Check that the symlink was created successfully + assert(JFiles.isSymbolicLink(symlinkDir.toPath)) + + // Now delete recursively + Utils.deleteRecursively(tempDir) + + // Verify that everything is deleted + assert(!tempDir.exists) + } + test("loading properties from file") { withTempDir { tmpDir => val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) From 0d0fa86a03119608159078b0f6474db96485a80f Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Sat, 4 Jan 2025 13:30:32 -0800 Subject: [PATCH 30/60] [SPARK-50578][PYTHON][SS][FOLLOWUP] Change to use `Thread.interrupt` instead of `Thread.stop` to interrupt the execution of TransformWithStateInPandasPythonPreInitRunner#daemonThread ### What changes were proposed in this pull request? This PR change to use `Thread.interrupt()` instead of `Thread.stop()` to attempt to interrupt the execution of `TransformWithStateInPandasPythonPreInitRunner#daemonThread`. Additionally, logic has been added in `TransformWithStateInPandasStateServer#run` to respond to the interrupt by setting the `CLOSED` state and exiting. ### Why are the changes needed? The `Thread.stop` method in Java 21 directly throws an `UnsupportedOperationException`, which led to the failure of the Java 21 daily tests: - https://github.com/apache/spark/actions/runs/12511573912/job/34903859772 - https://github.com/apache/spark/actions/runs/12523542188/job/34933207012 - https://github.com/apache/spark/actions/runs/12592534465/job/35097321533 ![image](https://github.com/user-attachments/assets/75cef6d7-d66a-4652-b01d-38412d6db3b0) So the primary purpose of this change is to restore the daily tests for Java 21. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass GitHub Actions - Pass Java 21 GitHub Action test: https://github.com/LuciferYang/spark/actions/runs/12606699142/job/35137180872 ![image](https://github.com/user-attachments/assets/9e5e8b08-d167-4f7a-959c-8ebe6e22f9bc) ### Was this patch authored or co-authored using generative AI tooling? No Closes #49354 from LuciferYang/java21-test-2. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../python/TransformWithStateInPandasPythonRunner.scala | 2 +- .../python/TransformWithStateInPandasStateServer.scala | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala index 79e199f598b0c..f415ae2543d34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasPythonRunner.scala @@ -275,7 +275,7 @@ class TransformWithStateInPandasPythonPreInitRunner( override def stop(): Unit = { super.stop() closeServerSocketChannelSilently(stateServerSocket) - daemonThread.stop() + daemonThread.interrupt() } private def startStateServer(): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index a48d0bfd15034..fe1bbdd66ac17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -146,6 +146,9 @@ class TransformWithStateInPandasStateServer( while (listeningSocket.isConnected && statefulProcessorHandle.getHandleState != StatefulProcessorHandleState.CLOSED) { + if (Thread.currentThread().isInterrupted) { + throw new InterruptedException("Thread was interrupted") + } try { val version = inputStream.readInt() if (version != -1) { @@ -159,6 +162,11 @@ class TransformWithStateInPandasStateServer( logWarning(log"No more data to read from the socket") statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED) return + case _: InterruptedException => + logInfo(log"Thread interrupted, shutting down state server") + Thread.currentThread().interrupt() + statefulProcessorHandle.setHandleState(StatefulProcessorHandleState.CLOSED) + return case e: Exception => logError(log"Error reading message: ${MDC(LogKeys.ERROR, e.getMessage)}", e) sendResponse(1, e.getMessage) From 91f3fdd25852b43095dd5273358fc394ffd11b66 Mon Sep 17 00:00:00 2001 From: Zhihong Yu Date: Mon, 6 Jan 2025 00:08:11 +0800 Subject: [PATCH 31/60] [SPARK-50578][PYTHON][SS][FOLLOWUP] Remove unnecessary if block ### What changes were proposed in this pull request? This PR removes the `if` block at the beginning of the while loop since `InterruptedException` is handled in the try / catch block. ### Why are the changes needed? This change makes the code easier to follow. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Existing Java 21 daily tests ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49356 from tedyu/tws-if. Authored-by: Zhihong Yu Signed-off-by: yangjie01 --- .../python/TransformWithStateInPandasStateServer.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala index fe1bbdd66ac17..e37e4266b46b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/TransformWithStateInPandasStateServer.scala @@ -146,9 +146,6 @@ class TransformWithStateInPandasStateServer( while (listeningSocket.isConnected && statefulProcessorHandle.getHandleState != StatefulProcessorHandleState.CLOSED) { - if (Thread.currentThread().isInterrupted) { - throw new InterruptedException("Thread was interrupted") - } try { val version = inputStream.readInt() if (version != -1) { From 24c9247208c20baf3985d9248eca4098cfe5bce3 Mon Sep 17 00:00:00 2001 From: William Hyun Date: Sun, 5 Jan 2025 15:18:22 -0800 Subject: [PATCH 32/60] [SPARK-50730][SQL][TESTS] Skip TPC-DS collation benchmark in branch-3.5 ### What changes were proposed in this pull request? This PR aims to skip TPC-DS collation benchmark in branch-3.5 ### Why are the changes needed? TPS-DS collation benchmark was added by the following PR at Spark 4.0 - https://github.com/apache/spark/pull/45739 However, branch-3.5 daily CI is trying to run non-existing tests. - https://github.com/apache/spark/actions/runs/12618956363/job/35162950623 Screenshot 2025-01-05 at 2 48 54 PM ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Check manually after merge. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49360 from williamhyun/SPARK-50730. Authored-by: William Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 13f4e66842ef1..29621930b1ffa 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1094,6 +1094,7 @@ jobs: spark.sql.autoBroadcastJoinThreshold=-1 spark.sql.join.forceApplyShuffledHashJoin=true - name: Run TPC-DS queries on collated data + if: inputs.branch != 'branch-3.5' run: | SPARK_TPCDS_DATA=`pwd`/tpcds-sf-1 build/sbt "sql/testOnly org.apache.spark.sql.TPCDSCollationQueryTestSuite" - name: Upload test results to report From b613b228892d7c450c91112f14d1ace85cbf4c0b Mon Sep 17 00:00:00 2001 From: Evan Wu Date: Mon, 6 Jan 2025 10:16:09 +0800 Subject: [PATCH 33/60] [SPARK-50703][PYTHON] Refine the docstring of regexp_replace, regexp_substr, and regexp_instr ### What changes were proposed in this pull request? Refined docstring for `regexp_replace`, `regexp_substr`, and `regexp_instr`. ### Why are the changes needed? to improve docs and test coverage ### Does this PR introduce _any_ user-facing change? doc-only changes ### How was this patch tested? doctests ### Was this patch authored or co-authored using generative AI tooling? No Closes #49348 from drexler-sky/docstring2. Authored-by: Evan Wu Signed-off-by: Ruifeng Zheng --- python/pyspark/sql/functions/builtin.py | 149 +++++++++++++++++++----- 1 file changed, 121 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/functions/builtin.py b/python/pyspark/sql/functions/builtin.py index 5d557c7277a30..0ff9b17d68a2c 100644 --- a/python/pyspark/sql/functions/builtin.py +++ b/python/pyspark/sql/functions/builtin.py @@ -15535,43 +15535,102 @@ def regexp_replace( Examples -------- - >>> df = spark.createDataFrame([("100-200", r"(\d+)", "--")], ["str", "pattern", "replacement"]) - >>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect() - [Row(d='-----')] - >>> df.select(regexp_replace("str", col("pattern"), col("replacement")).alias('d')).collect() - [Row(d='-----')] + >>> from pyspark.sql import functions as sf + >>> df = spark.createDataFrame( + ... [("100-200", r"(\d+)", "--")], + ... ["str", "pattern", "replacement"] + ... ) + + Example 1: Replaces all the substrings in the `str` column name that + match the regex pattern `(\d+)` (one or more digits) with the replacement + string "--". + + >>> df.select('*', sf.regexp_replace('str', r'(\d+)', '--')).show() + +-------+-------+-----------+---------------------------------+ + | str|pattern|replacement|regexp_replace(str, (\d+), --, 1)| + +-------+-------+-----------+---------------------------------+ + |100-200| (\d+)| --| -----| + +-------+-------+-----------+---------------------------------+ + + Example 2: Replaces all the substrings in the `str` Column that match + the regex pattern in the `pattern` Column with the string in the `replacement` + column. + + >>> df.select('*', \ + ... sf.regexp_replace(sf.col("str"), sf.col("pattern"), sf.col("replacement")) \ + ... ).show() + +-------+-------+-----------+--------------------------------------------+ + | str|pattern|replacement|regexp_replace(str, pattern, replacement, 1)| + +-------+-------+-----------+--------------------------------------------+ + |100-200| (\d+)| --| -----| + +-------+-------+-----------+--------------------------------------------+ """ return _invoke_function_over_columns("regexp_replace", string, lit(pattern), lit(replacement)) @_try_remote_functions def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: - r"""Returns the substring that matches the Java regex `regexp` within the string `str`. + r"""Returns the first substring that matches the Java regex `regexp` within the string `str`. If the regular expression is not found, the result is null. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. Returns ------- :class:`~pyspark.sql.Column` - the substring that matches a Java regex within the string `str`. + the first substring that matches a Java regex within the string `str`. Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("1a 2b 14m", r"\d+")], ["str", "regexp"]) - >>> df.select(regexp_substr('str', lit(r'\d+')).alias('d')).collect() - [Row(d='1')] - >>> df.select(regexp_substr('str', lit(r'mmm')).alias('d')).collect() - [Row(d=None)] - >>> df.select(regexp_substr("str", col("regexp")).alias('d')).collect() - [Row(d='1')] + + Example 1: Returns the first substring in the `str` column name that + matches the regex pattern `(\d+)` (one or more digits). + + >>> df.select('*', sf.regexp_substr('str', sf.lit(r'\d+'))).show() + +---------+------+-----------------------+ + | str|regexp|regexp_substr(str, \d+)| + +---------+------+-----------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+-----------------------+ + + Example 2: Returns the first substring in the `str` column name that + matches the regex pattern `(mmm)` (three consecutive 'm' characters) + + >>> df.select('*', sf.regexp_substr('str', sf.lit(r'mmm'))).show() + +---------+------+-----------------------+ + | str|regexp|regexp_substr(str, mmm)| + +---------+------+-----------------------+ + |1a 2b 14m| \d+| NULL| + +---------+------+-----------------------+ + + Example 3: Returns the first substring in the `str` column name that + matches the regex pattern in `regexp` Column. + + >>> df.select('*', sf.regexp_substr("str", sf.col("regexp"))).show() + +---------+------+--------------------------+ + | str|regexp|regexp_substr(str, regexp)| + +---------+------+--------------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+--------------------------+ + + Example 4: Returns the first substring in the `str` Column that + matches the regex pattern in `regexp` column name. + + >>> df.select('*', sf.regexp_substr(sf.col("str"), "regexp")).show() + +---------+------+--------------------------+ + | str|regexp|regexp_substr(str, regexp)| + +---------+------+--------------------------+ + |1a 2b 14m| \d+| 1| + +---------+------+--------------------------+ """ return _invoke_function_over_columns("regexp_substr", str, regexp) @@ -15580,36 +15639,70 @@ def regexp_substr(str: "ColumnOrName", regexp: "ColumnOrName") -> Column: def regexp_instr( str: "ColumnOrName", regexp: "ColumnOrName", idx: Optional[Union[int, Column]] = None ) -> Column: - r"""Extract all strings in the `str` that match the Java regex `regexp` + r"""Returns the position of the first substring in the `str` that match the Java regex `regexp` and corresponding to the regex group index. .. versionadded:: 3.5.0 Parameters ---------- - str : :class:`~pyspark.sql.Column` or str + str : :class:`~pyspark.sql.Column` or column name target column to work on. - regexp : :class:`~pyspark.sql.Column` or str + regexp : :class:`~pyspark.sql.Column` or column name regex pattern to apply. - idx : int, optional + idx : :class:`~pyspark.sql.Column` or int, optional matched group id. Returns ------- :class:`~pyspark.sql.Column` - all strings in the `str` that match a Java regex and corresponding to the regex group index. + the position of the first substring in the `str` that match a Java regex and corresponding + to the regex group index. Examples -------- + >>> from pyspark.sql import functions as sf >>> df = spark.createDataFrame([("1a 2b 14m", r"\d+(a|b|m)")], ["str", "regexp"]) - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)')).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)'), 1).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', lit(r'\d+(a|b|m)'), 2).alias('d')).collect() - [Row(d=1)] - >>> df.select(regexp_instr('str', col("regexp")).alias('d')).collect() - [Row(d=1)] + + Example 1: Returns the position of the first substring in the `str` column name that + match the regex pattern `(\d+(a|b|m))` (one or more digits followed by 'a', 'b', or 'm'). + + >>> df.select('*', sf.regexp_instr('str', sf.lit(r'\d+(a|b|m)'))).show() + +---------+----------+--------------------------------+ + | str| regexp|regexp_instr(str, \d+(a|b|m), 0)| + +---------+----------+--------------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+--------------------------------+ + + Example 2: Returns the position of the first substring in the `str` column name that + match the regex pattern `(\d+(a|b|m))` (one or more digits followed by 'a', 'b', or 'm'), + + >>> df.select('*', sf.regexp_instr('str', sf.lit(r'\d+(a|b|m)'), sf.lit(1))).show() + +---------+----------+--------------------------------+ + | str| regexp|regexp_instr(str, \d+(a|b|m), 1)| + +---------+----------+--------------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+--------------------------------+ + + Example 3: Returns the position of the first substring in the `str` column name that + match the regex pattern in `regexp` Column. + + >>> df.select('*', sf.regexp_instr('str', sf.col("regexp"))).show() + +---------+----------+----------------------------+ + | str| regexp|regexp_instr(str, regexp, 0)| + +---------+----------+----------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+----------------------------+ + + Example 4: Returns the position of the first substring in the `str` Column that + match the regex pattern in `regexp` column name. + + >>> df.select('*', sf.regexp_instr(sf.col("str"), "regexp")).show() + +---------+----------+----------------------------+ + | str| regexp|regexp_instr(str, regexp, 0)| + +---------+----------+----------------------------+ + |1a 2b 14m|\d+(a|b|m)| 1| + +---------+----------+----------------------------+ """ if idx is None: return _invoke_function_over_columns("regexp_instr", str, regexp) From 3ecb2908dea68b96a0ec8a00ffd69ad872a6d42a Mon Sep 17 00:00:00 2001 From: cashmand Date: Mon, 6 Jan 2025 10:32:54 +0800 Subject: [PATCH 34/60] [SPARK-50614][SQL] Revert API change for shredding ### What changes were proposed in this pull request? https://github.com/apache/spark/pull/49234 changed the `setSchema` method of ParquetWriteSupport to add a shredding schema parameter. To avoid the risk of breaking libraries that call `setSchema`, this PR instead creates a separate `setShreddingSchema` method to set the shredding schema. If not called, shredding will not be used. ### Why are the changes needed? Avoid breaking API. ### Does this PR introduce _any_ user-facing change? No, the feature has not been released yet. ### How was this patch tested? Existing unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49353 from cashmand/fix_set_schema. Authored-by: cashmand Signed-off-by: Wenchen Fan --- .../execution/datasources/parquet/ParquetUtils.scala | 3 ++- .../datasources/parquet/ParquetWriteSupport.scala | 10 +++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala index a609a4e0a25f3..663182d8d1820 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -485,7 +485,8 @@ object ParquetUtils extends Logging { } // This metadata is useful for keeping UDTs like Vector/Matrix. - ParquetWriteSupport.setSchema(dataSchema, conf, shreddingSchema) + ParquetWriteSupport.setSchema(dataSchema, conf) + shreddingSchema.foreach(ParquetWriteSupport.setShreddingSchema(_, conf)) // Sets flags for `ParquetWriteSupport`, which converts Catalyst schema to Parquet // schema and writes actual rows to Parquet files. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala index 02b432f98d7d8..35eb57a2e4fb2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala @@ -539,14 +539,14 @@ object ParquetWriteSupport { val SPARK_VARIANT_SHREDDING_SCHEMA: String = "org.apache.spark.sql.parquet.variant.shredding.attributes" - def setSchema(schema: StructType, configuration: Configuration, - shreddingSchema: Option[StructType]): Unit = { + def setSchema(schema: StructType, configuration: Configuration): Unit = { configuration.set(SPARK_ROW_SCHEMA, schema.json) configuration.setIfUnset( ParquetOutputFormat.WRITER_VERSION, ParquetProperties.WriterVersion.PARQUET_1_0.toString) - shreddingSchema.foreach { s => - configuration.set(SPARK_VARIANT_SHREDDING_SCHEMA, s.json) - } + } + + def setShreddingSchema(shreddingSchema: StructType, configuration: Configuration): Unit = { + configuration.set(SPARK_VARIANT_SHREDDING_SCHEMA, shreddingSchema.json) } } From 1793a20dbdcb10eab4fe2eecbba4ebc5258aa637 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 6 Jan 2025 11:28:51 +0800 Subject: [PATCH 35/60] [SPARK-50700][SQL] `spark.sql.catalog.spark_catalog` supports `builtin` magic value ### What changes were proposed in this pull request? This PR adds a magic value `builtin`(and sets it to the default value) for `spark.sql.catalog.spark_catalog`. ### Why are the changes needed? Currently, `spark.sql.catalog.spark_catalog` is optional and has `None` as the default value. When `spark.sql.catalog.spark_catalog=a.bad.catalog.impl` is wrongly set in `spark-defaults.conf`, the user has no way to overwrite it in `spark-submit`. Note that, explicitly setting it to `o.a.s.sql.execution.datasources.v2.V2SessionCatalog` does not work either, because `V2SessionCatalog` does not have a zero-args constructor. ``` spark-submit \ --conf spark.sql.catalog.spark_catalog=org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog \ ... ``` To fix the above issue, similar to what we did for `spark.sql.hive.metastore.jars`, just use "builtin" to represent the built-in `V2SessionCatalog`. ### Does this PR introduce _any_ user-facing change? No change for default behavior, and users are allowed to use `spark.sql.catalog.spark_catalog=builtin` to set `spark_catalog` as the built-in `V2SessionCatalog`. ### How was this patch tested? Code in UTs like ``` // unset this config to use the default v2 session catalog. spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) ``` are replaced with ``` spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49332 from pan3793/SPARK-50700. Authored-by: Cheng Pan Signed-off-by: Kent Yao --- .../connector/catalog/CatalogManager.scala | 7 ++-- .../apache/spark/sql/internal/SQLConf.scala | 10 ++++- .../analysis/ResolveSessionCatalog.scala | 2 +- .../datasources/v2/V2SessionCatalog.scala | 2 +- .../sql/internal/DataFrameWriterImpl.scala | 8 +++- .../sql/connector/DataSourceV2SQLSuite.scala | 42 +++++++++---------- .../spark/sql/connector/DeleteFromTests.scala | 4 +- 7 files changed, 43 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala index db94659b1033b..9b8584604d32f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/connector/catalog/CatalogManager.scala @@ -85,9 +85,10 @@ class CatalogManager( * in the fallback configuration, spark.sql.sources.useV1SourceList */ private[sql] def v2SessionCatalog: CatalogPlugin = { - conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).map { _ => - catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) - }.getOrElse(defaultSessionCatalog) + conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) match { + case "builtin" => defaultSessionCatalog + case _ => catalogs.getOrElseUpdate(SESSION_CATALOG_NAME, loadV2SessionCatalog()) + } } private var _currentNamespace: Option[Array[String]] = None 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 6a45380b7a99d..7bc4051b45d4c 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 @@ -4404,10 +4404,16 @@ object SQLConf { s"the $SESSION_CATALOG_NAME and must be consistent with it; for example, if a table can " + s"be loaded by the $SESSION_CATALOG_NAME, this catalog must also return the table " + s"metadata. To delegate operations to the $SESSION_CATALOG_NAME, implementations can " + - "extend 'CatalogExtension'.") + "extend 'CatalogExtension'. The value should be either 'builtin' which represents the " + + "spark's builit-in V2SessionCatalog, or a fully qualified class name of the catalog " + + "implementation.") .version("3.0.0") .stringConf - .createOptional + .transform { + case builtin if builtin.equalsIgnoreCase("builtin") => "builtin" + case fullClassName => fullClassName + } + .createWithDefault("builtin") object MapKeyDedupPolicy extends Enumeration { val EXCEPTION, LAST_WIN = Value diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index fa28a2cb9ead6..87ea3071f490b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -726,7 +726,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) private def supportsV1Command(catalog: CatalogPlugin): Boolean = { isSessionCatalog(catalog) && ( - SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isEmpty || + SQLConf.get.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) == "builtin" || catalog.isInstanceOf[CatalogExtension]) } } 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 e9927cdcc7a33..0a533645648e6 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 @@ -83,7 +83,7 @@ class V2SessionCatalog(catalog: SessionCatalog) } private def hasCustomSessionCatalog: Boolean = { - catalog.conf.contains(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key) + catalog.conf.getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) != "builtin" } override def loadTable(ident: Identifier): Table = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala index 0069062e63078..5a96db5e34bbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/DataFrameWriterImpl.scala @@ -383,6 +383,11 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram } } + private def hasCustomSessionCatalog: Boolean = { + df.sparkSession.sessionState.conf + .getConf(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION) != "builtin" + } + /** * Saves the content of the `DataFrame` as the specified table. * @@ -426,8 +431,7 @@ final class DataFrameWriterImpl[T] private[sql](ds: Dataset[T]) extends DataFram import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ val session = df.sparkSession - val canUseV2 = lookupV2Provider().isDefined || (df.sparkSession.sessionState.conf.getConf( - SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION).isDefined && + val canUseV2 = lookupV2Provider().isDefined || (hasCustomSessionCatalog && !df.sparkSession.sessionState.catalogManager.catalog(CatalogManager.SESSION_CATALOG_NAME) .isInstanceOf[CatalogExtension]) 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 87d0a1ff4e7bf..8d255e9efda54 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 @@ -278,8 +278,8 @@ class DataSourceV2SQLSuiteV1Filter test("CreateTable: without USING clause") { withSQLConf(SQLConf.LEGACY_CREATE_HIVE_TABLE_BY_DEFAULT.key -> "false") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val testCatalog = catalog("testcat").asTableCatalog sql("CREATE TABLE testcat.t1 (id int)") @@ -785,8 +785,8 @@ class DataSourceV2SQLSuiteV1Filter } test("CreateTableAsSelect: v2 session catalog can load v1 source table") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") df.createOrReplaceTempView("source") @@ -846,8 +846,8 @@ class DataSourceV2SQLSuiteV1Filter // TODO: ignored by SPARK-31707, restore the test after create table syntax unification ignore("CreateTableAsSelect: without USING clause") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val testCatalog = catalog("testcat").asTableCatalog sql("CREATE TABLE testcat.t1 AS SELECT 1 i") @@ -1086,11 +1086,11 @@ class DataSourceV2SQLSuiteV1Filter Seq(true, false).foreach { useV1Table => val format = if (useV1Table) "json" else v2Format if (useV1Table) { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") } else { spark.conf.set( - V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[InMemoryTableSessionCatalog].getName) + V2_SESSION_CATALOG_IMPLEMENTATION, classOf[InMemoryTableSessionCatalog].getName) } withTable("t") { @@ -1815,8 +1815,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-46972: asymmetrical replacement for char/varchar in V2SessionCatalog.createTable") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("t") { sql(s"CREATE TABLE t(c char(1), v varchar(2)) USING $v2Source") } @@ -2533,8 +2533,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30001: session catalog name can be specified in SQL statements") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("t") { sql("CREATE TABLE t USING json AS SELECT 1 AS i") @@ -2598,8 +2598,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30094: current namespace is used during table resolution") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") withTable("spark_catalog.default.t", "testcat.ns.t") { sql("CREATE TABLE t USING parquet AS SELECT 1") @@ -2613,8 +2613,8 @@ class DataSourceV2SQLSuiteV1Filter } test("SPARK-30284: CREATE VIEW should track the current catalog and namespace") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val sessionCatalogName = CatalogManager.SESSION_CATALOG_NAME sql("CREATE NAMESPACE testcat.ns1.ns2") @@ -2651,8 +2651,8 @@ class DataSourceV2SQLSuiteV1Filter } test("COMMENT ON NAMESPACE") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") // Session catalog is used. sql("CREATE NAMESPACE ns") checkNamespaceComment("ns", "minor revision") @@ -2685,8 +2685,8 @@ class DataSourceV2SQLSuiteV1Filter } test("COMMENT ON TABLE") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") // Session catalog is used. withTable("t") { sql("CREATE TABLE t(k int) USING json") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala index fd022580db42b..26f64ceb33fe3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/connector/DeleteFromTests.scala @@ -100,8 +100,8 @@ trait DeleteFromTests extends DatasourceV2SQLBase { } test("DeleteFrom: DELETE is only supported with v2 tables") { - // unset this config to use the default v2 session catalog. - spark.conf.unset(V2_SESSION_CATALOG_IMPLEMENTATION.key) + // use the default v2 session catalog. + spark.conf.set(V2_SESSION_CATALOG_IMPLEMENTATION, "builtin") val v1Table = "tbl" withTable(v1Table) { sql(s"CREATE TABLE $v1Table" + From aed5d110c2aabe8e6176c64f93d4a91bf07acc62 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 11:30:26 +0800 Subject: [PATCH 36/60] [SPARK-50731][SQL][TESTS] Mark `*TransformWithStateSuite` as `SlowSQLTest` ### What changes were proposed in this pull request? This PR aims to mark `*TransformWithStateSuite` as `SlowSQLTest`. For example, `StateDataSourceTransformWithStateSuite` took almost 2 minutes. ``` 54950 2025-01-05T19:48:22.9043468Z [info] StateDataSourceTransformWithStateSuite: ... 56139 2025-01-05T19:50:15.1404546Z [info] BloomFilterAggregateQuerySuite: ``` ### Why are the changes needed? To balance the test pipelines by moving newly added long-running test suites. Currently, - https://github.com/apache/spark/actions/runs/12613864071/job/35152182685 (`sql - other tests` took 1h 38 mins) - https://github.com/apache/spark/actions/runs/12613864071/job/35152182636 (`sql - slow tests` took 47 mins) ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manually check the CI logs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49361 from dongjoon-hyun/SPARK-50731. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../v2/state/StateDataSourceTransformWithStateSuite.scala | 2 ++ .../apache/spark/sql/streaming/TransformWithStateSuite.scala | 2 ++ 2 files changed, 4 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala index b402344a73177..fe224608fffd8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/state/StateDataSourceTransformWithStateSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.functions.{col, explode, timestamp_seconds} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{InputMapRow, ListState, MapInputEvent, MapOutputEvent, MapStateTTLProcessor, MaxEventTimeStatefulProcessor, OutputMode, RunningCountStatefulProcessor, RunningCountStatefulProcessorWithProcTimeTimerUpdates, StatefulProcessor, StateStoreMetricsTest, TestMapStateProcessor, TimeMode, TimerValues, TransformWithStateSuiteUtils, Trigger, TTLConfig, ValueState} import org.apache.spark.sql.streaming.util.StreamManualClock +import org.apache.spark.tags.SlowSQLTest import org.apache.spark.util.Utils /** Stateful processor of single value state var with non-primitive type */ @@ -125,6 +126,7 @@ class SessionGroupsStatefulProcessorWithTTL extends /** * Test suite to verify integration of state data source reader with the transformWithState operator */ +@SlowSQLTest class StateDataSourceTransformWithStateSuite extends StateStoreMetricsTest with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { 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 601699c8320fb..97dad5fe78a19 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 @@ -38,6 +38,7 @@ import org.apache.spark.sql.functions.timestamp_seconds import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType} +import org.apache.spark.tags.SlowSQLTest object TransformWithStateSuiteUtils { val NUM_SHUFFLE_PARTITIONS = 5 @@ -441,6 +442,7 @@ class SleepingTimerProcessor extends StatefulProcessor[String, String, String] { /** * Class that adds tests for transformWithState stateful streaming operator */ +@SlowSQLTest class TransformWithStateSuite extends StateStoreMetricsTest with AlsoTestWithRocksDBFeatures with AlsoTestWithEncodingTypes { From 7ebd3950008694f5ed85420fbe620d3e2247bb90 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 11:33:23 +0800 Subject: [PATCH 37/60] [SPARK-50729][SQL][TESTS] Remove `DebugFilesystem` from `TPCDS(Collation)?QueryTestSuite` ### What changes were proposed in this pull request? This PR aims to remove `DebugFilesystem` from `TPCDS(Collation)?QueryTestSuite` while keeping it in all other places. ### Why are the changes needed? `TPCDS(Collation)?QueryTestSuite` extends `TPCDSBase` and `TPCHBase` respectively and they are based on `TPCBase` which extends `SharedSparkSession`. https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/sql/core/src/test/scala/org/apache/spark/sql/TPCBase.scala#L24 And, `SharedSparkSession` always uses `DebugFilesystem`. https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala#L68 This PR aims to remove this resetting. ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manual review. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49359 from dongjoon-hyun/SPARK-50729. Authored-by: Dongjoon Hyun Signed-off-by: Kent Yao --- .../org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala | 1 + .../test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala | 1 + 2 files changed, 2 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala index 46a24acb475c4..43e6111fc99ca 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSCollationQueryTestSuite.scala @@ -67,6 +67,7 @@ class TPCDSCollationQueryTestSuite extends QueryTest with TPCDSBase with SQLQuer // To make output results deterministic override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + .remove("spark.hadoop.fs.file.impl") protected override def createSparkSession: TestSparkSession = { new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala index bde6155529872..c1246a167b8cc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQueryTestSuite.scala @@ -62,6 +62,7 @@ class TPCDSQueryTestSuite extends QueryTest with TPCDSBase with SQLQueryTestHelp // To make output results deterministic override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.SHUFFLE_PARTITIONS.key, "1") + .remove("spark.hadoop.fs.file.impl") protected override def createSparkSession: TestSparkSession = { new TestSparkSession(new SparkContext("local[1]", this.getClass.getSimpleName, sparkConf)) From 2cc45c2d027301275b19f8e79d1f714c95d8a28d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 5 Jan 2025 21:38:35 -0800 Subject: [PATCH 38/60] [SPARK-50733][K8S][INFRA] Update K8s IT CI to use K8s 1.32 ### What changes were proposed in this pull request? As a part of Apache Spark 4.0.0 preparation, this PR aims to use the latest K8s v1.32.0 (2024-12-11) for K8s integration tests instead of v1.31.0 (August 13, 2024). ### Why are the changes needed? K8s v1.32 was released on last December. - https://kubernetes.io/releases/#release-v1-32 Previously, we has been depending on the default version of Minikube because Minikube has a rapid release cycle. However, the release cycle of Minikube is a little slow in these days. We had better control the test target K8s version explicitly for now. ### Does this PR introduce _any_ user-facing change? No, this is a infra-only change. ### How was this patch tested? Pass the CIs and check the K8s CI log. The following is the current log of this PR. ``` Kubelet Version: v1.32.0 Kube-Proxy Version: v1.32.0 ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49364 from dongjoon-hyun/k8s_1.32. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .github/workflows/build_and_test.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 29621930b1ffa..3d08c338f053e 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -1226,6 +1226,7 @@ jobs: - name: Start Minikube uses: medyagh/setup-minikube@v0.0.18 with: + kubernetes-version: "1.32.0" # Github Action limit cpu:2, memory: 6947MB, limit to 2U6G for better resource statistic cpus: 2 memory: 6144m From 54b4c26504c321ee30e65de7d674c2b3ad99ff7e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 5 Jan 2025 21:43:02 -0800 Subject: [PATCH 39/60] [SPARK-50732][SQL][TESTS] Mark `DataFrameAggregateSuite` and its derived classes as `SlowSQLTest` ### What changes were proposed in this pull request? This PR aims to mark `DataFrameAggregateSuite` and its derived classes as `SlowSQLTest`. ``` $ git grep DataFrameAggregateSuite sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala:class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala:class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala: extends DataFrameAggregateSuite sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala:class DataFrameAggregateSuite extends QueryTest ``` ### Why are the changes needed? When we are adding more tests into `DataFrameAggregateSuite`, its derived three classes also grow together. To balance the test pipelines by moving this long-running test suite group. The following is the AS-IS run time before this PR. Screenshot 2025-01-05 at 20 58 15 ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Manually check the CI logs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49363 from dongjoon-hyun/SPARK-50732. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../scala/org/apache/spark/sql/AggregateHashMapSuite.scala | 4 ++++ .../scala/org/apache/spark/sql/DataFrameAggregateSuite.scala | 2 ++ 2 files changed, 6 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala index b253c4a70bbf9..e1b0676831549 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -21,7 +21,9 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.tags.SlowSQLTest +@SlowSQLTest class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.CODEGEN_FALLBACK.key, "false") @@ -37,6 +39,7 @@ class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with Befo } } +@SlowSQLTest class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf .set(SQLConf.CODEGEN_FALLBACK.key, "false") @@ -52,6 +55,7 @@ class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeA } } +@SlowSQLTest class TwoLevelAggregateHashMapWithVectorizedMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index ad80dc65926bc..141d6b219f2a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -39,10 +39,12 @@ import org.apache.spark.sql.test.SQLTestData.DecimalData import org.apache.spark.sql.types._ import org.apache.spark.sql.types.DayTimeIntervalType.{DAY, HOUR, MINUTE, SECOND} import org.apache.spark.sql.types.YearMonthIntervalType.{MONTH, YEAR} +import org.apache.spark.tags.SlowSQLTest import org.apache.spark.unsafe.types.CalendarInterval case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double) +@SlowSQLTest class DataFrameAggregateSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { From 56284bfb1e894f5f80b53b1dd528090f9a9b3427 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Mon, 6 Jan 2025 15:00:11 +0900 Subject: [PATCH 40/60] [SPARK-48459][FOLLOWUP][MINOR] Cleanup unused global variable ### What changes were proposed in this pull request? This PR followups for https://github.com/apache/spark/pull/46789 to remove unused global variable ### Why are the changes needed? Code cleanup ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? The existing CI should pass ### Was this patch authored or co-authored using generative AI tooling? No Closes #49365 from itholic/minor_48459. Authored-by: Haejoon Lee Signed-off-by: Hyukjin Kwon --- python/pyspark/errors/utils.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/pyspark/errors/utils.py b/python/pyspark/errors/utils.py index 5488940645a13..0d01cbb961bb6 100644 --- a/python/pyspark/errors/utils.py +++ b/python/pyspark/errors/utils.py @@ -259,8 +259,6 @@ def wrapper(*args: Any, **kwargs: Any) -> Any: if spark is not None and hasattr(func, "__name__") and is_debugging_enabled(): if is_remote(): - global current_origin - # Getting the configuration requires RPC call. Uses the default value for now. depth = 1 set_current_origin(func.__name__, _capture_call_site(depth)) From 459483acb45e44592c81e0f449c09c4607a680a4 Mon Sep 17 00:00:00 2001 From: Haejoon Lee Date: Mon, 6 Jan 2025 16:33:19 +0900 Subject: [PATCH 41/60] [SPARK-50357][PYTHON] Support Interrupt(Tag|All) APIs for PySpark ### What changes were proposed in this pull request? This PR proposes to support `Interrupt(Tag|All)` for PySpark ### Why are the changes needed? To improve the compatibility between Spark Connect and Spark Classic. ### Does this PR introduce _any_ user-facing change? New APIs are added - InterruptTag - InterruptAll ### How was this patch tested? Added UTs ### Was this patch authored or co-authored using generative AI tooling? No Closes #49014 from itholic/SPARK-50357. Authored-by: Haejoon Lee Signed-off-by: Haejoon Lee --- .../reference/pyspark.sql/spark_session.rst | 4 +-- python/pyspark/sql/session.py | 34 +++++++++++++------ .../connect/test_parity_job_cancellation.py | 22 ------------ .../sql/tests/test_connect_compatibility.py | 2 -- .../sql/tests/test_job_cancellation.py | 22 ++++++++++++ python/pyspark/sql/tests/test_session.py | 1 - 6 files changed, 48 insertions(+), 37 deletions(-) diff --git a/python/docs/source/reference/pyspark.sql/spark_session.rst b/python/docs/source/reference/pyspark.sql/spark_session.rst index 1677d3e8e0209..a35fccbcffe99 100644 --- a/python/docs/source/reference/pyspark.sql/spark_session.rst +++ b/python/docs/source/reference/pyspark.sql/spark_session.rst @@ -52,6 +52,8 @@ See also :class:`SparkSession`. SparkSession.dataSource SparkSession.getActiveSession SparkSession.getTags + SparkSession.interruptAll + SparkSession.interruptTag SparkSession.newSession SparkSession.profile SparkSession.removeTag @@ -86,8 +88,6 @@ Spark Connect Only SparkSession.clearProgressHandlers SparkSession.client SparkSession.copyFromLocalToFs - SparkSession.interruptAll SparkSession.interruptOperation - SparkSession.interruptTag SparkSession.registerProgressHandler SparkSession.removeProgressHandler diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index f3a1639fddafa..fc434cd16bfbd 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -2197,13 +2197,15 @@ def copyFromLocalToFs(self, local_path: str, dest_path: str) -> None: messageParameters={"feature": "SparkSession.copyFromLocalToFs"}, ) - @remote_only def interruptAll(self) -> List[str]: """ Interrupt all operations of this session currently running on the connected server. .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- list of str @@ -2213,18 +2215,25 @@ def interruptAll(self) -> List[str]: ----- There is still a possibility of operation finishing just as it is interrupted. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.interruptAll"}, - ) + java_list = self._jsparkSession.interruptAll() + python_list = list() + + # Use iterator to manually iterate through Java list + java_iterator = java_list.iterator() + while java_iterator.hasNext(): + python_list.append(str(java_iterator.next())) + + return python_list - @remote_only def interruptTag(self, tag: str) -> List[str]: """ Interrupt all operations of this session with the given operation tag. .. versionadded:: 3.5.0 + .. versionchanged:: 4.0.0 + Supports Spark Classic. + Returns ------- list of str @@ -2234,10 +2243,15 @@ def interruptTag(self, tag: str) -> List[str]: ----- There is still a possibility of operation finishing just as it is interrupted. """ - raise PySparkRuntimeError( - errorClass="ONLY_SUPPORTED_WITH_SPARK_CONNECT", - messageParameters={"feature": "SparkSession.interruptTag"}, - ) + java_list = self._jsparkSession.interruptTag(tag) + python_list = list() + + # Use iterator to manually iterate through Java list + java_iterator = java_list.iterator() + while java_iterator.hasNext(): + python_list.append(str(java_iterator.next())) + + return python_list @remote_only def interruptOperation(self, op_id: str) -> List[str]: diff --git a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py index c5184b04d6aa5..ddb4554afa55a 100644 --- a/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py +++ b/python/pyspark/sql/tests/connect/test_parity_job_cancellation.py @@ -32,28 +32,6 @@ def func(target): create_thread=lambda target, session: threading.Thread(target=func, args=(target,)) ) - def test_interrupt_tag(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: self.spark.addTag(job_group), - lambda job_group: self.spark.interruptTag(job_group), - thread_ids, - [i for i in thread_ids if i % 2 == 0], - [i for i in thread_ids if i % 2 != 0], - ) - self.spark.clearTags() - - def test_interrupt_all(self): - thread_ids = range(4) - self.check_job_cancellation( - lambda job_group: None, - lambda job_group: self.spark.interruptAll(), - thread_ids, - thread_ids, - [], - ) - self.spark.clearTags() - if __name__ == "__main__": import unittest diff --git a/python/pyspark/sql/tests/test_connect_compatibility.py b/python/pyspark/sql/tests/test_connect_compatibility.py index ef83dc3834d0c..25b8be1f9ac7a 100644 --- a/python/pyspark/sql/tests/test_connect_compatibility.py +++ b/python/pyspark/sql/tests/test_connect_compatibility.py @@ -266,9 +266,7 @@ def test_spark_session_compatibility(self): "addArtifacts", "clearProgressHandlers", "copyFromLocalToFs", - "interruptAll", "interruptOperation", - "interruptTag", "newSession", "registerProgressHandler", "removeProgressHandler", diff --git a/python/pyspark/sql/tests/test_job_cancellation.py b/python/pyspark/sql/tests/test_job_cancellation.py index a046c9c01811b..3f30f78808892 100644 --- a/python/pyspark/sql/tests/test_job_cancellation.py +++ b/python/pyspark/sql/tests/test_job_cancellation.py @@ -166,6 +166,28 @@ def get_outer_local_prop(): self.assertEqual(first, {"a", "b"}) self.assertEqual(second, {"a", "b", "c"}) + def test_interrupt_tag(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: self.spark.addTag(job_group), + lambda job_group: self.spark.interruptTag(job_group), + thread_ids, + [i for i in thread_ids if i % 2 == 0], + [i for i in thread_ids if i % 2 != 0], + ) + self.spark.clearTags() + + def test_interrupt_all(self): + thread_ids = range(4) + self.check_job_cancellation( + lambda job_group: None, + lambda job_group: self.spark.interruptAll(), + thread_ids, + thread_ids, + [], + ) + self.spark.clearTags() + class JobCancellationTests(JobCancellationTestsMixin, ReusedSQLTestCase): pass diff --git a/python/pyspark/sql/tests/test_session.py b/python/pyspark/sql/tests/test_session.py index 3fbc0be943e45..a22fe777e3c9a 100644 --- a/python/pyspark/sql/tests/test_session.py +++ b/python/pyspark/sql/tests/test_session.py @@ -227,7 +227,6 @@ def test_unsupported_api(self): (lambda: session.client, "client"), (session.addArtifacts, "addArtifact(s)"), (lambda: session.copyFromLocalToFs("", ""), "copyFromLocalToFs"), - (lambda: session.interruptTag(""), "interruptTag"), (lambda: session.interruptOperation(""), "interruptOperation"), ] From b3d2f28962dbabfe6632a3c9f945bd20770184ba Mon Sep 17 00:00:00 2001 From: morvenhuang Date: Mon, 6 Jan 2025 16:24:11 +0800 Subject: [PATCH 42/60] [SPARK-50671][BUILD] Error compiling spark-protobuf module using user-defined protoc ### What changes were proposed in this pull request? People may use user-defined protoc(as below) for various reasons, for example, if they're on macOS 11, they have to use it since the default version of protoc and protoc-gen-grpc-java plugin do not work on macOS 11. ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ``` But this results in compilation failure: ``` [ERROR] Failed to execute goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run (default) on project spark-protobuf_2.13: Execution default of goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run failed: Cannot read the array length because "" is null ``` This PR aims to fix this compilation failure, the error is caused by `protocol-jar-maven-plugin` bug: - os72/protoc-jar-maven-plugin#104 ### Why are the changes needed? Before: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ... [ERROR] Failed to execute goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run (default) on project spark-protobuf_2.13: Execution default of goal com.github.os72:protoc-jar-maven-plugin:3.11.4:run failed: Cannot read the array length because "" is null ``` After: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ... [INFO] ------------------------------------------------------------------------ [INFO] Reactor Summary for Spark Project Parent POM 4.0.0-SNAPSHOT: [INFO] [INFO] Spark Project Parent POM ........................... SUCCESS [ 7.539 s] [INFO] Spark Project Tags ................................. SUCCESS [ 7.527 s] [INFO] Spark Project Sketch ............................... SUCCESS [ 8.413 s] [INFO] Spark Project Common Utils ......................... SUCCESS [ 29.923 s] [INFO] Spark Project Local DB ............................. SUCCESS [ 10.732 s] [INFO] Spark Project Networking ........................... SUCCESS [ 16.691 s] [INFO] Spark Project Shuffle Streaming Service ............ SUCCESS [ 12.653 s] [INFO] Spark Project Variant .............................. SUCCESS [ 5.262 s] [INFO] Spark Project Unsafe ............................... SUCCESS [ 16.246 s] [INFO] Spark Project Connect Shims ........................ SUCCESS [ 4.356 s] [INFO] Spark Project Launcher ............................. SUCCESS [ 8.666 s] [INFO] Spark Project Core ................................. SUCCESS [02:56 min] [INFO] Spark Project ML Local Library ..................... SUCCESS [ 30.063 s] [INFO] Spark Project GraphX ............................... SUCCESS [ 33.989 s] [INFO] Spark Project Streaming ............................ SUCCESS [ 51.897 s] [INFO] Spark Project SQL API .............................. SUCCESS [ 39.508 s] [INFO] Spark Project Catalyst ............................. SUCCESS [03:15 min] [INFO] Spark Project SQL .................................. SUCCESS [04:56 min] [INFO] Spark Project ML Library ........................... SUCCESS [02:30 min] [INFO] Spark Project Tools ................................ SUCCESS [ 4.463 s] [INFO] Spark Project Hive ................................. SUCCESS [01:33 min] [INFO] Spark Project Connect Common ....................... SUCCESS [ 55.308 s] [INFO] Spark Avro ......................................... SUCCESS [ 33.350 s] [INFO] Spark Protobuf ..................................... SUCCESS [ 38.981 s] [INFO] Spark Project REPL ................................. SUCCESS [ 21.165 s] [INFO] Spark Project Connect Server ....................... SUCCESS [ 58.033 s] [INFO] Spark Project Connect Client ....................... SUCCESS [ 59.078 s] [INFO] Spark Project Assembly ............................. SUCCESS [ 6.703 s] [INFO] Kafka 0.10+ Token Provider for Streaming ........... SUCCESS [ 20.047 s] [INFO] Spark Integration for Kafka 0.10 ................... SUCCESS [ 26.882 s] [INFO] Kafka 0.10+ Source for Structured Streaming ........ SUCCESS [ 41.397 s] [INFO] Spark Project Examples ............................. SUCCESS [ 41.308 s] [INFO] Spark Integration for Kafka 0.10 Assembly .......... SUCCESS [ 11.300 s] [INFO] ------------------------------------------------------------------------ [INFO] BUILD SUCCESS ``` ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? Run command: ``` $ export SPARK_PROTOC_EXEC_PATH=/Users/foobar/dev/protoc-4.29.1/protoc $ export CONNECT_PLUGIN_EXEC_PATH=/Users/foobar/dev/protoc-gen-grpc-java-1.59.1/protoc-gen-grpc-java $ ./build/mvn -DskipTests clean package -Puser-defined-protoc -DskipDefaultProtoc ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49296 from morvenhuang/SPARK-50671. Authored-by: morvenhuang Signed-off-by: yangjie01 --- connector/protobuf/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/connector/protobuf/pom.xml b/connector/protobuf/pom.xml index e85481ef9e1c8..22d24a7cdb62d 100644 --- a/connector/protobuf/pom.xml +++ b/connector/protobuf/pom.xml @@ -189,7 +189,7 @@ src/test/resources/protobuf - true + direct java From a1a2251ed7e4adaf275db2f2470777ec9d9b5ea1 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 6 Jan 2025 19:23:21 +0900 Subject: [PATCH 43/60] [SPARK-50738][PYTHON] Upgrade black to 23.12.1 ### What changes were proposed in this pull request? This PR proposes to upgrade `black` to 23.12.1 ### Why are the changes needed? To leverage bug fixes in `black`. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49369 from HyukjinKwon/upgrade-black. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .github/workflows/build_and_test.yml | 2 +- .github/workflows/pages.yml | 2 +- dev/create-release/spark-rm/Dockerfile | 2 +- dev/pyproject.toml | 2 +- dev/reformat-python | 2 +- dev/requirements.txt | 2 +- dev/spark-test-image/docs/Dockerfile | 2 +- dev/spark-test-image/lint/Dockerfile | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index 3d08c338f053e..a46b90b4913ee 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -743,7 +743,7 @@ jobs: python-version: '3.11' - name: Install dependencies for Python CodeGen check run: | - python3.11 -m pip install 'black==23.9.1' 'protobuf==5.29.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' + python3.11 -m pip install 'black==23.12.1' 'protobuf==5.29.1' 'mypy==1.8.0' 'mypy-protobuf==3.3.0' python3.11 -m pip list - name: Python CodeGen check for branch-3.5 if: inputs.branch == 'branch-3.5' diff --git a/.github/workflows/pages.yml b/.github/workflows/pages.yml index 637abb86b36cd..4bcc275064d3c 100644 --- a/.github/workflows/pages.yml +++ b/.github/workflows/pages.yml @@ -62,7 +62,7 @@ jobs: run: | pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow 'pandas==2.2.3' 'plotly>=4.8' 'docutils<0.18.0' \ - 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ + 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' - name: Install Ruby for documentation generation diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 190c82429e29e..7b97aba21205b 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -130,7 +130,7 @@ RUN python3.9 -m pip install --force $BASIC_PIP_PKGS unittest-xml-reporting $CON # See 'docutils<0.18.0' in SPARK-39421 RUN python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ -'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ +'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' RUN python3.9 -m pip list diff --git a/dev/pyproject.toml b/dev/pyproject.toml index f19107b3782a6..8b91943009555 100644 --- a/dev/pyproject.toml +++ b/dev/pyproject.toml @@ -27,7 +27,7 @@ testpaths = [ [tool.black] # When changing the version, we have to update # GitHub workflow version and dev/reformat-python -required-version = "23.9.1" +required-version = "23.12.1" line-length = 100 target-version = ['py39'] include = '\.pyi?$' diff --git a/dev/reformat-python b/dev/reformat-python index 46b7efc931aae..9a1199faa938e 100755 --- a/dev/reformat-python +++ b/dev/reformat-python @@ -22,7 +22,7 @@ FWDIR="$( cd "$DIR"/.. && pwd )" cd "$FWDIR" BLACK_BUILD="${PYTHON_EXECUTABLE} -m black" -BLACK_VERSION="23.9.1" +BLACK_VERSION="23.12.1" $PYTHON_EXECUTABLE -c 'import black' 2> /dev/null if [ $? -ne 0 ]; then echo "The Python library providing the 'black' module was not found. Please install Black, for example, via 'pip install black==$BLACK_VERSION'." diff --git a/dev/requirements.txt b/dev/requirements.txt index c1ae808c5e291..168025cf1db5a 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -54,7 +54,7 @@ jira>=3.5.2 PyGithub # pandas API on Spark Code formatter. -black==23.9.1 +black==23.12.1 py # Spark Connect (required) diff --git a/dev/spark-test-image/docs/Dockerfile b/dev/spark-test-image/docs/Dockerfile index 6b087f08281c3..f1e33763df468 100644 --- a/dev/spark-test-image/docs/Dockerfile +++ b/dev/spark-test-image/docs/Dockerfile @@ -85,7 +85,7 @@ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 # See 'docutils<0.18.0' in SPARK-39421 RUN python3.9 -m pip install 'sphinx==4.5.0' mkdocs 'pydata_sphinx_theme>=0.13' sphinx-copybutton nbsphinx numpydoc jinja2 markupsafe 'pyzmq<24.0.0' \ ipython ipython_genutils sphinx_plotly_directive 'numpy>=1.20.0' pyarrow pandas 'plotly>=4.8' 'docutils<0.18.0' \ - 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.9.1' \ + 'flake8==3.9.0' 'mypy==1.8.0' 'pytest==7.1.3' 'pytest-mypy-plugins==1.9.3' 'black==23.12.1' \ 'pandas-stubs==1.2.0.53' 'grpcio==1.67.0' 'grpcio-status==1.67.0' 'protobuf==5.29.1' 'grpc-stubs==1.24.11' 'googleapis-common-protos-stubs==2.2.0' \ 'sphinxcontrib-applehelp==1.0.4' 'sphinxcontrib-devhelp==1.0.2' 'sphinxcontrib-htmlhelp==2.0.1' 'sphinxcontrib-qthelp==1.0.3' 'sphinxcontrib-serializinghtml==1.1.5' \ && python3.9 -m pip cache purge diff --git a/dev/spark-test-image/lint/Dockerfile b/dev/spark-test-image/lint/Dockerfile index c663c3dac3b9f..c3ffd7ba4e4b2 100644 --- a/dev/spark-test-image/lint/Dockerfile +++ b/dev/spark-test-image/lint/Dockerfile @@ -72,7 +72,7 @@ RUN apt-get update && apt-get install -y python3.9 python3.9-distutils \ RUN curl -sS https://bootstrap.pypa.io/get-pip.py | python3.9 RUN python3.9 -m pip install \ - 'black==23.9.1' \ + 'black==23.12.1' \ 'flake8==3.9.0' \ 'googleapis-common-protos-stubs==2.2.0' \ 'grpc-stubs==1.24.11' \ From 51501dfaa7aee3b370e796de54be1b05e6432987 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Mon, 6 Jan 2025 19:23:56 +0900 Subject: [PATCH 44/60] [SPARK-50596][PYTHON][FOLLOW-UP] Pin Py4J version in dev/requirements.txt ### What changes were proposed in this pull request? This PR is a followup PR that also fixes `dev/requirementst.txt` ### Why are the changes needed? To match the Py4J version with ours. ### Does this PR introduce _any_ user-facing change? No, dev-only. ### How was this patch tested? Manually. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49368 from HyukjinKwon/SPARK-50596-followup. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- dev/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/requirements.txt b/dev/requirements.txt index 168025cf1db5a..845778f0e9b99 100644 --- a/dev/requirements.txt +++ b/dev/requirements.txt @@ -1,5 +1,5 @@ # PySpark dependencies (required) -py4j>=0.10.9.7 +py4j>=0.10.9.8 # PySpark dependencies (optional) numpy>=1.21 From 0d3f4cddf0f5acd169c8221b552836f6c088b3bc Mon Sep 17 00:00:00 2001 From: vicennial Date: Mon, 6 Jan 2025 19:33:09 +0900 Subject: [PATCH 45/60] [SPARK-50708][CONNECT] Delete Artifact resources on GC of `ArtifactManager` instance ### What changes were proposed in this pull request? Registers a `java.lang.ref.Cleaner` for `ArtifactManager`. `ArtifactManager#cleanUpResources` is further made thread-safe and swallows FileSystem exceptions in order to let the remaining cleanup go through. ### Why are the changes needed? Currently, an instance of ArtifactManager's resources isn't cleaned up automatically during its GC. Previously, Artifact Manager was only used in Spark Connect, where its lifetime was controlled by the Spark Connect Session which would manually call `ArtifactManager#cleanUpResources` when it itself is closed. With the recent changes allowing the artifact manager to be used in 'Classic Spark', we should GC related resources when the SparkSession gets GCed (and thus, the ArtifactManager instance is no longer reachable) ### 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 #49341 from vicennial/artifactGC. Authored-by: vicennial Signed-off-by: Hyukjin Kwon --- .../CheckConnectJvmClientCompatibility.scala | 4 + .../sql/connect/service/SessionHolder.scala | 2 +- .../spark/sql/artifact/ArtifactManager.scala | 105 +++++++++++++----- .../sql/artifact/ArtifactManagerSuite.scala | 6 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../sql/hive/test/TestHiveSingleton.scala | 2 +- 6 files changed, 89 insertions(+), 32 deletions(-) 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 4ec84a4087eb7..be659a0fa95ba 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 @@ -183,6 +183,10 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.ExtendedExplainGenerator"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataSourceRegistration"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.artifact.ArtifactStateForCleanup"), + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.artifact.ArtifactStateForCleanup$"), // DataFrameNaFunctions ProblemFilters.exclude[Problem]("org.apache.spark.sql.DataFrameNaFunctions.fillValue"), diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala index 5dced7acfb0d2..36f3bcd1e6cd7 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SessionHolder.scala @@ -301,7 +301,7 @@ case class SessionHolder(userId: String, sessionId: String, session: SparkSessio // Clean up all artifacts. // Note: there can be concurrent AddArtifact calls still adding something. - artifactManager.cleanUpResources() + artifactManager.close() // Clean up running streaming queries. // Note: there can be concurrent streaming queries being started. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala index 65f361cbae3f2..6394cef9fc760 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/artifact/ArtifactManager.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql.artifact -import java.io.File +import java.io.{File, IOException} +import java.lang.ref.Cleaner import java.net.{URI, URL, URLClassLoader} import java.nio.ByteBuffer import java.nio.file.{CopyOption, Files, Path, Paths, StandardCopyOption} @@ -30,8 +31,8 @@ import scala.reflect.ClassTag import org.apache.commons.io.{FilenameUtils, FileUtils} import org.apache.hadoop.fs.{LocalFileSystem, Path => FSPath} -import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkEnv, SparkException, SparkUnsupportedOperationException} -import org.apache.spark.internal.Logging +import org.apache.spark.{JobArtifactSet, JobArtifactState, SparkContext, SparkEnv, SparkException, SparkUnsupportedOperationException} +import org.apache.spark.internal.{Logging, LogKeys, MDC} import org.apache.spark.internal.config.{CONNECT_SCALA_UDF_STUB_PREFIXES, EXECUTOR_USER_CLASS_PATH_FIRST} import org.apache.spark.sql.{Artifact, SparkSession} import org.apache.spark.sql.internal.SQLConf @@ -51,7 +52,7 @@ import org.apache.spark.util.{ChildFirstURLClassLoader, StubClassLoader, Utils} * * @param session The object used to hold the Spark Connect session state. */ -class ArtifactManager(session: SparkSession) extends Logging { +class ArtifactManager(session: SparkSession) extends AutoCloseable with Logging { import ArtifactManager._ // The base directory where all artifacts are stored. @@ -373,34 +374,28 @@ class ArtifactManager(session: SparkSession) extends Logging { newArtifactManager } + private val cleanUpStateForGlobalResources = ArtifactStateForCleanup( + session.sessionUUID, + session.sparkContext, + state, + artifactPath) + // Ensure that no reference to `this` is captured/help by the cleanup lambda + private def getCleanable: Cleaner.Cleanable = cleaner.register( + this, + () => ArtifactManager.cleanUpGlobalResources(cleanUpStateForGlobalResources) + ) + private var cleanable = getCleanable + /** * Cleans up all resources specific to this `session`. */ - private[sql] def cleanUpResources(): Unit = { + private def cleanUpResources(): Unit = { logDebug( s"Cleaning up resources for session with sessionUUID ${session.sessionUUID}") - // Clean up added files - val fileserver = SparkEnv.get.rpcEnv.fileServer - val sparkContext = session.sparkContext - if (state != null) { - val shouldUpdateEnv = sparkContext.addedFiles.contains(state.uuid) || - sparkContext.addedArchives.contains(state.uuid) || - sparkContext.addedJars.contains(state.uuid) - if (shouldUpdateEnv) { - sparkContext.addedFiles.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeFile)) - sparkContext.addedArchives.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeFile)) - sparkContext.addedJars.remove(state.uuid).foreach(_.keys.foreach(fileserver.removeJar)) - sparkContext.postEnvironmentUpdate() - } - } - - // Clean up cached relations - val blockManager = sparkContext.env.blockManager - blockManager.removeCache(session.sessionUUID) - - // Clean up artifacts folder - FileUtils.deleteDirectory(artifactPath.toFile) + // Clean up global resources via the Cleaner process. + // Note that this will only be run once per instance. + cleanable.clean() // Clean up internal trackers jarsList.clear() @@ -412,6 +407,17 @@ class ArtifactManager(session: SparkSession) extends Logging { cachedClassLoader = None } + override def close(): Unit = { + cleanUpResources() + } + + private[sql] def cleanUpResourcesForTesting(): Unit = { + cleanUpResources() + // Tests reuse the same instance so we need to re-register the cleanable otherwise, it is run + // only once per instance. + cleanable = getCleanable + } + def uploadArtifactToFs( remoteRelativePath: Path, serverLocalStagingPath: Path): Unit = { @@ -481,4 +487,51 @@ object ArtifactManager extends Logging { throw SparkException.internalError(s"Block $fromId not found in the block manager.") } } + + // Shared cleaner instance + private val cleaner: Cleaner = Cleaner.create() + + /** + * Helper method to clean up global resources (i.e. resources associated with the calling + * instance but held externally in sparkContext, blockManager, disk etc.) + */ + private def cleanUpGlobalResources(cleanupState: ArtifactStateForCleanup): Unit = { + // Clean up added files + val (sparkSessionUUID, sparkContext, state, artifactPath) = ( + cleanupState.sparkSessionUUID, + cleanupState.sparkContext, + cleanupState.jobArtifactState, + cleanupState.artifactPath) + val fileServer = SparkEnv.get.rpcEnv.fileServer + if (state != null) { + val shouldUpdateEnv = sparkContext.addedFiles.contains(state.uuid) || + sparkContext.addedArchives.contains(state.uuid) || + sparkContext.addedJars.contains(state.uuid) + if (shouldUpdateEnv) { + sparkContext.addedFiles.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeFile)) + sparkContext.addedArchives.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeFile)) + sparkContext.addedJars.remove(state.uuid).foreach(_.keys.foreach(fileServer.removeJar)) + sparkContext.postEnvironmentUpdate() + } + } + + // Clean up cached relations + val blockManager = sparkContext.env.blockManager + blockManager.removeCache(sparkSessionUUID) + + // Clean up artifacts folder + try { + FileUtils.deleteDirectory(artifactPath.toFile) + } catch { + case e: IOException => + logWarning(log"Failed to delete directory ${MDC(LogKeys.PATH, artifactPath.toFile)}: " + + log"${MDC(LogKeys.EXCEPTION, e.getMessage)}", e) + } + } } + +private[artifact] case class ArtifactStateForCleanup( + sparkSessionUUID: String, + sparkContext: SparkContext, + jobArtifactState: JobArtifactState, + artifactPath: Path) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala index f09e896fb39a3..a24982aea1585 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/artifact/ArtifactManagerSuite.scala @@ -48,7 +48,7 @@ class ArtifactManagerSuite extends SharedSparkSession { private def sessionUUID: String = spark.sessionUUID override def afterEach(): Unit = { - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() super.afterEach() } @@ -209,7 +209,7 @@ class ArtifactManagerSuite extends SharedSparkSession { assert(expectedPath.toFile.exists()) // Remove resources - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() assert(blockManager.getLocalBytes(blockId).isEmpty) assert(!expectedPath.toFile.exists()) @@ -294,7 +294,7 @@ class ArtifactManagerSuite extends SharedSparkSession { val sessionDirectory = artifactManager.artifactPath.toFile assert(sessionDirectory.exists()) - artifactManager.cleanUpResources() + artifactManager.cleanUpResourcesForTesting() assert(!sessionDirectory.exists()) assert(ArtifactManager.artifactRootDirectory.toFile.exists()) } 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 c41370c96241a..5431066c30a9f 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 @@ -72,7 +72,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd override def afterEach(): Unit = { try { - spark.artifactManager.cleanUpResources() + spark.artifactManager.cleanUpResourcesForTesting() } finally { super.afterEach() } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala index 770e1da94a1c7..7a0599cda2fe7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHiveSingleton.scala @@ -42,7 +42,7 @@ trait TestHiveSingleton extends SparkFunSuite with BeforeAndAfterAll { protected override def afterEach(): Unit = { try { - spark.artifactManager.cleanUpResources() + spark.artifactManager.cleanUpResourcesForTesting() } finally { super.afterEach() } From 47825e502a22ff8c14a27f05a09b953bf403ae65 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Mon, 6 Jan 2025 21:33:13 +0800 Subject: [PATCH 46/60] [SPARK-50736][INFRA] Enable testing module `pyspark-logger` ### What changes were proposed in this pull request? Enable testing module `pyspark-logger` ### Why are the changes needed? this testing module is missing in CI, so following tests were always skipped https://github.com/apache/spark/blob/85d92d7c3a6a38b1b6cfc667caac9176fab5813b/dev/sparktestsupport/modules.py#L1441-L1450 ### Does this PR introduce _any_ user-facing change? no, infra-only ### How was this patch tested? ci ### Was this patch authored or co-authored using generative AI tooling? no Closes #49367 from zhengruifeng/py_test_logging. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .github/workflows/build_and_test.yml | 2 +- dev/sparktestsupport/modules.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/build_and_test.yml b/.github/workflows/build_and_test.yml index a46b90b4913ee..614fdd49d83df 100644 --- a/.github/workflows/build_and_test.yml +++ b/.github/workflows/build_and_test.yml @@ -499,7 +499,7 @@ jobs: - >- pyspark-sql, pyspark-resource, pyspark-testing - >- - pyspark-core, pyspark-errors, pyspark-streaming + pyspark-core, pyspark-errors, pyspark-streaming, pyspark-logger - >- pyspark-mllib, pyspark-ml, pyspark-ml-connect - >- diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 2a19439bd92f9..71817a64b3581 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -1438,7 +1438,7 @@ def __hash__(self): ], ) -pyspark_logging = Module( +pyspark_logger = Module( name="pyspark-logger", dependencies=[], source_file_regexes=["python/pyspark/logger"], From fbb4502070dab7f6480de7e8c5d088453eff3eac Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Mon, 6 Jan 2025 08:47:55 -0800 Subject: [PATCH 47/60] [SPARK-50716][CORE][FOLLOWUP] Fix the scenario in `JavaUtils#deleteRecursivelyUsingJavaIO` where `BasicFileAttributes` cannot be read ### What changes were proposed in this pull request? This PR adds protection against IOException (IOE) scenarios when reading the `BasicFileAttributes` of a file in the `deleteRecursivelyUsingJavaIO` method: it catches the IOE and returns null, and silently handles the scenario where `fileAttributes` is null in the subsequent logic. ### Why are the changes needed? When the inode itself does not exist, it is impossible to read its `BasicFileAttributes`, and an IOException (IOE) will be thrown, which caused the failure of the MacOS daily test: - https://github.com/apache/spark/actions/runs/12622568770/job/35170636435 ``` - JobArtifactSet uses resources from SparkContext *** FAILED *** java.nio.file.NoSuchFileException: /Users/runner/work/spark/spark/core/target/tmp/spark-6a6b2d5d-1371-4801-a6c4-59dc9d69c2f2/userFiles-e450317a-136c-49ff-8099-9e8282c766b5/testFile661537940680128228.zip at java.base/sun.nio.fs.UnixException.translateToIOException(UnixException.java:92) at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:106) at java.base/sun.nio.fs.UnixException.rethrowAsIOException(UnixException.java:111) at java.base/sun.nio.fs.UnixFileAttributeViews$Basic.readAttributes(UnixFileAttributeViews.java:55) at java.base/sun.nio.fs.UnixFileSystemProvider.readAttributes(UnixFileSystemProvider.java:171) at java.base/java.nio.file.Files.readAttributes(Files.java:1853) at org.apache.spark.network.util.JavaUtils.deleteRecursivelyUsingJavaIO(JavaUtils.java:130) at org.apache.spark.network.util.JavaUtils.deleteRecursively(JavaUtils.java:123) at org.apache.spark.network.util.JavaUtils.deleteRecursively(JavaUtils.java:94) at org.apache.spark.util.SparkFileUtils.deleteRecursively(SparkFileUtils.scala:121) ... ``` https://github.com/apache/spark/pull/49347 aimed to fix the cleanup of symbolic links by moving the operation to read `BasicFileAttributes` before `!file.exists` to add a check for broken symbolic links. However, in Spark, there is a logic that first cleans up the potentially existing destination path before overwriting it. The target path being cleaned up may itself be a non-existent inode, such as: https://github.com/apache/spark/blob/91f3fdd25852b43095dd5273358fc394ffd11b66/core/src/main/scala/org/apache/spark/SparkContext.scala#L1879-L1888 Therefore, additional protection is needed for this scenario to maintain compatibility with the old behavior. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? - Pass Github Actions - Pass MacOs 15 & Java 21 Github Actions: https://github.com/LuciferYang/spark/runs/35170478542 - Pass Macos PySpark Github Actions: https://github.com/LuciferYang/spark/runs/35178442154 ### Was this patch authored or co-authored using generative AI tooling? No Closes #49357 from LuciferYang/SPARK-50716-FOLLOWUP. Authored-by: yangjie01 Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/JavaUtils.java | 20 +++++++++++++++---- 1 file changed, 16 insertions(+), 4 deletions(-) diff --git a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java index 027a0d5fb402e..3482c6addfee3 100644 --- a/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/common/utils/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -126,10 +126,10 @@ public static void deleteRecursively(File file, FilenameFilter filter) throws IO private static void deleteRecursivelyUsingJavaIO( File file, FilenameFilter filter) throws IOException { - BasicFileAttributes fileAttributes = - Files.readAttributes(file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); - // SPARK-50716: If the file does not exist and not a broken symbolic link, return directly. - if (!file.exists() && !fileAttributes.isSymbolicLink()) return; + BasicFileAttributes fileAttributes = readFileAttributes(file); + // SPARK-50716: If the file attributes are null, that is, the file attributes cannot be read, + // or if the file does not exist and is not a broken symbolic link, then return directly. + if (fileAttributes == null || (!file.exists() && !fileAttributes.isSymbolicLink())) return; if (fileAttributes.isDirectory()) { IOException savedIOException = null; for (File child : listFilesSafely(file, filter)) { @@ -156,6 +156,18 @@ private static void deleteRecursivelyUsingJavaIO( } } + /** + * Reads basic attributes of a given file, of return null if an I/O error occurs. + */ + private static BasicFileAttributes readFileAttributes(File file) { + try { + return Files.readAttributes( + file.toPath(), BasicFileAttributes.class, LinkOption.NOFOLLOW_LINKS); + } catch (IOException e) { + return null; + } + } + private static void deleteRecursivelyUsingUnixNative(File file) throws IOException { ProcessBuilder builder = new ProcessBuilder("rm", "-rf", file.getAbsolutePath()); Process process = null; From d44fd2b5711095a3ea39b6d6e0fcc0dbc7118727 Mon Sep 17 00:00:00 2001 From: Takuya Ueshin Date: Mon, 6 Jan 2025 12:27:47 -0800 Subject: [PATCH 48/60] [SPARK-50694][SQL] Support renames in subqueries ### What changes were proposed in this pull request? Supports renames in subqueries: - `sub.toDF(...)` - `sub.alias(...)` ### Why are the changes needed? When the query is used as a subquery by adding `col.outer()`, `toDF` or `alias` doesn't work because they need analyzed plans. ### Does this PR introduce _any_ user-facing change? Yes, those APIs are available in subqueries. ### How was this patch tested? Added / modified the related tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49336 from ueshin/issues/SPARK-50694/renames. Authored-by: Takuya Ueshin Signed-off-by: Takuya Ueshin --- .../spark/sql/DataFrameSubquerySuite.scala | 37 +++++++++- .../DataFrameTableValuedFunctionsSuite.scala | 73 ++++++++++--------- python/pyspark/sql/dataframe.py | 3 +- python/pyspark/sql/tests/test_subquery.py | 42 ++++++++++- python/pyspark/sql/tests/test_tvf.py | 58 ++++++++------- .../connect/planner/SparkConnectPlanner.scala | 9 +-- .../scala/org/apache/spark/sql/Dataset.scala | 67 +++++++++++------ .../spark/sql/DataFrameSubquerySuite.scala | 40 +++++++++- .../DataFrameTableValuedFunctionsSuite.scala | 47 ++++++------ 9 files changed, 255 insertions(+), 121 deletions(-) diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index fc37444f77197..4b36d36983a5d 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -354,6 +354,28 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { } } + test("lateral join with star expansion") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select().select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT *)")) + checkAnswer( + t1.lateralJoin(t2.select($"*")).toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2)").toDF("c1", "c2", "c3", "c4")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.*".outer(), $"t2.*")) + .toDF("c1", "c2", "c3", "c4", "c5", "c6"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)") + .toDF("c1", "c2", "c3", "c4", "c5", "c6")) + checkAnswer( + t1.lateralJoin(t2.alias("t1").select($"t1.*")).toDF("c1", "c2", "c3", "c4"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)").toDF("c1", "c2", "c3", "c4")) + } + } + test("lateral join with different join types") { withView("t1") { val t1 = table1() @@ -375,6 +397,17 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { } } + test("lateral join with subquery alias") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer(), $"c2".outer()).toDF("a", "b").as("s")) + .select("a", "b"), + sql("SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)")) + } + } + test("lateral join with correlated equality / non-equality predicates") { withView("t1", "t2") { val t1 = table1() @@ -441,8 +474,8 @@ class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { val t2 = table2() checkAnswer( - t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left") - .join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left") + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2").as("s"), "left") + .join(t1.as("t3"), $"s.c2" === $"t3.c2", "left") .toDF("c1", "c2", "c3", "c4", "c5"), sql(""" |SELECT * FROM t1 diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index aeef2e8f0fcf2..12a49ad21676e 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -61,10 +61,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) checkAnswer( - t3.lateralJoin(spark.tvf.explode($"c2".outer())), + t3.lateralJoin(spark.tvf.explode($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)")) checkAnswer( spark.tvf @@ -113,10 +114,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)")) checkAnswer( - t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) checkAnswer( spark.tvf @@ -161,7 +163,10 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer())), sql("SELECT * FROM array_struct JOIN LATERAL INLINE(arr)")) checkAnswer( - arrayStruct.lateralJoin(spark.tvf.inline($"arr".outer()), $"id" === $"col1", "left"), + arrayStruct.lateralJoin( + spark.tvf.inline($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", + "left"), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k")) } } @@ -202,8 +207,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi sql("SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)")) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline_outer($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline_outer($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left"), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k")) } @@ -238,30 +243,27 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi jsonTable .as("t1") .lateralJoin( - spark.tvf.json_tuple( - $"t1.jstring".outer(), - lit("f1"), - lit("f2"), - lit("f3"), - lit("f4"), - lit("f5"))) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + spark.tvf + .json_tuple( + $"t1.jstring".outer(), + lit("f1"), + lit("f2"), + lit("f3"), + lit("f4"), + lit("f5")) + .as("t2")) + .select($"t1.key", $"t2.*"), sql( "SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2")) checkAnswer( jsonTable .as("t1") - .lateralJoin( - spark.tvf.json_tuple( - $"jstring".outer(), - lit("f1"), - lit("f2"), - lit("f3"), - lit("f4"), - lit("f5"))) - .where($"c0".isNotNull) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + .lateralJoin(spark.tvf + .json_tuple($"jstring".outer(), lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) + .as("t2")) + .where($"t2.c0".isNotNull) + .select($"t1.key", $"t2.*"), sql( "SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + @@ -390,17 +392,18 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer())) - .select($"col0", $"col1"), + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()).as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t")) checkAnswer( - t1.lateralJoin(spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer())) - .select($"col0".as("x"), $"col1".as("y")), + t1.lateralJoin( + spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()).toDF("x", "y").as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)")) checkAnswer( t1.join(t3, $"t1.c1" === $"t3.c1") - .lateralJoin(spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer())) - .select($"col0", $"col1"), + .lateralJoin(spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()).as("t")) + .select($"t.*"), sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t")) } } @@ -463,8 +466,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( variantTable .as("t1") - .lateralJoin(spark.tvf.variant_explode($"v".outer())) - .select($"id", $"pos", $"key", $"value"), + .lateralJoin(spark.tvf.variant_explode($"v".outer()).as("t")) + .select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t")) } } @@ -515,8 +518,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer( variantTable .as("t1") - .lateralJoin(spark.tvf.variant_explode_outer($"v".outer())) - .select($"id", $"pos", $"key", $"value"), + .lateralJoin(spark.tvf.variant_explode_outer($"v".outer()).as("t")) + .select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t")) } } diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index 660f577f56f8b..e321f2c8d755b 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -2713,11 +2713,10 @@ def lateralJoin( >>> customers.alias("c").lateralJoin( ... orders.alias("o") ... .where(sf.col("o.customer_id") == sf.col("c.customer_id").outer()) + ... .select("order_id", "order_date") ... .orderBy(sf.col("order_date").desc()) ... .limit(2), ... how="left" - ... ).select( - ... "c.customer_id", "name", "order_id", "order_date" ... ).orderBy("customer_id", "order_id").show() +-----------+-------+--------+----------+ |customer_id| name|order_id|order_date| diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 0f431589b4616..99a22d7c29664 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -518,6 +518,28 @@ def test_lateral_join_with_single_column_select(self): self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)"""), ) + def test_lateral_join_with_star_expansion(self): + with self.tempView("t1", "t2"): + t1 = self.table1() + t2 = self.table2() + + assertDataFrameEqual( + t1.lateralJoin(self.spark.range(1).select().select(sf.col("*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT *)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT * FROM t2)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.select(sf.col("t1.*").outer(), sf.col("t2.*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)"""), + ) + assertDataFrameEqual( + t1.lateralJoin(t2.alias("t1").select(sf.col("t1.*"))), + self.spark.sql("""SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)"""), + ) + def test_lateral_join_with_different_join_types(self): with self.tempView("t1"): t1 = self.table1() @@ -572,6 +594,20 @@ def test_lateral_join_with_different_join_types(self): }, ) + def test_lateral_join_with_subquery_alias(self): + with self.tempView("t1"): + t1 = self.table1() + + assertDataFrameEqual( + t1.lateralJoin( + self.spark.range(1) + .select(sf.col("c1").outer(), sf.col("c2").outer()) + .toDF("a", "b") + .alias("s") + ).select("a", "b"), + self.spark.sql("""SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)"""), + ) + def test_lateral_join_with_correlated_predicates(self): with self.tempView("t1", "t2"): t1 = self.table1() @@ -661,9 +697,11 @@ def test_lateral_join_in_between_regular_joins(self): assertDataFrameEqual( t1.lateralJoin( - t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")).select(sf.col("c2")), + t2.where(sf.col("t1.c1").outer() == sf.col("t2.c1")) + .select(sf.col("c2")) + .alias("s"), how="left", - ).join(t1.alias("t3"), sf.col("t2.c2") == sf.col("t3.c2"), how="left"), + ).join(t1.alias("t3"), sf.col("s.c2") == sf.col("t3.c2"), how="left"), self.spark.sql( """ SELECT * FROM t1 diff --git a/python/pyspark/sql/tests/test_tvf.py b/python/pyspark/sql/tests/test_tvf.py index ea20cbf9b8f3a..c7274c0810cfb 100644 --- a/python/pyspark/sql/tests/test_tvf.py +++ b/python/pyspark/sql/tests/test_tvf.py @@ -65,11 +65,13 @@ def test_explode_with_lateral_join(self): assertDataFrameEqual( t1.lateralJoin( self.spark.tvf.explode(sf.array(sf.col("c1").outer(), sf.col("c2").outer())) - ).toDF("c1", "c2", "c3"), + .toDF("c3") + .alias("t2") + ), self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)"""), ) assertDataFrameEqual( - t3.lateralJoin(self.spark.tvf.explode(sf.col("c2").outer())).toDF("c1", "c2", "v"), + t3.lateralJoin(self.spark.tvf.explode(sf.col("c2").outer()).toDF("v").alias("t2")), self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)"""), ) assertDataFrameEqual( @@ -127,12 +129,14 @@ def test_explode_outer_with_lateral_join(self): self.spark.tvf.explode_outer( sf.array(sf.col("c1").outer(), sf.col("c2").outer()) ) - ).toDF("c1", "c2", "c3"), + .toDF("c3") + .alias("t2") + ), self.spark.sql("""SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)"""), ) assertDataFrameEqual( - t3.lateralJoin(self.spark.tvf.explode_outer(sf.col("c2").outer())).toDF( - "c1", "c2", "v" + t3.lateralJoin( + self.spark.tvf.explode_outer(sf.col("c2").outer()).toDF("v").alias("t2") ), self.spark.sql("""SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)"""), ) @@ -193,10 +197,10 @@ def test_inline_with_lateral_join(self): ) assertDataFrameEqual( array_struct.lateralJoin( - self.spark.tvf.inline(sf.col("arr").outer()), - sf.col("id") == sf.col("col1"), + self.spark.tvf.inline(sf.col("arr").outer()).toDF("k", "v").alias("t"), + sf.col("id") == sf.col("k"), "left", - ).toDF("id", "arr", "k", "v"), + ), self.spark.sql( """ SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k @@ -252,10 +256,10 @@ def test_inline_outer_with_lateral_join(self): ) assertDataFrameEqual( array_struct.lateralJoin( - self.spark.tvf.inline_outer(sf.col("arr").outer()), - sf.col("id") == sf.col("col1"), + self.spark.tvf.inline_outer(sf.col("arr").outer()).toDF("k", "v").alias("t"), + sf.col("id") == sf.col("k"), "left", - ).toDF("id", "arr", "k", "v"), + ), self.spark.sql( """ SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k @@ -302,9 +306,9 @@ def test_json_tuple_with_lateral_join(self): sf.lit("f3"), sf.lit("f4"), sf.lit("f5"), - ) + ).alias("t2") ) - .select("key", "c0", "c1", "c2", "c3", "c4"), + .select("t1.key", "t2.*"), self.spark.sql( """ SELECT t1.key, t2.* FROM json_table t1, @@ -322,10 +326,10 @@ def test_json_tuple_with_lateral_join(self): sf.lit("f3"), sf.lit("f4"), sf.lit("f5"), - ) + ).alias("t2") ) - .where(sf.col("c0").isNotNull()) - .select("key", "c0", "c1", "c2", "c3", "c4"), + .where(sf.col("t2.c0").isNotNull()) + .select("t1.key", "t2.*"), self.spark.sql( """ SELECT t1.key, t2.* FROM json_table t1, @@ -485,8 +489,8 @@ def test_stack_with_lateral_join(self): sf.col("c1").outer(), sf.lit("Value"), sf.col("c2").outer(), - ) - ).select("col0", "col1"), + ).alias("t") + ).select("t.*"), self.spark.sql( """SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t""" ), @@ -494,17 +498,19 @@ def test_stack_with_lateral_join(self): assertDataFrameEqual( t1.lateralJoin( self.spark.tvf.stack(sf.lit(1), sf.col("c1").outer(), sf.col("c2").outer()) - ).select("col0", "col1"), - self.spark.sql("""SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t"""), + .toDF("x", "y") + .alias("t") + ).select("t.*"), + self.spark.sql("""SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)"""), ) assertDataFrameEqual( t1.join(t3, sf.col("t1.c1") == sf.col("t3.c1")) .lateralJoin( self.spark.tvf.stack( sf.lit(1), sf.col("t1.c2").outer(), sf.col("t3.c2").outer() - ) + ).alias("t") ) - .select("col0", "col1"), + .select("t.*"), self.spark.sql( """ SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 @@ -570,8 +576,8 @@ def test_variant_explode_with_lateral_join(self): assertDataFrameEqual( variant_table.alias("t1") - .lateralJoin(self.spark.tvf.variant_explode(sf.col("v").outer())) - .select("id", "pos", "key", "value"), + .lateralJoin(self.spark.tvf.variant_explode(sf.col("v").outer()).alias("t")) + .select("t1.id", "t.*"), self.spark.sql( """ SELECT t1.id, t.* FROM variant_table AS t1, @@ -629,8 +635,8 @@ def test_variant_explode_outer_with_lateral_join(self): assertDataFrameEqual( variant_table.alias("t1") - .lateralJoin(self.spark.tvf.variant_explode_outer(sf.col("v").outer())) - .select("id", "pos", "key", "value"), + .lateralJoin(self.spark.tvf.variant_explode_outer(sf.col("v").outer()).alias("t")) + .select("t1.id", "t.*"), self.spark.sql( """ SELECT t1.id, t.* FROM variant_table AS t1, diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index 490ae473a6e4c..c0b4384af8b6d 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -45,7 +45,7 @@ import org.apache.spark.internal.LogKeys.{DATAFRAME_ID, SESSION_ID} import org.apache.spark.resource.{ExecutorResourceRequest, ResourceProfile, TaskResourceProfile, TaskResourceRequest} import org.apache.spark.sql.{Column, Dataset, Encoders, ForeachWriter, Observation, RelationalGroupedDataset, Row, SparkSession} import org.apache.spark.sql.catalyst.{expressions, AliasIdentifier, FunctionIdentifier, QueryPlanningTracker} -import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedTableValuedFunction, UnresolvedTranspose} +import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, GlobalTempView, LazyExpression, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, UnresolvedPlanId, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction, UnresolvedTranspose} import org.apache.spark.sql.catalyst.encoders.{encoderFor, AgnosticEncoder, ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.UnboundRowEncoder import org.apache.spark.sql.catalyst.expressions._ @@ -566,10 +566,9 @@ class SparkConnectPlanner( } private def transformToDF(rel: proto.ToDF): LogicalPlan = { - Dataset - .ofRows(session, transformRelation(rel.getInput)) - .toDF(rel.getColumnNamesList.asScala.toSeq: _*) - .logicalPlan + UnresolvedSubqueryColumnAliases( + rel.getColumnNamesList.asScala.toSeq, + transformRelation(rel.getInput)) } private def transformMapPartitions(rel: proto.MapPartitions): LogicalPlan = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index b9ae0e5b91318..287628f2cbef3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -92,6 +92,23 @@ private[sql] object Dataset { dataset } + def apply[T]( + sparkSession: SparkSession, + logicalPlan: LogicalPlan, + encoderGenerator: () => Encoder[T]): Dataset[T] = { + val dataset = new Dataset(sparkSession, logicalPlan, encoderGenerator) + // Eagerly bind the encoder so we verify that the encoder matches the underlying + // schema. The user will get an error if this is not the case. + // optimization: it is guaranteed that [[InternalRow]] can be converted to [[Row]] so + // do not do this check in that case. this check can be expensive since it requires running + // the whole [[Analyzer]] to resolve the deserializer + if (!dataset.queryExecution.isLazyAnalysis + && dataset.encoder.clsTag.runtimeClass != classOf[Row]) { + dataset.resolvedEnc + } + dataset + } + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = sparkSession.withActive { val qe = sparkSession.sessionState.executePlan(logicalPlan) @@ -241,8 +258,13 @@ class Dataset[T] private[sql]( this(queryExecution, () => encoder) } + def this( + sparkSession: SparkSession, logicalPlan: LogicalPlan, encoderGenerator: () => Encoder[T]) = { + this(sparkSession.sessionState.executePlan(logicalPlan), encoderGenerator) + } + def this(sparkSession: SparkSession, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { - this(sparkSession.sessionState.executePlan(logicalPlan), encoder) + this(sparkSession, logicalPlan, () => encoder) } def this(sqlContext: SQLContext, logicalPlan: LogicalPlan, encoder: Encoder[T]) = { @@ -508,16 +530,8 @@ class Dataset[T] private[sql]( /** @inheritdoc */ @scala.annotation.varargs - def toDF(colNames: String*): DataFrame = { - require(schema.size == colNames.size, - "The number of columns doesn't match.\n" + - s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" + - s"New column names (${colNames.size}): " + colNames.mkString(", ")) - - val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) => - Column(oldAttribute).as(newName) - } - select(newCols : _*) + def toDF(colNames: String*): DataFrame = withPlan { + UnresolvedSubqueryColumnAliases(colNames, logicalPlan) } /** @inheritdoc */ @@ -854,7 +868,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def as(alias: String): Dataset[T] = withTypedPlan { + def as(alias: String): Dataset[T] = withSameTypedPlan { SubqueryAlias(alias, logicalPlan) } @@ -909,7 +923,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def filter(condition: Column): Dataset[T] = withTypedPlan { + def filter(condition: Column): Dataset[T] = withSameTypedPlan { Filter(condition.expr, logicalPlan) } @@ -1038,7 +1052,7 @@ class Dataset[T] private[sql]( /** @inheritdoc */ @scala.annotation.varargs - def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withTypedPlan { + def observe(name: String, expr: Column, exprs: Column*): Dataset[T] = withSameTypedPlan { CollectMetrics(name, (expr +: exprs).map(_.named), logicalPlan, id) } @@ -1050,12 +1064,12 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def limit(n: Int): Dataset[T] = withTypedPlan { + def limit(n: Int): Dataset[T] = withSameTypedPlan { Limit(Literal(n), logicalPlan) } /** @inheritdoc */ - def offset(n: Int): Dataset[T] = withTypedPlan { + def offset(n: Int): Dataset[T] = withSameTypedPlan { Offset(Literal(n), logicalPlan) } @@ -1142,7 +1156,7 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def sample(withReplacement: Boolean, fraction: Double, seed: Long): Dataset[T] = { - withTypedPlan { + withSameTypedPlan { Sample(0.0, fraction, withReplacement, seed, logicalPlan) } } @@ -1340,7 +1354,7 @@ class Dataset[T] private[sql]( def dropDuplicates(): Dataset[T] = dropDuplicates(this.columns) /** @inheritdoc */ - def dropDuplicates(colNames: Seq[String]): Dataset[T] = withTypedPlan { + def dropDuplicates(colNames: Seq[String]): Dataset[T] = withSameTypedPlan { val groupCols = groupColsFromDropDuplicates(colNames) Deduplicate(groupCols, logicalPlan) } @@ -1351,7 +1365,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withTypedPlan { + def dropDuplicatesWithinWatermark(colNames: Seq[String]): Dataset[T] = withSameTypedPlan { val groupCols = groupColsFromDropDuplicates(colNames) // UnsupportedOperationChecker will fail the query if this is called with batch Dataset. DeduplicateWithinWatermark(groupCols, logicalPlan) @@ -1511,7 +1525,7 @@ class Dataset[T] private[sql]( } /** @inheritdoc */ - def repartition(numPartitions: Int): Dataset[T] = withTypedPlan { + def repartition(numPartitions: Int): Dataset[T] = withSameTypedPlan { Repartition(numPartitions, shuffle = true, logicalPlan) } @@ -1526,7 +1540,7 @@ class Dataset[T] private[sql]( s"""Invalid partitionExprs specified: $sortOrders |For range partitioning use repartitionByRange(...) instead. """.stripMargin) - withTypedPlan { + withSameTypedPlan { RepartitionByExpression(partitionExprs.map(_.expr), logicalPlan, numPartitions) } } @@ -1539,13 +1553,13 @@ class Dataset[T] private[sql]( case expr: SortOrder => expr case expr: Expression => SortOrder(expr, Ascending) }) - withTypedPlan { + withSameTypedPlan { RepartitionByExpression(sortOrder, logicalPlan, numPartitions) } } /** @inheritdoc */ - def coalesce(numPartitions: Int): Dataset[T] = withTypedPlan { + def coalesce(numPartitions: Int): Dataset[T] = withSameTypedPlan { Repartition(numPartitions, shuffle = false, logicalPlan) } @@ -2240,7 +2254,7 @@ class Dataset[T] private[sql]( SortOrder(expr, Ascending) } } - withTypedPlan { + withSameTypedPlan { Sort(sortOrder, global = global, logicalPlan) } } @@ -2255,6 +2269,11 @@ class Dataset[T] private[sql]( Dataset(sparkSession, logicalPlan) } + /** A convenient function to wrap a logical plan and produce a Dataset. */ + @inline private def withSameTypedPlan(logicalPlan: LogicalPlan): Dataset[T] = { + Dataset(sparkSession, logicalPlan, encoderGenerator) + } + /** A convenient function to wrap a set based logical plan and produce a Dataset. */ @inline private def withSetOperator[U : Encoder](logicalPlan: LogicalPlan): Dataset[U] = { if (isUnTyped) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala index f94cf89276ec4..fdfb909d9ba73 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -418,6 +418,30 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } + test("lateral join with star expansion") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select().select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT *)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"*")), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.select($"t1.*".outer(), $"t2.*")), + sql("SELECT * FROM t1, LATERAL (SELECT t1.*, t2.* FROM t2)") + ) + checkAnswer( + t1.lateralJoin(t2.alias("t1").select($"t1.*")), + sql("SELECT * FROM t1, LATERAL (SELECT t1.* FROM t2 AS t1)") + ) + } + } + test("lateral join with different join types") { withView("t1") { val t1 = table1() @@ -444,6 +468,18 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { } } + test("lateral join with subquery alias") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer(), $"c2".outer()).toDF("a", "b").as("s")) + .select("a", "b"), + sql("SELECT a, b FROM t1, LATERAL (SELECT c1, c2) s(a, b)") + ) + } + } + test("lateral join with correlated equality / non-equality predicates") { withView("t1", "t2") { val t1 = table1() @@ -516,8 +552,8 @@ class DataFrameSubquerySuite extends QueryTest with SharedSparkSession { checkAnswer( t1.lateralJoin( - t2.where($"t1.c1".outer() === $"t2.c1").select($"c2"), "left" - ).join(t1.as("t3"), $"t2.c2" === $"t3.c2", "left"), + t2.where($"t1.c1".outer() === $"t2.c1").select($"c2").as("s"), "left" + ).join(t1.as("t3"), $"s.c2" === $"t3.c2", "left"), sql( """ |SELECT * FROM t1 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala index 4f2cd275ffdfd..637e0cf964fe5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameTableValuedFunctionsSuite.scala @@ -60,11 +60,11 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)") ) checkAnswer( - t3.lateralJoin(spark.tvf.explode($"c2".outer())), + t3.lateralJoin(spark.tvf.explode($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)") ) checkAnswer( @@ -112,11 +112,12 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi val t3 = spark.table("t3") checkAnswer( - t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + t1.lateralJoin( + spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer())).toDF("c3").as("t2")), sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)") ) checkAnswer( - t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer()).toDF("v").as("t2")), sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)") ) checkAnswer( @@ -164,8 +165,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi ) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left" ), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k") @@ -210,8 +211,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi ) checkAnswer( arrayStruct.lateralJoin( - spark.tvf.inline_outer($"arr".outer()), - $"id" === $"col1", + spark.tvf.inline_outer($"arr".outer()).toDF("k", "v").as("t"), + $"id" === $"k", "left" ), sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k") @@ -249,8 +250,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi jsonTable.as("t1").lateralJoin( spark.tvf.json_tuple( $"t1.jstring".outer(), - lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) - ).select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")).as("t2") + ).select($"t1.key", $"t2.*"), sql("SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2") ) @@ -258,9 +259,9 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi jsonTable.as("t1").lateralJoin( spark.tvf.json_tuple( $"jstring".outer(), - lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")) - ).where($"c0".isNotNull) - .select($"key", $"c0", $"c1", $"c2", $"c3", $"c4"), + lit("f1"), lit("f2"), lit("f3"), lit("f4"), lit("f5")).as("t2") + ).where($"t2.c0".isNotNull) + .select($"t1.key", $"t2.*"), sql("SELECT t1.key, t2.* FROM json_table t1, " + "LATERAL json_tuple(t1.jstring, 'f1', 'f2', 'f3', 'f4', 'f5') t2 " + "WHERE t2.c0 IS NOT NULL") @@ -388,21 +389,21 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()) - ).select($"col0", $"col1"), + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer()).as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1, LATERAL stack(2, 'Key', c1, 'Value', c2) t") ) checkAnswer( t1.lateralJoin( - spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()) - ).select($"col0".as("x"), $"col1".as("y")), + spark.tvf.stack(lit(1), $"c1".outer(), $"c2".outer()).toDF("x", "y").as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1 JOIN LATERAL stack(1, c1, c2) t(x, y)") ) checkAnswer( t1.join(t3, $"t1.c1" === $"t3.c1") .lateralJoin( - spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()) - ).select($"col0", $"col1"), + spark.tvf.stack(lit(1), $"t1.c2".outer(), $"t3.c2".outer()).as("t") + ).select($"t.*"), sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t") ) } @@ -466,8 +467,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( variantTable.as("t1").lateralJoin( - spark.tvf.variant_explode($"v".outer()) - ).select($"id", $"pos", $"key", $"value"), + spark.tvf.variant_explode($"v".outer()).as("t") + ).select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t") ) } @@ -519,8 +520,8 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with SharedSparkSessi checkAnswer( variantTable.as("t1").lateralJoin( - spark.tvf.variant_explode_outer($"v".outer()) - ).select($"id", $"pos", $"key", $"value"), + spark.tvf.variant_explode_outer($"v".outer()).as("t") + ).select($"t1.id", $"t.*"), sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t") ) } From 2f0340701f5c8c958324b5a801b69b3acc39a21f Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 15:39:13 -0800 Subject: [PATCH 49/60] [SPARK-50243][SQL][TESTS][FOLLOWUP] Remove `ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER` setting from `AdaptiveQueryExecSuite` ### What changes were proposed in this pull request? This PR is a follow-up to revert one line from the irrelevant test case of `AdaptiveQueryExecSuite`. - #49007 ### Why are the changes needed? `AdaptiveQueryExecSuite` has been a flaky test and was designed to repeat the test logic many times. It's irrelevant to #49007 . ### Does this PR introduce _any_ user-facing change? No, this is a revert to the original code. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49374 from dongjoon-hyun/SPARK-50243. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala index d298b98aaa8da..ad28fd5176d99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -749,7 +749,6 @@ class AdaptiveQueryExecSuite // so retry several times here to avoid unit test failure. eventually(timeout(15.seconds), interval(500.milliseconds)) { withSQLConf( - SQLConf.ARTIFACTS_SESSION_ISOLATION_ALWAYS_APPLY_CLASSLOADER.key -> "true", SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", SQLConf.NON_EMPTY_PARTITION_RATIO_FOR_BROADCAST_JOIN.key -> "0.5") { // `testData` is small enough to be broadcast but has empty partition ratio over the config. From 91bc50930dc9ed57e33479b2e4595621d765d4f1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:08:30 -0800 Subject: [PATCH 50/60] [SPARK-50741][BUILD] Upgrade `zstd-jni` to 1.5.6-9 ### What changes were proposed in this pull request? This PR aims to upgrade `zstd-jni` to 1.5.6-9. ### Why are the changes needed? To bring the latest bug fix: - https://github.com/luben/zstd-jni/issues/336 ### 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 #49375 from dongjoon-hyun/SPARK-50741. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-3-hive-2.3 | 2 +- pom.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-3-hive-2.3 b/dev/deps/spark-deps-hadoop-3-hive-2.3 index d6db12457eeeb..aa171f0cbfffd 100644 --- a/dev/deps/spark-deps-hadoop-3-hive-2.3 +++ b/dev/deps/spark-deps-hadoop-3-hive-2.3 @@ -284,4 +284,4 @@ xz/1.10//xz-1.10.jar zjsonpatch/7.0.1//zjsonpatch-7.0.1.jar zookeeper-jute/3.9.3//zookeeper-jute-3.9.3.jar zookeeper/3.9.3//zookeeper-3.9.3.jar -zstd-jni/1.5.6-8//zstd-jni-1.5.6-8.jar +zstd-jni/1.5.6-9//zstd-jni-1.5.6-9.jar diff --git a/pom.xml b/pom.xml index ba8f42a3b181d..3a8cbd1f033c2 100644 --- a/pom.xml +++ b/pom.xml @@ -838,7 +838,7 @@ com.github.luben zstd-jni - 1.5.6-8 + 1.5.6-9 com.clearspring.analytics From 32e3df7fea181bbf9fad8f92acd4149d505d92dc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:09:19 -0800 Subject: [PATCH 51/60] [SPARK-50742][CORE] Remove `spark.hadoop.fs.s3a.connection.establish.timeout` setting ### What changes were proposed in this pull request? This PR aims to remove `spark.hadoop.fs.s3a.connection.establish.timeout` setting from `SparkContext` because Apache Spark 4.0.0 uses Apache Hadoop 3.4.1 which has the same default value. - #48295 ### Why are the changes needed? This is a logical cleanup by reverting two patches. - #45710 - #46874 ### Does this PR introduce _any_ user-facing change? No. There is no behavior change because we will use the same `fs.s3a.connection.establish.timeout` value. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49376 from dongjoon-hyun/SPARK-50742. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/SparkContext.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5d3a9c2690c41..30d772bd62d77 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -423,9 +423,6 @@ class SparkContext(config: SparkConf) extends Logging { if (!_conf.contains("spark.app.name")) { throw new SparkException("An application name must be set in your configuration") } - // HADOOP-19097 Set fs.s3a.connection.establish.timeout to 30s - // We can remove this after Apache Hadoop 3.4.1 releases - conf.setIfMissing("spark.hadoop.fs.s3a.connection.establish.timeout", "30000") // This should be set as early as possible. SparkContext.fillMissingMagicCommitterConfsIfNeeded(_conf) From 4aa389fdb16a8d5e6811cb0ad0e53e94ce319a68 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:10:07 -0800 Subject: [PATCH 52/60] [SPARK-50745][SQL][TESTS] Use `eventually` in `BufferHolderSparkSubmitSuite` ### What changes were proposed in this pull request? This PR aims to use `eventually` in `BufferHolderSparkSubmitSuite` to stabilize the long-standing flakiness. ### Why are the changes needed? - https://github.com/dongjoon-hyun/spark/actions/runs/12638427790/job/35214696371 ``` [info] BufferHolderSparkSubmitSuite: [info] - SPARK-22222: Buffer holder should be able to allocate memory larger than 1GB *** FAILED *** (3 seconds, 721 milliseconds) ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Pass the CIs. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49380 from dongjoon-hyun/SPARK-50745. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../expressions/codegen/BufferHolderSparkSubmitSuite.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala index 891e2d048b7a8..b0ed1ecabb8d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/BufferHolderSparkSubmitSuite.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.scalatest.{Assertions, BeforeAndAfterEach} +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.scalatest.matchers.must.Matchers +import org.scalatest.time.SpanSugar._ import org.apache.spark.{SparkIllegalArgumentException, TestUtils} import org.apache.spark.deploy.SparkSubmitTestUtils @@ -46,7 +48,10 @@ class BufferHolderSparkSubmitSuite "--conf", "spark.master.rest.enabled=false", "--conf", "spark.driver.extraJavaOptions=-ea", unusedJar.toString) - runSparkSubmit(argsForSparkSubmit) + // Given that the default timeout of runSparkSubmit is 60 seconds, try 3 times in total. + eventually(timeout(210.seconds), interval(70.seconds)) { + runSparkSubmit(argsForSparkSubmit) + } } } From 1419f38bd50e737f0df78f66119c84373610f18b Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:11:20 -0800 Subject: [PATCH 53/60] [SPARK-39903][SQL][TESTS] Reenable TPCDS q72 of `TPCDSQuerySuite` in GitHub Actions ### What changes were proposed in this pull request? This PR aims to re-enable TPCDS q72 test of `TPCDSQuerySuite` in GitHub Actions. ### Why are the changes needed? We have been running q72 query already in other test suites like `TPCDSQueryTestSuite`) in these days. - https://github.com/apache/spark/actions/runs/12624895360/job/35175796403 ``` [info] - q72 (57 seconds, 895 milliseconds) [info] - q72-v2.7 (58 seconds, 223 milliseconds) ``` ### Does this PR introduce _any_ user-facing change? No, this is a test-only change. ### How was this patch tested? Pass the CIs and check the log. - https://github.com/dongjoon-hyun/spark/actions/runs/12639834198/job/35219025146 ``` [info] TPCDSQuerySuite: ... [info] - q72 (40 milliseconds) ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49382 from dongjoon-hyun/SPARK-39903. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala index ffd15eb46a48e..e8b36d8b130cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCDSQuerySuite.scala @@ -33,8 +33,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { // Disable read-side char padding so that the generated code is less than 8000. super.sparkConf.set(SQLConf.READ_SIDE_CHAR_PADDING, false) - // q72 is skipped due to GitHub Actions' memory limit. - tpcdsQueries.filterNot(sys.env.contains("GITHUB_ACTIONS") && _ == "q72").foreach { name => + tpcdsQueries.foreach { name => val queryString = resourceToString(s"tpcds/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) test(name) { @@ -44,8 +43,7 @@ class TPCDSQuerySuite extends BenchmarkQueryTest with TPCDSBase { } } - // q72 is skipped due to GitHub Actions' memory limit. - tpcdsQueriesV2_7_0.filterNot(sys.env.contains("GITHUB_ACTIONS") && _ == "q72").foreach { name => + tpcdsQueriesV2_7_0.foreach { name => val queryString = resourceToString(s"tpcds-v2.7.0/$name.sql", classLoader = Thread.currentThread().getContextClassLoader) test(s"$name-v2.7") { From 9820108dbc1ec6a0e3576d404a0e0623bd3b8355 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 6 Jan 2025 16:14:08 -0800 Subject: [PATCH 54/60] [MINOR][DOCS] Use SBT slash syntax in `ExternalAppendOnlyUnsafeRowArrayBenchmark` doc ### What changes were proposed in this pull request? This PR aims to use `slash` syntax in `ExternalAppendOnlyUnsafeRowArrayBenchmark` doc instead of the deprecated syntax. - https://www.scala-sbt.org/1.x/docs/Migrating-from-sbt-013x.html#slash ### Why are the changes needed? This PR is aligned with our previous efforts. It seems that this is a leftover which we missed the documentation. - #32115 - #37326 - #37754 ### Does this PR introduce _any_ user-facing change? No. This is a doc-only update. ### How was this patch tested? Manual review. ``` $ build/sbt "project sql; set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArrayBenchmark" ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49384 from dongjoon-hyun/minor. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../ExternalAppendOnlyUnsafeRowArrayBenchmark.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index 0078c3f9f65de..31b002a1e245d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -32,10 +32,10 @@ import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter * {{{ * 1. without sbt: * bin/spark-submit --class --jars - * 2. build/sbt build/sbt ";project sql;set javaOptions - * in Test += \"-Dspark.memory.debugFill=false\";Test/runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql;set javaOptions - * in Test += \"-Dspark.memory.debugFill=false\";Test/runMain " + * 2. build/sbt build/sbt ";project sql; + * set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql; + * set Test / javaOptions += \"-Dspark.memory.debugFill=false\";Test/runMain " * Results will be written to * "benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt". * }}} From e24e7b4640b548a11c0e89f805a96ee33359fbe9 Mon Sep 17 00:00:00 2001 From: Xinrong Meng Date: Tue, 7 Jan 2025 09:34:37 +0800 Subject: [PATCH 55/60] [SPARK-50392][PYTHON] DataFrame conversion to table argument in Spark Classic ### What changes were proposed in this pull request? Support DataFrame conversion to table arguments in Spark Classic, and enable UDTFs to accept table arguments in both PySpark and Scala. Spark Connect support will be a follow-up, with the goal of completing it by the end of this month. ### Why are the changes needed? Table-Valued Functions (TVFs) and User-Defined Table Functions (UDTFs) are widely used in Spark workflows. These functions often require a table argument, which Spark internally represents as a Catalyst expression. While Spark SQL supports constructs like TABLE() for this purpose, **there is no direct API in PySpark or Scala to convert a DataFrame into a table argument**. So we propose to support DataFrame conversion to table arguments (in Spark Classic first), and enable UDTFs to accept table arguments in both PySpark and Scala.. ### Does this PR introduce _any_ user-facing change? Yes DataFrame conversion to table argument is supported in Spark Classic, and UDTFs accept table arguments in both PySpark and Scala. ```py >>> from pyspark.sql.functions import udtf >>> from pyspark.sql import Row >>> >>> udtf(returnType="a: int") ... class TestUDTF: ... def eval(self, row: Row): ... if row[0] > 5: ... yield row[0], ... >>> df = spark.range(8) >>> >>> TestUDTF(df.asTable()).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id)).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id).orderBy(df.id)).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().withSinglePartition()).show() +---+ | a| +---+ | 6| | 7| +---+ >>> TestUDTF(df.asTable().partitionBy(df.id).withSinglePartition()).show() Traceback (most recent call last): ... pyspark.errors.exceptions.captured.IllegalArgumentException: Cannot call withSinglePartition() after partitionBy() has been called. ``` ### How was this patch tested? Unit tests. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49055 from xinrong-meng/TableArg. Authored-by: Xinrong Meng Signed-off-by: Xinrong Meng --- .../CheckConnectJvmClientCompatibility.scala | 1 + python/pyspark/sql/_typing.pyi | 2 + python/pyspark/sql/classic/dataframe.py | 4 + python/pyspark/sql/column.py | 3 +- python/pyspark/sql/connect/dataframe.py | 8 ++ python/pyspark/sql/dataframe.py | 24 ++++ python/pyspark/sql/table_arg.py | 52 ++++++++ .../sql/tests/connect/test_parity_udtf.py | 8 ++ python/pyspark/sql/tests/test_udtf.py | 112 ++++++++++++++++++ python/pyspark/sql/tvf_argument.py | 27 +++++ python/pyspark/sql/udtf.py | 33 ++++-- .../scala/org/apache/spark/sql/Column.scala | 2 +- .../sql/TableValuedFunctionArgument.scala | 20 ++++ .../scala/org/apache/spark/sql/Dataset.scala | 23 ++++ .../scala/org/apache/spark/sql/TableArg.scala | 74 ++++++++++++ .../python/UserDefinedPythonFunction.scala | 14 ++- 16 files changed, 395 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/sql/table_arg.py create mode 100644 python/pyspark/sql/tvf_argument.py create mode 100644 sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala 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 be659a0fa95ba..7bac10e79d0b4 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 @@ -183,6 +183,7 @@ object CheckConnectJvmClientCompatibility { "org.apache.spark.sql.ExtendedExplainGenerator"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.UDTFRegistration"), ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.DataSourceRegistration"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.TableArg"), ProblemFilters.exclude[MissingClassProblem]( "org.apache.spark.sql.artifact.ArtifactStateForCleanup"), ProblemFilters.exclude[MissingClassProblem]( diff --git a/python/pyspark/sql/_typing.pyi b/python/pyspark/sql/_typing.pyi index 4969268939adf..27fa0f2a90133 100644 --- a/python/pyspark/sql/_typing.pyi +++ b/python/pyspark/sql/_typing.pyi @@ -36,8 +36,10 @@ from pyspark._typing import PrimitiveType from pyspark.profiler import CodeMapDict import pyspark.sql.types from pyspark.sql.column import Column +from pyspark.sql.tvf_argument import TableValuedFunctionArgument ColumnOrName = Union[Column, str] +TVFArgumentOrName = Union[TableValuedFunctionArgument, str] ColumnOrNameOrOrdinal = Union[Column, str, int] DecimalLiteral = decimal.Decimal DateTimeLiteral = Union[datetime.datetime, datetime.date] diff --git a/python/pyspark/sql/classic/dataframe.py b/python/pyspark/sql/classic/dataframe.py index cc44d2c9942d2..8beabda6c135d 100644 --- a/python/pyspark/sql/classic/dataframe.py +++ b/python/pyspark/sql/classic/dataframe.py @@ -74,6 +74,7 @@ from pyspark.sql.utils import get_active_spark_context, to_java_array, to_scala_map from pyspark.sql.pandas.conversion import PandasConversionMixin from pyspark.sql.pandas.map_ops import PandasMapOpsMixin +from pyspark.sql.table_arg import TableArg if TYPE_CHECKING: @@ -1787,6 +1788,9 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataF else: return DataFrame(self._jdf.transpose(), self.sparkSession) + def asTable(self) -> TableArg: + return TableArg(self._jdf.asTable()) + def scalar(self) -> Column: return Column(self._jdf.scalar()) diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index 285d30fad3bc2..e5640dd81b1fb 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -25,6 +25,7 @@ Union, ) +from pyspark.sql.tvf_argument import TableValuedFunctionArgument from pyspark.sql.utils import dispatch_col_method from pyspark.sql.types import DataType from pyspark.errors import PySparkValueError @@ -37,7 +38,7 @@ __all__ = ["Column"] -class Column: +class Column(TableValuedFunctionArgument): """ A column in a DataFrame. diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 3d8f0eced34b2..789292bdd56f6 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -86,6 +86,7 @@ from pyspark.sql.connect.functions import builtin as F from pyspark.sql.pandas.types import from_arrow_schema, to_arrow_schema from pyspark.sql.pandas.functions import _validate_pandas_udf # type: ignore[attr-defined] +from pyspark.sql.table_arg import TableArg if TYPE_CHECKING: @@ -1801,6 +1802,13 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> ParentDataF self._session, ) + def asTable(self) -> TableArg: + # TODO(SPARK-50393): Support DataFrame conversion to table argument in Spark Connect + raise PySparkNotImplementedError( + errorClass="NOT_IMPLEMENTED", + messageParameters={"feature": "asTable()"}, + ) + def scalar(self) -> Column: from pyspark.sql.connect.column import Column as ConnectColumn diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index e321f2c8d755b..394ac6bdb69c9 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -42,6 +42,7 @@ from pyspark.sql.readwriter import DataFrameWriter, DataFrameWriterV2 from pyspark.sql.merge import MergeIntoWriter from pyspark.sql.streaming import DataStreamWriter +from pyspark.sql.table_arg import TableArg from pyspark.sql.types import StructType, Row from pyspark.sql.utils import dispatch_df_method @@ -6578,6 +6579,29 @@ def transpose(self, indexColumn: Optional["ColumnOrName"] = None) -> "DataFrame" """ ... + def asTable(self) -> TableArg: + """ + Converts the DataFrame into a `TableArg` object, which can be used as a table argument + in a user-defined table function (UDTF). + + After obtaining a TableArg from a DataFrame using this method, you can specify partitioning + and ordering for the table argument by calling methods such as `partitionBy`, `orderBy`, and + `withSinglePartition` on the `TableArg` instance. + - partitionBy(*cols): Partitions the data based on the specified columns. This method cannot + be called after withSinglePartition() has been called. + - orderBy(*cols): Orders the data within partitions based on the specified columns. + - withSinglePartition(): Indicates that the data should be treated as a single partition. + This method cannot be called after partitionBy() has been called. + + .. versionadded:: 4.0.0 + + Returns + ------- + :class:`TableArg` + A `TableArg` object representing a table argument. + """ + ... + def scalar(self) -> Column: """ Return a `Column` object for a SCALAR Subquery containing exactly one row and one column. diff --git a/python/pyspark/sql/table_arg.py b/python/pyspark/sql/table_arg.py new file mode 100644 index 0000000000000..d4b5e1653c7a1 --- /dev/null +++ b/python/pyspark/sql/table_arg.py @@ -0,0 +1,52 @@ +# +# 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. +# + +from typing import TYPE_CHECKING + +from pyspark.sql.classic.column import _to_java_column, _to_seq +from pyspark.sql.tvf_argument import TableValuedFunctionArgument +from pyspark.sql.utils import get_active_spark_context + + +if TYPE_CHECKING: + from py4j.java_gateway import JavaObject + from pyspark.sql._typing import ColumnOrName + + +class TableArg(TableValuedFunctionArgument): + def __init__(self, j_table_arg: "JavaObject"): + self._j_table_arg = j_table_arg + + def partitionBy(self, *cols: "ColumnOrName") -> "TableArg": + sc = get_active_spark_context() + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + j_cols = _to_seq(sc, cols, _to_java_column) + new_j_table_arg = self._j_table_arg.partitionBy(j_cols) + return TableArg(new_j_table_arg) + + def orderBy(self, *cols: "ColumnOrName") -> "TableArg": + sc = get_active_spark_context() + if len(cols) == 1 and isinstance(cols[0], list): + cols = cols[0] + j_cols = _to_seq(sc, cols, _to_java_column) + new_j_table_arg = self._j_table_arg.orderBy(j_cols) + return TableArg(new_j_table_arg) + + def withSinglePartition(self) -> "TableArg": + new_j_table_arg = self._j_table_arg.withSinglePartition() + return TableArg(new_j_table_arg) diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 6955e7377b4c4..6f4e4133335eb 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -76,6 +76,14 @@ def test_udtf_with_analyze_using_file(self): def test_udtf_access_spark_session(self): super().test_udtf_access_spark_session() + @unittest.skip("TODO(SPARK-50393): support df.asTable() in Spark Connect") + def test_df_asTable(self): + super().test_df_asTable() + + @unittest.skip("TODO(SPARK-50393): support df.asTable() in Spark Connect") + def test_df_asTable_chaining_methods(self): + super().test_df_asTable_chaining_methods() + def _add_pyfile(self, path): self.spark.addArtifacts(path, pyfile=True) diff --git a/python/pyspark/sql/tests/test_udtf.py b/python/pyspark/sql/tests/test_udtf.py index 2424f74f93d36..eca3ab0013b92 100644 --- a/python/pyspark/sql/tests/test_udtf.py +++ b/python/pyspark/sql/tests/test_udtf.py @@ -27,6 +27,7 @@ PySparkTypeError, AnalysisException, PySparkPicklingError, + IllegalArgumentException, ) from pyspark.util import PythonEvalType from pyspark.sql.functions import ( @@ -1055,6 +1056,15 @@ def test_udtf_with_table_argument_query(self): [Row(a=6), Row(a=7)], ) + def test_df_asTable(self): + func = self.udtf_for_table_argument() + self.spark.udtf.register("test_udtf", func) + df = self.spark.range(8) + assertDataFrameEqual( + func(df.asTable()), + self.spark.sql("SELECT * FROM test_udtf(TABLE (SELECT id FROM range(0, 8)))"), + ) + def udtf_for_table_argument(self): class TestUDTF: def eval(self, row: Row): @@ -1064,6 +1074,108 @@ def eval(self, row: Row): func = udtf(TestUDTF, returnType="a: int") return func + def test_df_asTable_chaining_methods(self): + class TestUDTF: + def eval(self, row: Row): + yield row["key"], row["value"] + + func = udtf(TestUDTF, returnType="key: int, value: string") + df = self.spark.createDataFrame( + [(1, "a", 3), (1, "b", 3), (2, "c", 4), (2, "d", 4)], ["key", "value", "number"] + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(df.value)), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy(["key", "number"]).orderBy(df.value)), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(df.value.desc())), + [ + Row(key=1, value="b"), + Row(key=1, value="a"), + Row(key=2, value="d"), + Row(key=2, value="c"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().partitionBy("key").orderBy(["number", "value"])), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + checkRowOrder=True, + ) + assertDataFrameEqual( + func(df.asTable().withSinglePartition()), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + ) + + assertDataFrameEqual( + func(df.asTable().withSinglePartition().orderBy("value")), + [ + Row(key=1, value="a"), + Row(key=1, value="b"), + Row(key=2, value="c"), + Row(key=2, value="d"), + ], + ) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call withSinglePartition\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().partitionBy(df.key).withSinglePartition() + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call partitionBy\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().withSinglePartition().partitionBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Please call partitionBy\(\) or withSinglePartition\(\) before orderBy\(\)", + ): + df.asTable().orderBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Please call partitionBy\(\) or withSinglePartition\(\) before orderBy\(\)", + ): + df.asTable().partitionBy().orderBy(df.key) + + with self.assertRaisesRegex( + IllegalArgumentException, + r"Cannot call partitionBy\(\) after partitionBy\(\)" + r" or withSinglePartition\(\) has been called", + ): + df.asTable().partitionBy(df.key).partitionBy() + def test_udtf_with_int_and_table_argument_query(self): class TestUDTF: def eval(self, i: int, row: Row): diff --git a/python/pyspark/sql/tvf_argument.py b/python/pyspark/sql/tvf_argument.py new file mode 100644 index 0000000000000..cb373d35d9ec2 --- /dev/null +++ b/python/pyspark/sql/tvf_argument.py @@ -0,0 +1,27 @@ +# +# 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. +# + + +class TableValuedFunctionArgument: + """ + Base class for arguments passed to Table Valued Functions. + + This class is intentionally left empty and serves as a marker to achieve + parity with the Scala `TableValuedFunctionArgument` trait. + """ + + pass diff --git a/python/pyspark/sql/udtf.py b/python/pyspark/sql/udtf.py index 12c3e985fb89d..3ea32349d81d2 100644 --- a/python/pyspark/sql/udtf.py +++ b/python/pyspark/sql/udtf.py @@ -32,7 +32,7 @@ if TYPE_CHECKING: from py4j.java_gateway import JavaObject - from pyspark.sql._typing import ColumnOrName + from pyspark.sql._typing import TVFArgumentOrName from pyspark.sql.dataframe import DataFrame from pyspark.sql.session import SparkSession @@ -372,22 +372,41 @@ def _create_judtf(self, func: Type) -> "JavaObject": )(self._name, wrapped_func, jdt, self.evalType, self.deterministic) return judtf - def __call__(self, *args: "ColumnOrName", **kwargs: "ColumnOrName") -> "DataFrame": + def __call__(self, *args: "TVFArgumentOrName", **kwargs: "TVFArgumentOrName") -> "DataFrame": from pyspark.sql.classic.column import _to_java_column, _to_seq from pyspark.sql import DataFrame, SparkSession + from pyspark.sql.table_arg import TableArg spark = SparkSession._getActiveSessionOrCreate() sc = spark.sparkContext assert sc._jvm is not None - jcols = [_to_java_column(arg) for arg in args] + [ - sc._jvm.PythonSQLUtils.namedArgumentExpression(key, _to_java_column(value)) - for key, value in kwargs.items() - ] + # Process positional arguments + jargs = [] + for arg in args: + if isinstance(arg, TableArg): + # If the argument is a TableArg, get the Java TableArg object + jargs.append(arg._j_table_arg) + else: + # Otherwise, convert it to a Java column + jargs.append(_to_java_column(arg)) # type: ignore[arg-type] + + # Process keyword arguments + jkwargs = [] + for key, value in kwargs.items(): + if isinstance(value, TableArg): + # If the value is a TableArg, get the Java TableArg object + j_arg = value._j_table_arg + else: + # Otherwise, convert it to a Java column + j_arg = _to_java_column(value) # type: ignore[arg-type] + # Create a named argument expression + j_named_arg = sc._jvm.PythonSQLUtils.namedArgumentExpression(key, j_arg) + jkwargs.append(j_named_arg) judtf = self._judtf - jPythonUDTF = judtf.apply(spark._jsparkSession, _to_seq(sc, jcols)) + jPythonUDTF = judtf.apply(spark._jsparkSession, _to_seq(sc, jargs + jkwargs)) return DataFrame(jPythonUDTF, spark) def asDeterministic(self) -> "UserDefinedTableFunction": diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala index 50ef61d4a7a19..f13b340e5e9c8 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Column.scala @@ -137,7 +137,7 @@ class TypedColumn[-T, U](node: ColumnNode, private[sql] val encoder: Encoder[U]) * @since 1.3.0 */ @Stable -class Column(val node: ColumnNode) extends Logging { +class Column(val node: ColumnNode) extends Logging with TableValuedFunctionArgument { private[sql] def this(name: String, planId: Option[Long]) = this(withOrigin { name match { case "*" => internal.UnresolvedStar(None, planId) diff --git a/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala new file mode 100644 index 0000000000000..f99c4ecd48554 --- /dev/null +++ b/sql/api/src/main/scala/org/apache/spark/sql/TableValuedFunctionArgument.scala @@ -0,0 +1,20 @@ +/* + * 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 + +trait TableValuedFunctionArgument diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 287628f2cbef3..e4e782a50e3d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -1040,6 +1040,29 @@ class Dataset[T] private[sql]( ) } + /** + * Converts the DataFrame into a `TableArg` object, which can be used as a table argument + * in a user-defined table function (UDTF). + * + * After obtaining a `TableArg` from a DataFrame using this method, you can specify + * partitioning and ordering for the table argument by calling methods such as `partitionBy`, + * `orderBy`, and `withSinglePartition` on the `TableArg` instance. + * - partitionBy(*cols): Partitions the data based on the specified columns. + * This method cannot be called after withSinglePartition() has been called. + * - orderBy(*cols): Orders the data within partitions based on the specified columns. + * - withSinglePartition(): Indicates that the data should be treated as a single partition. + * This method cannot be called after partitionBy() has been called. + * + * @group untypedrel + * @since 4.0.0 + */ + def asTable(): TableArg = { + new TableArg( + FunctionTableSubqueryArgumentExpression(plan = logicalPlan), + sparkSession + ) + } + /** @inheritdoc */ def scalar(): Column = { Column(ExpressionColumnNode(ScalarSubqueryExpr(logicalPlan))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala b/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala new file mode 100644 index 0000000000000..133775c0b666c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/TableArg.scala @@ -0,0 +1,74 @@ +/* + * 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.catalyst.expressions.{Ascending, Expression, FunctionTableSubqueryArgumentExpression, SortOrder} + +class TableArg( + private[sql] val expression: FunctionTableSubqueryArgumentExpression, + sparkSession: SparkSession) + extends TableValuedFunctionArgument { + import sparkSession.toRichColumn + + private def isPartitioned: Boolean = + expression.partitionByExpressions.nonEmpty || expression.withSinglePartition + + @scala.annotation.varargs + def partitionBy(cols: Column*): TableArg = { + if (isPartitioned) { + throw new IllegalArgumentException( + "Cannot call partitionBy() after partitionBy() or withSinglePartition() has been called." + ) + } + val partitionByExpressions = cols.map(_.expr) + new TableArg( + expression.copy( + partitionByExpressions = partitionByExpressions), + sparkSession) + } + + @scala.annotation.varargs + def orderBy(cols: Column*): TableArg = { + if (!isPartitioned) { + throw new IllegalArgumentException( + "Please call partitionBy() or withSinglePartition() before orderBy()." + ) + } + val orderByExpressions = cols.map { col => + col.expr match { + case sortOrder: SortOrder => sortOrder + case expr: Expression => SortOrder(expr, Ascending) + } + } + new TableArg( + expression.copy(orderByExpressions = orderByExpressions), + sparkSession) + } + + def withSinglePartition(): TableArg = { + if (isPartitioned) { + throw new IllegalArgumentException( + "Cannot call withSinglePartition() after partitionBy() or " + + "withSinglePartition() has been called." + ) + } + new TableArg( + expression.copy(withSinglePartition = true), + sparkSession) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index 388ede5d062e5..575e3d4072b8c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import net.razorvine.pickle.Pickler import org.apache.spark.api.python.{PythonEvalType, PythonFunction, PythonWorkerUtils, SpecialLengths} -import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession, TableArg, TableValuedFunctionArgument} import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Descending, Expression, FunctionTableSubqueryArgumentExpression, NamedArgumentExpression, NullsFirst, NullsLast, PythonUDAF, PythonUDF, PythonUDTF, PythonUDTFAnalyzeResult, PythonUDTFSelectedExpression, SortOrder, UnresolvedPolymorphicPythonUDTF} import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, NamedParametersSupport, OneRowRelation} @@ -160,8 +160,16 @@ case class UserDefinedPythonTableFunction( } /** Returns a [[DataFrame]] that will evaluate to calling this UDTF with the given input. */ - def apply(session: SparkSession, exprs: Column*): DataFrame = { - val udtf = builder(exprs.map(session.expression), session.sessionState.sqlParser) + def apply(session: SparkSession, exprs: TableValuedFunctionArgument*): DataFrame = { + val parser = session.sessionState.sqlParser + val expressions = exprs.map { + case col: Column => session.expression(col) + case tableArg: TableArg => tableArg.expression + case other => throw new IllegalArgumentException( + s"Unsupported argument type: ${other.getClass.getName}" + ) + } + val udtf = builder(expressions, parser) Dataset.ofRows(session, udtf) } } From 5be9587a3ae587678680359f88f84d8554a70a66 Mon Sep 17 00:00:00 2001 From: Allison Wang Date: Tue, 7 Jan 2025 10:14:43 +0800 Subject: [PATCH 56/60] [SPARK-48730][SQL] Implement CreateSQLFunctionCommand for SQL Scalar and Table Functions ### What changes were proposed in this pull request? This PR implements the `CreateSQLFunctionCommand` to support the creation of SQL scalar and table functions. Note that the logic for storing and resolving SQL UDFs will be implemented in subsequent PRs. And more SQL tests will be added once it can work E2E. ### Why are the changes needed? To support SQL UDFs. ### Does this PR introduce _any_ user-facing change? Yes. After this PR, users can create persistent SQL UDFs. ### How was this patch tested? New UTs. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49126 from allisonwang-db/spark-48730-create-sql-udf. Authored-by: Allison Wang Signed-off-by: Allison Wang --- .../resources/error/error-conditions.json | 66 ++++ .../apache/spark/sql/types/StructField.scala | 12 + .../sql/catalyst/analysis/unresolved.scala | 3 +- .../catalog/UserDefinedFunctionErrors.scala | 68 +++- .../catalyst/plans/logical/v2Commands.scala | 22 +- .../sql/errors/QueryCompilationErrors.scala | 9 + .../sql/catalyst/analysis/AnalysisTest.scala | 15 + .../analysis/ResolveSessionCatalog.scala | 21 ++ .../sql/catalyst/catalog/SQLFunction.scala | 84 +++++ .../catalog/UserDefinedFunction.scala | 2 + .../plans/logical/SQLFunctionNode.scala | 45 +++ .../spark/sql/execution/SparkSqlParser.scala | 15 +- .../command/CreateSQLFunctionCommand.scala | 328 +++++++++++++++++- .../CreateUserDefinedFunctionCommand.scala | 110 ++++++ .../spark/sql/execution/command/views.scala | 15 +- .../CreateSQLFunctionParserSuite.scala | 203 +++++++++++ .../execution/command/DDLParserSuite.scala | 44 --- 17 files changed, 1004 insertions(+), 58 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala diff --git a/common/utils/src/main/resources/error/error-conditions.json b/common/utils/src/main/resources/error/error-conditions.json index 573e7f3a6a384..52c0315bd0732 100644 --- a/common/utils/src/main/resources/error/error-conditions.json +++ b/common/utils/src/main/resources/error/error-conditions.json @@ -1229,6 +1229,18 @@ }, "sqlState" : "4274K" }, + "DUPLICATE_ROUTINE_PARAMETER_NAMES" : { + "message" : [ + "Found duplicate name(s) in the parameter list of the user-defined routine : ." + ], + "sqlState" : "42734" + }, + "DUPLICATE_ROUTINE_RETURNS_COLUMNS" : { + "message" : [ + "Found duplicate column(s) in the RETURNS clause column list of the user-defined routine : ." + ], + "sqlState" : "42711" + }, "EMITTING_ROWS_OLDER_THAN_WATERMARK_NOT_ALLOWED" : { "message" : [ "Previous node emitted a row with eventTime= which is older than current_watermark_value=", @@ -4695,6 +4707,12 @@ ], "sqlState" : "42P01" }, + "TABLE_VALUED_ARGUMENTS_NOT_YET_IMPLEMENTED_FOR_SQL_FUNCTIONS" : { + "message" : [ + "Cannot SQL user-defined function with TABLE arguments because this functionality is not yet implemented." + ], + "sqlState" : "0A000" + }, "TABLE_VALUED_FUNCTION_FAILED_TO_ANALYZE_IN_PYTHON" : { "message" : [ "Failed to analyze the Python user defined table function: " @@ -5827,6 +5845,54 @@ ], "sqlState" : "42K0E" }, + "USER_DEFINED_FUNCTIONS" : { + "message" : [ + "User defined function is invalid:" + ], + "subClass" : { + "CANNOT_CONTAIN_COMPLEX_FUNCTIONS" : { + "message" : [ + "SQL scalar function cannot contain aggregate/window/generate functions: " + ] + }, + "CANNOT_REPLACE_NON_SQL_UDF_WITH_SQL_UDF" : { + "message" : [ + "Cannot replace the non-SQL function with a SQL function." + ] + }, + "NOT_A_VALID_DEFAULT_EXPRESSION" : { + "message" : [ + "The DEFAULT expression of ``.`` is not supported because it contains a subquery." + ] + }, + "NOT_A_VALID_DEFAULT_PARAMETER_POSITION" : { + "message" : [ + "In routine `` parameter `` with DEFAULT must not be followed by parameter `` without DEFAULT." + ] + }, + "NOT_NULL_ON_FUNCTION_PARAMETERS" : { + "message" : [ + "Cannot specify NOT NULL on function parameters: " + ] + }, + "RETURN_COLUMN_COUNT_MISMATCH" : { + "message" : [ + "The number of columns produced by the RETURN clause (num: ``) does not match the number of column names specified by the RETURNS clause (num: ``) of ." + ] + }, + "SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY" : { + "message" : [ + "SQL table function body must be a query." + ] + }, + "SQL_TABLE_UDF_MISSING_COLUMN_NAMES" : { + "message" : [ + "The relation returned by the query in the CREATE FUNCTION statement for with RETURNS TABLE clause lacks explicit names for one or more output columns; please rewrite the function body to provide explicit column names or add column names to the RETURNS TABLE clause, and re-run the command." + ] + } + }, + "sqlState" : "42601" + }, "USER_RAISED_EXCEPTION" : { "message" : [ "" diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala index d4e590629921c..f33a49e686a59 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -147,6 +147,18 @@ case class StructField( if (metadata.contains("comment")) Option(metadata.getString("comment")) else None } + /** + * Return the default value of this StructField. This is used for storing the default value of a + * function parameter. + */ + private[sql] def getDefault(): Option[String] = { + if (metadata.contains("default")) { + Option(metadata.getString("default")) + } else { + None + } + } + /** * Updates the StructField with a new current default value. */ 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 87a5e94d9f630..b47af90c651a6 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 @@ -206,7 +206,8 @@ case class ResolvedInlineTable(rows: Seq[Seq[Expression]], output: Seq[Attribute */ case class UnresolvedTableValuedFunction( name: Seq[String], - functionArgs: Seq[Expression]) + functionArgs: Seq[Expression], + override val isStreaming: Boolean = false) extends UnresolvedLeafNode { final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_TABLE_VALUED_FUNCTION) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala index a5381669caea8..e8cfa8d74e83f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunctionErrors.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.catalyst.catalog import org.apache.spark.SparkException +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.errors.QueryErrorsBase /** - * Errors during registering and executing [[UserDefinedFunction]]s. + * Errors during registering and executing + * [[org.apache.spark.sql.expressions.UserDefinedFunction]]s. */ object UserDefinedFunctionErrors extends QueryErrorsBase { def unsupportedUserDefinedFunction(language: RoutineLanguage): Throwable = { @@ -31,4 +33,68 @@ object UserDefinedFunctionErrors extends QueryErrorsBase { def unsupportedUserDefinedFunction(language: String): Throwable = { SparkException.internalError(s"Unsupported user defined function type: $language") } + + def duplicateParameterNames(routineName: String, names: String): Throwable = { + new AnalysisException( + errorClass = "DUPLICATE_ROUTINE_PARAMETER_NAMES", + messageParameters = Map("routineName" -> routineName, "names" -> names)) + } + + def duplicateReturnsColumns(routineName: String, columns: String): Throwable = { + new AnalysisException( + errorClass = "DUPLICATE_ROUTINE_RETURNS_COLUMNS", + messageParameters = Map("routineName" -> routineName, "columns" -> columns)) + } + + def cannotSpecifyNotNullOnFunctionParameters(input: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_NULL_ON_FUNCTION_PARAMETERS", + messageParameters = Map("input" -> input)) + } + + def bodyIsNotAQueryForSqlTableUdf(functionName: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.SQL_TABLE_UDF_BODY_MUST_BE_A_QUERY", + messageParameters = Map("name" -> functionName)) + } + + def missingColumnNamesForSqlTableUdf(functionName: String): Throwable = { + new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.SQL_TABLE_UDF_MISSING_COLUMN_NAMES", + messageParameters = Map("functionName" -> toSQLId(functionName))) + } + + def invalidTempViewReference(routineName: Seq[String], tempViewName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "VIEW", + "tempObjName" -> toSQLId(tempViewName) + ) + ) + } + + def invalidTempFuncReference(routineName: Seq[String], tempFuncName: String): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "FUNCTION", + "tempObjName" -> toSQLId(tempFuncName) + ) + ) + } + + def invalidTempVarReference(routineName: Seq[String], varName: Seq[String]): Throwable = { + new AnalysisException( + errorClass = "INVALID_TEMP_OBJ_REFERENCE", + messageParameters = Map( + "obj" -> "FUNCTION", + "objName" -> toSQLId(routineName), + "tempObj" -> "VARIABLE", + "tempObjName" -> toSQLId(varName))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala index 85b5e8379d3d5..58c62a90225aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/v2Commands.scala @@ -21,8 +21,8 @@ import org.apache.spark.{SparkIllegalArgumentException, SparkUnsupportedOperatio import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.{AnalysisContext, AssignmentUtils, EliminateSubqueryAliases, FieldName, NamedRelation, PartitionSpec, ResolvedIdentifier, ResolvedProcedure, TypeCheckResult, UnresolvedException, UnresolvedProcedure, ViewSchemaMode} import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.{DataTypeMismatch, TypeCheckSuccess} +import org.apache.spark.sql.catalyst.catalog.{FunctionResource, RoutineLanguage} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.catalog.FunctionResource import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, AttributeSet, Expression, MetadataAttribute, NamedExpression, UnaryExpression, Unevaluable, V2ExpressionUtils} import org.apache.spark.sql.catalyst.plans.DescribeCommandSchema import org.apache.spark.sql.catalyst.trees.BinaryLike @@ -1072,6 +1072,26 @@ case class CreateFunction( copy(child = newChild) } +/** + * The logical plan of the CREATE FUNCTION command for SQL Functions. + */ +case class CreateUserDefinedFunction( + child: LogicalPlan, + inputParamText: Option[String], + returnTypeText: String, + exprText: Option[String], + queryText: Option[String], + comment: Option[String], + isDeterministic: Option[Boolean], + containsSQL: Option[Boolean], + language: RoutineLanguage, + isTableFunc: Boolean, + ignoreIfExists: Boolean, + replace: Boolean) extends UnaryCommand { + override protected def withNewChildInternal(newChild: LogicalPlan): CreateUserDefinedFunction = + copy(child = newChild) +} + /** * The logical plan of the DROP FUNCTION command. */ 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 d38c7a01e1c44..65ae8da3c4da1 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 @@ -2172,6 +2172,15 @@ private[sql] object QueryCompilationErrors extends QueryErrorsBase with Compilat "ability" -> ability)) } + def tableValuedArgumentsNotYetImplementedForSqlFunctions( + action: String, functionName: String): Throwable = { + new AnalysisException( + errorClass = "TABLE_VALUED_ARGUMENTS_NOT_YET_IMPLEMENTED_FOR_SQL_FUNCTIONS", + messageParameters = Map( + "action" -> action, + "functionName" -> functionName)) + } + def tableValuedFunctionTooManyTableArgumentsError(num: Int): Throwable = { new AnalysisException( errorClass = "TABLE_VALUED_FUNCTION_TOO_MANY_TABLE_ARGUMENTS", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala index 71744f4d15105..58e6cd7fe1695 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala @@ -198,6 +198,21 @@ trait AnalysisTest extends PlanTest { } } + protected def assertParseErrorClass( + parser: String => Any, + sqlCommand: String, + errorClass: String, + parameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty): Unit = { + val e = parseException(parser)(sqlCommand) + checkError( + exception = e, + condition = errorClass, + parameters = parameters, + queryContext = queryContext + ) + } + protected def interceptParseException(parser: String => Any)( sqlCommand: String, messages: String*)(condition: Option[String] = None): Unit = { val e = parseException(parser)(sqlCommand) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala index 87ea3071f490b..6a388a7849f75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveSessionCatalog.scala @@ -497,6 +497,27 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager) case CreateFunction(ResolvedIdentifier(catalog, _), _, _, _, _) => throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") + + case c @ CreateUserDefinedFunction( + ResolvedIdentifierInSessionCatalog(ident), _, _, _, _, _, _, _, _, _, _, _) => + CreateUserDefinedFunctionCommand( + FunctionIdentifier(ident.table, ident.database, ident.catalog), + c.inputParamText, + c.returnTypeText, + c.exprText, + c.queryText, + c.comment, + c.isDeterministic, + c.containsSQL, + c.language, + c.isTableFunc, + isTemp = false, + c.ignoreIfExists, + c.replace) + + case CreateUserDefinedFunction( + ResolvedIdentifier(catalog, _), _, _, _, _, _, _, _, _, _, _, _) => + throw QueryCompilationErrors.missingCatalogAbilityError(catalog, "CREATE FUNCTION") } private def constructV1TableCmd( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala index 8ae0341e5646c..c0bd4ac80f5ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/SQLFunction.scala @@ -17,9 +17,16 @@ package org.apache.spark.sql.catalyst.catalog +import scala.collection.mutable + +import org.json4s.JsonAST.{JArray, JString} +import org.json4s.jackson.JsonMethods.{compact, render} + import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._ +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalarSubquery} import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project} import org.apache.spark.sql.types.{DataType, StructType} /** @@ -56,10 +63,48 @@ case class SQLFunction( assert((isTableFunc && returnType.isRight) || (!isTableFunc && returnType.isLeft)) override val language: RoutineLanguage = LanguageSQL + + /** + * Optionally get the function body as an expression or query using the given parser. + */ + def getExpressionAndQuery( + parser: ParserInterface, + isTableFunc: Boolean): (Option[Expression], Option[LogicalPlan]) = { + // The RETURN clause of the CREATE FUNCTION statement looks like this in the parser: + // RETURN (query | expression) + // If the 'query' matches and parses as a SELECT clause of one item with no FROM clause, and + // this is a scalar function, we skip a level of subquery expression wrapping by using the + // referenced expression directly. + val parsedExpression = exprText.map(parser.parseExpression) + val parsedQuery = queryText.map(parser.parsePlan) + (parsedExpression, parsedQuery) match { + case (None, Some(Project(expr :: Nil, _: OneRowRelation))) + if !isTableFunc => + (Some(expr), None) + case (Some(ScalarSubquery(Project(expr :: Nil, _: OneRowRelation), _, _, _, _, _, _)), None) + if !isTableFunc => + (Some(expr), None) + case (_, _) => + (parsedExpression, parsedQuery) + } + } } object SQLFunction { + private val SQL_FUNCTION_PREFIX = "sqlFunction." + + private val FUNCTION_CATALOG_AND_NAMESPACE = "catalogAndNamespace.numParts" + private val FUNCTION_CATALOG_AND_NAMESPACE_PART_PREFIX = "catalogAndNamespace.part." + + private val FUNCTION_REFERRED_TEMP_VIEW_NAMES = "referredTempViewNames" + private val FUNCTION_REFERRED_TEMP_FUNCTION_NAMES = "referredTempFunctionsNames" + private val FUNCTION_REFERRED_TEMP_VARIABLE_NAMES = "referredTempVariableNames" + + def parseDefault(text: String, parser: ParserInterface): Expression = { + parser.parseExpression(text) + } + /** * This method returns an optional DataType indicating, when present, either the return type for * scalar user-defined functions, or a StructType indicating the names and types of the columns in @@ -92,4 +137,43 @@ object SQLFunction { } } } + + def isSQLFunction(className: String): Boolean = className == SQL_FUNCTION_PREFIX + + /** + * Convert the current catalog and namespace to properties. + */ + def catalogAndNamespaceToProps( + currentCatalog: String, + currentNamespace: Seq[String]): Map[String, String] = { + val props = new mutable.HashMap[String, String] + val parts = currentCatalog +: currentNamespace + if (parts.nonEmpty) { + props.put(FUNCTION_CATALOG_AND_NAMESPACE, parts.length.toString) + parts.zipWithIndex.foreach { case (name, index) => + props.put(s"$FUNCTION_CATALOG_AND_NAMESPACE_PART_PREFIX$index", name) + } + } + props.toMap + } + + /** + * Convert the temporary object names to properties. + */ + def referredTempNamesToProps( + viewNames: Seq[Seq[String]], + functionsNames: Seq[String], + variableNames: Seq[Seq[String]]): Map[String, String] = { + val viewNamesJson = + JArray(viewNames.map(nameParts => JArray(nameParts.map(JString).toList)).toList) + val functionsNamesJson = JArray(functionsNames.map(JString).toList) + val variableNamesJson = + JArray(variableNames.map(nameParts => JArray(nameParts.map(JString).toList)).toList) + + val props = new mutable.HashMap[String, String] + props.put(FUNCTION_REFERRED_TEMP_VIEW_NAMES, compact(render(viewNamesJson))) + props.put(FUNCTION_REFERRED_TEMP_FUNCTION_NAMES, compact(render(functionsNamesJson))) + props.put(FUNCTION_REFERRED_TEMP_VARIABLE_NAMES, compact(render(variableNamesJson))) + props.toMap + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala index 1473f19cb71bd..6567062841de5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/catalog/UserDefinedFunction.scala @@ -56,6 +56,8 @@ trait UserDefinedFunction { } object UserDefinedFunction { + val SQL_CONFIG_PREFIX = "sqlConfig." + def parseTableSchema(text: String, parser: ParserInterface): StructType = { val parsed = parser.parseTableSchema(text) CharVarcharUtils.failIfHasCharVarchar(parsed).asInstanceOf[StructType] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala new file mode 100644 index 0000000000000..0a3274af33b5b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/SQLFunctionNode.scala @@ -0,0 +1,45 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.trees.TreePattern.FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION +import org.apache.spark.sql.errors.DataTypeErrors.toSQLId +import org.apache.spark.sql.errors.QueryCompilationErrors + +/** + * A container for holding a SQL function query plan and its function identifier. + * + * @param function: the SQL function that this node represents. + * @param child: the SQL function body. + */ +case class SQLFunctionNode( + function: SQLFunction, + child: LogicalPlan) extends UnaryNode { + override def output: Seq[Attribute] = child.output + override def stringArgs: Iterator[Any] = Iterator(function.name, child) + override protected def withNewChildInternal(newChild: LogicalPlan): SQLFunctionNode = + copy(child = newChild) + + // Throw a reasonable error message when trying to call a SQL UDF with TABLE argument(s). + if (child.containsPattern(FUNCTION_TABLE_RELATION_ARGUMENT_EXPRESSION)) { + throw QueryCompilationErrors + .tableValuedArgumentsNotYetImplementedForSqlFunctions("call", toSQLId(function.name.funcName)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index 8d5ddb2d85c4e..744ab03d5d037 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -723,8 +723,19 @@ class SparkSqlAstBuilder extends AstBuilder { withIdentClause(ctx.identifierReference(), functionIdentifier => { if (ctx.TEMPORARY == null) { - // TODO: support creating persistent UDFs. - operationNotAllowed(s"creating persistent SQL functions is not supported", ctx) + CreateUserDefinedFunction( + UnresolvedIdentifier(functionIdentifier), + inputParamText, + returnTypeText, + exprText, + queryText, + comment, + deterministic, + containsSQL, + language, + isTableFunc, + ctx.EXISTS != null, + ctx.REPLACE != null) } else { // Disallow to define a temporary function with `IF NOT EXISTS` if (ctx.EXISTS != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala index d2aaa93fcca06..25598a12af222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionCommand.scala @@ -17,9 +17,19 @@ package org.apache.spark.sql.execution.command -import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.catalog.SQLFunction +import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation} +import org.apache.spark.sql.catalyst.catalog.{SessionCatalog, SQLFunction, UserDefinedFunctionErrors} +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Generator, LateralSubquery, Literal, ScalarSubquery, SubqueryExpression, WindowExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical.{LateralJoin, LogicalPlan, OneRowRelation, Project, SQLFunctionNode, UnresolvedWith} +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_ATTRIBUTE +import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.sql.execution.command.CreateUserDefinedFunctionCommand._ +import org.apache.spark.sql.types.{DataType, StructField, StructType} /** * The DDL command that creates a SQL function. @@ -52,10 +62,13 @@ case class CreateSQLFunctionCommand( replace: Boolean) extends CreateUserDefinedFunctionCommand { - override def run(sparkSession: SparkSession): Seq[Row] = { - import SQLFunction._ + import SQLFunction._ + override def run(sparkSession: SparkSession): Seq[Row] = { val parser = sparkSession.sessionState.sqlParser + val analyzer = sparkSession.sessionState.analyzer + val catalog = sparkSession.sessionState.catalog + val conf = sparkSession.sessionState.conf val inputParam = inputParamText.map(parser.parseTableSchema) val returnType = parseReturnTypeText(returnTypeText, isTableFunc, parser) @@ -72,8 +85,313 @@ case class CreateSQLFunctionCommand( isTableFunc, Map.empty) - // TODO: Implement the rest of the method. + val newFunction = { + val (expression, query) = function.getExpressionAndQuery(parser, isTableFunc) + assert(query.nonEmpty || expression.nonEmpty) + + // Check if the function can be replaced. + if (replace && catalog.functionExists(name)) { + checkFunctionSignatures(catalog, name) + } + + // Build function input. + val inputPlan = if (inputParam.isDefined) { + val param = inputParam.get + checkParameterNotNull(param, inputParamText.get) + checkParameterNameDuplication(param, conf, name) + checkDefaultsTrailing(param, name) + + // Qualify the input parameters with the function name so that attributes referencing + // the function input parameters can be resolved correctly. + val qualifier = Seq(name.funcName) + val input = param.map(p => Alias( + { + val defaultExpr = p.getDefault() + if (defaultExpr.isEmpty) { + Literal.create(null, p.dataType) + } else { + val defaultPlan = parseDefault(defaultExpr.get, parser) + if (SubqueryExpression.hasSubquery(defaultPlan)) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_EXPRESSION", + messageParameters = + Map("functionName" -> name.funcName, "parameterName" -> p.name)) + } else if (defaultPlan.containsPattern(UNRESOLVED_ATTRIBUTE)) { + // TODO(SPARK-50698): use parsed expression instead of expression string. + defaultPlan.collect { + case a: UnresolvedAttribute => + throw QueryCompilationErrors.unresolvedAttributeError( + "UNRESOLVED_COLUMN", a.sql, Seq.empty, a.origin) + } + } + Cast(defaultPlan, p.dataType) + } + }, p.name)(qualifier = qualifier)) + Project(input, OneRowRelation()) + } else { + OneRowRelation() + } + + // Build the function body and check if the function body can be analyzed successfully. + val (unresolvedPlan, analyzedPlan, inferredReturnType) = if (!isTableFunc) { + // Build SQL scalar function plan. + val outputExpr = if (query.isDefined) ScalarSubquery(query.get) else expression.get + val plan: LogicalPlan = returnType.map { t => + val retType: DataType = t match { + case Left(t) => t + case _ => throw SparkException.internalError( + "Unexpected return type for a scalar SQL UDF.") + } + val outputCast = Seq(Alias(Cast(outputExpr, retType), name.funcName)()) + Project(outputCast, inputPlan) + }.getOrElse { + // If no explicit RETURNS clause is present, infer the result type from the function body. + val outputAlias = Seq(Alias(outputExpr, name.funcName)()) + Project(outputAlias, inputPlan) + } + + // Check the function body can be analyzed correctly. + val analyzed = analyzer.execute(plan) + val (resolved, resolvedReturnType) = analyzed match { + case p @ Project(expr :: Nil, _) if expr.resolved => + (p, Left(expr.dataType)) + case other => + (other, function.returnType) + } + + // Check if the SQL function body contains aggregate/window functions. + // This check needs to be performed before checkAnalysis to provide better error messages. + checkAggOrWindowOrGeneratorExpr(resolved) + + // Check if the SQL function body can be analyzed. + checkFunctionBodyAnalysis(analyzer, function, resolved) + + (plan, resolved, resolvedReturnType) + } else { + // Build SQL table function plan. + if (query.isEmpty) { + throw UserDefinedFunctionErrors.bodyIsNotAQueryForSqlTableUdf(name.funcName) + } + + // Construct a lateral join to analyze the function body. + val plan = LateralJoin(inputPlan, LateralSubquery(query.get), Inner, None) + val analyzed = analyzer.execute(plan) + val newPlan = analyzed match { + case Project(_, j: LateralJoin) => j + case j: LateralJoin => j + case _ => throw SparkException.internalError("Unexpected plan returned when " + + s"creating a SQL TVF: ${analyzed.getClass.getSimpleName}.") + } + val maybeResolved = newPlan.asInstanceOf[LateralJoin].right.plan + + // Check if the function body can be analyzed. + checkFunctionBodyAnalysis(analyzer, function, maybeResolved) + + // Get the function's return schema. + val returnParam: StructType = returnType.map { + case Right(t) => t + case Left(_) => throw SparkException.internalError( + "Unexpected return schema for a SQL table function.") + }.getOrElse { + // If no explicit RETURNS clause is present, infer the result type from the function body. + // To detect this, we search for instances of the UnresolvedAlias expression. Examples: + // CREATE TABLE t USING PARQUET AS VALUES (0, 1), (1, 2) AS tab(c1, c2); + // SELECT c1 FROM t --> UnresolvedAttribute: 'c1 + // SELECT c1 + 1 FROM t --> UnresolvedAlias: unresolvedalias(('c1 + 1), None) + // SELECT c1 + 1 AS a FROM t --> Alias: ('c1 + 1) AS a#2 + query.get match { + case Project(projectList, _) if projectList.exists(_.isInstanceOf[UnresolvedAlias]) => + throw UserDefinedFunctionErrors.missingColumnNamesForSqlTableUdf(name.funcName) + case _ => + StructType(analyzed.asInstanceOf[LateralJoin].right.plan.output.map { col => + StructField(col.name, col.dataType) + }) + } + } + + // Check the return columns cannot have NOT NULL specified. + checkParameterNotNull(returnParam, returnTypeText) + + // Check duplicated return column names. + checkReturnsColumnDuplication(returnParam, conf, name) + + // Check if the actual output size equals to the number of return parameters. + val outputSize = maybeResolved.output.size + if (outputSize != returnParam.size) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.RETURN_COLUMN_COUNT_MISMATCH", + messageParameters = Map( + "outputSize" -> s"$outputSize", + "returnParamSize" -> s"${returnParam.size}", + "name" -> s"$name" + ) + ) + } + + (plan, analyzed, Right(returnParam)) + } + + // A permanent function is not allowed to reference temporary objects. + // This should be called after `qe.assertAnalyzed()` (i.e., `plan` can be resolved) + verifyTemporaryObjectsNotExists(catalog, isTemp, name, unresolvedPlan, analyzedPlan) + + // Generate function properties. + val properties = generateFunctionProperties(sparkSession, unresolvedPlan, analyzedPlan) + + // Derive determinism of the SQL function. + val deterministic = analyzedPlan.deterministic + + function.copy( + // Assign the return type, inferring from the function body if needed. + returnType = inferredReturnType, + deterministic = Some(function.deterministic.getOrElse(deterministic)), + properties = properties + ) + } + + // TODO: create/register sql functions in catalog Seq.empty } + + /** + * Check if the function body can be analyzed. + */ + private def checkFunctionBodyAnalysis( + analyzer: Analyzer, + function: SQLFunction, + body: LogicalPlan): Unit = { + analyzer.checkAnalysis(SQLFunctionNode(function, body)) + } + + /** Check whether the new function is replacing an existing SQL function. */ + private def checkFunctionSignatures(catalog: SessionCatalog, name: FunctionIdentifier): Unit = { + val info = catalog.lookupFunctionInfo(name) + if (!isSQLFunction(info.getClassName)) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.CANNOT_REPLACE_NON_SQL_UDF_WITH_SQL_UDF", + messageParameters = Map("name" -> s"$name") + ) + } + } + + /** + * Collect all temporary views and functions and return the identifiers separately + * This func traverses the unresolved plan `child`. Below are the reasons: + * 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding + * logical plan. After replacement, it is impossible to detect whether the SubqueryAlias is + * added/generated from a temporary view. + * 2) The temp functions are represented by multiple classes. Most are inaccessible from this + * package (e.g., HiveGenericUDF). + * 3) Temporary SQL functions, once resolved, cannot be identified as temp functions. + */ + private def collectTemporaryObjectsInUnresolvedPlan( + catalog: SessionCatalog, + child: LogicalPlan): (Seq[Seq[String]], Seq[String]) = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + def collectTempViews(child: LogicalPlan): Seq[Seq[String]] = { + child.flatMap { + case UnresolvedRelation(nameParts, _, _) if catalog.isTempView(nameParts) => + Seq(nameParts) + case w: UnresolvedWith if !w.resolved => w.innerChildren.flatMap(collectTempViews) + case plan if !plan.resolved => plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempViews(e.plan) + case _ => Seq.empty + }) + case _ => Seq.empty + }.distinct + } + + def collectTempFunctions(child: LogicalPlan): Seq[String] = { + child.flatMap { + case w: UnresolvedWith if !w.resolved => w.innerChildren.flatMap(collectTempFunctions) + case plan if !plan.resolved => + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => collectTempFunctions(e.plan) + case e: UnresolvedFunction + if catalog.isTemporaryFunction(e.nameParts.asFunctionIdentifier) => + Seq(e.nameParts.asFunctionIdentifier.funcName) + case _ => Seq.empty + }) + case _ => Seq.empty + }.distinct + } + (collectTempViews(child), collectTempFunctions(child)) + } + + /** + * Permanent functions are not allowed to reference temp objects, including temp functions + * and temp views. + */ + private def verifyTemporaryObjectsNotExists( + catalog: SessionCatalog, + isTemporary: Boolean, + name: FunctionIdentifier, + child: LogicalPlan, + analyzed: LogicalPlan): Unit = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + if (!isTemporary) { + val (tempViews, tempFunctions) = collectTemporaryObjectsInUnresolvedPlan(catalog, child) + tempViews.foreach { nameParts => + throw UserDefinedFunctionErrors.invalidTempViewReference( + routineName = name.asMultipart, tempViewName = nameParts) + } + tempFunctions.foreach { funcName => + throw UserDefinedFunctionErrors.invalidTempFuncReference( + routineName = name.asMultipart, tempFuncName = funcName) + } + val tempVars = ViewHelper.collectTemporaryVariables(analyzed) + tempVars.foreach { varName => + throw UserDefinedFunctionErrors.invalidTempVarReference( + routineName = name.asMultipart, varName = varName) + } + } + } + + /** + * Check if the SQL function body contains aggregate/window/generate functions. + * Note subqueries inside the SQL function body can contain aggregate/window/generate functions. + */ + private def checkAggOrWindowOrGeneratorExpr(plan: LogicalPlan): Unit = { + if (plan.resolved) { + plan.transformAllExpressions { + case e if e.isInstanceOf[WindowExpression] || e.isInstanceOf[Generator] || + e.isInstanceOf[AggregateExpression] => + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.CANNOT_CONTAIN_COMPLEX_FUNCTIONS", + messageParameters = Map("queryText" -> s"${exprText.orElse(queryText).get}") + ) + } + } + } + + /** + * Generate the function properties, including: + * 1. the SQL configs when creating the function. + * 2. the catalog and database name when creating the function. This will be used to provide + * context during nested function resolution. + * 3. referred temporary object names if the function is a temp function. + */ + private def generateFunctionProperties( + session: SparkSession, + plan: LogicalPlan, + analyzed: LogicalPlan): Map[String, String] = { + val catalog = session.sessionState.catalog + val conf = session.sessionState.conf + val manager = session.sessionState.catalogManager + + // Only collect temporary object names when the function is a temp function. + val (tempViews, tempFunctions) = if (isTemp) { + collectTemporaryObjectsInUnresolvedPlan(catalog, plan) + } else { + (Nil, Nil) + } + val tempVars = ViewHelper.collectTemporaryVariables(analyzed) + + sqlConfigsToProps(conf) ++ + catalogAndNamespaceToProps( + manager.currentCatalog.name, + manager.currentNamespace.toIndexedSeq) ++ + referredTempNamesToProps(tempViews, tempFunctions, tempVars) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala index bebb0f5cf6c38..1ee3c8a4c388f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/CreateUserDefinedFunctionCommand.scala @@ -17,9 +17,15 @@ package org.apache.spark.sql.execution.command +import java.util.Locale + +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.catalog.{LanguageSQL, RoutineLanguage, UserDefinedFunctionErrors} +import org.apache.spark.sql.catalyst.catalog.UserDefinedFunction._ import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.StructType /** * The base class for CreateUserDefinedFunctionCommand @@ -74,4 +80,108 @@ object CreateUserDefinedFunctionCommand { throw UserDefinedFunctionErrors.unsupportedUserDefinedFunction(other) } } + + /** + * Convert SQL configs to properties by prefixing all configs with a key. + * When converting a function to [[org.apache.spark.sql.catalyst.catalog.CatalogFunction]] or + * [[org.apache.spark.sql.catalyst.expressions.ExpressionInfo]], all SQL configs and other + * function properties (such as the function parameters and the function return type) + * are saved together in a property map. + */ + def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { + val modifiedConfs = ViewHelper.getModifiedConf(conf) + modifiedConfs.map { case (key, value) => s"$SQL_CONFIG_PREFIX$key" -> value } + } + + /** + * Check whether the function parameters contain duplicated column names. + * It takes the function input parameter struct as input and verifies that there is no duplicates + * in the parameter column names. + * If any duplicates are found, it throws an exception with helpful information for users to + * fix the wrong function parameters. + * + * Perform this check while registering the function to fail early. + * This check does not need to run the function itself. + */ + def checkParameterNameDuplication( + param: StructType, + conf: SQLConf, + name: FunctionIdentifier): Unit = { + val names = if (conf.caseSensitiveAnalysis) { + param.fields.map(_.name) + } else { + param.fields.map(_.name.toLowerCase(Locale.ROOT)) + } + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"`$x`" + } + throw UserDefinedFunctionErrors.duplicateParameterNames( + routineName = name.funcName, + names = duplicateColumns.toSeq.sorted.mkString(", ")) + } + } + + /** + * Check whether the function has duplicate column names in the RETURNS clause. + */ + def checkReturnsColumnDuplication( + columns: StructType, + conf: SQLConf, + name: FunctionIdentifier): Unit = { + val names = if (conf.caseSensitiveAnalysis) { + columns.fields.map(_.name) + } else { + columns.fields.map(_.name.toLowerCase(Locale.ROOT)) + } + if (names.distinct.length != names.length) { + val duplicateColumns = names.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => s"`$x`" + } + throw UserDefinedFunctionErrors.duplicateReturnsColumns( + routineName = name.funcName, + columns = duplicateColumns.toSeq.sorted.mkString(", ")) + } + } + + /** + * Check whether the function parameters contain non trailing defaults. + * For languages that support default values for input parameters, + * this check ensures once a default value is given to a parameter, + * all subsequent parameters must also have a default value. It throws error if otherwise. + * + * Perform this check on function input parameters while registering the function to fail early. + * This check does not need to run the function itself. + */ + def checkDefaultsTrailing(param: StructType, name: FunctionIdentifier): Unit = { + var defaultFound = false + var previousParamName = ""; + param.fields.foreach { field => + if (field.getDefault().isEmpty && defaultFound) { + throw new AnalysisException( + errorClass = "USER_DEFINED_FUNCTIONS.NOT_A_VALID_DEFAULT_PARAMETER_POSITION", + messageParameters = Map( + "functionName" -> name.funcName, + "parameterName" -> previousParamName, + "nextParameterName" -> field.name)) + } + defaultFound |= field.getDefault().isDefined + previousParamName = field.name + } + } + + /** + * Check whether the function input or return columns (for TABLE Return type) have NOT NULL + * specified. Throw exception if NOT NULL is found. + * + * Perform this check on function input and return parameters while registering the function + * to fail early. This check does not need to run the function itself. + */ + def checkParameterNotNull(param: StructType, input: String): Unit = { + param.fields.foreach { field => + if (!field.nullable) { + throw UserDefinedFunctionErrors.cannotSpecifyNotNullOnFunctionParameters(input) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala index d5a72fd6c441a..f654c846c8a57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/views.scala @@ -464,12 +464,19 @@ object ViewHelper extends SQLConfHelper with Logging { } /** - * Convert the view SQL configs to `properties`. + * Get all configurations that are modifiable and should be captured. */ - private def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { - val modifiedConfs = conf.getAllConfs.filter { case (k, _) => + def getModifiedConf(conf: SQLConf): Map[String, String] = { + conf.getAllConfs.filter { case (k, _) => conf.isModifiable(k) && shouldCaptureConfig(k) } + } + + /** + * Convert the view SQL configs to `properties`. + */ + private def sqlConfigsToProps(conf: SQLConf): Map[String, String] = { + val modifiedConfs = getModifiedConf(conf) // Some configs have dynamic default values, such as SESSION_LOCAL_TIMEZONE whose // default value relies on the JVM system timezone. We need to always capture them to // to make sure we apply the same configs when reading the view. @@ -690,7 +697,7 @@ object ViewHelper extends SQLConfHelper with Logging { /** * Collect all temporary SQL variables and return the identifiers separately. */ - private def collectTemporaryVariables(child: LogicalPlan): Seq[Seq[String]] = { + def collectTemporaryVariables(child: LogicalPlan): Seq[Seq[String]] = { def collectTempVars(child: LogicalPlan): Seq[Seq[String]] = { child.flatMap { plan => plan.expressions.flatMap(_.flatMap { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala new file mode 100644 index 0000000000000..75b42c6440719 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/CreateSQLFunctionParserSuite.scala @@ -0,0 +1,203 @@ +/* + * 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.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.catalog.LanguageSQL +import org.apache.spark.sql.catalyst.plans.logical.CreateUserDefinedFunction +import org.apache.spark.sql.execution.SparkSqlParser + +class CreateSQLFunctionParserSuite extends AnalysisTest { + private lazy val parser = new SparkSqlParser() + + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parser.parsePlan)(sqlCommand, messages: _*)() + + private def checkParseError( + sqlCommand: String, + errorClass: String, + parameters: Map[String, String], + queryContext: Array[ExpectedContext] = Array.empty): Unit = + assertParseErrorClass(parser.parsePlan, sqlCommand, errorClass, parameters, queryContext) + + // scalastyle:off argcount + private def createSQLFunction( + nameParts: Seq[String], + inputParamText: Option[String] = None, + returnTypeText: String = "INT", + exprText: Option[String] = None, + queryText: Option[String] = None, + comment: Option[String] = None, + isDeterministic: Option[Boolean] = None, + containsSQL: Option[Boolean] = None, + isTableFunc: Boolean = false, + ignoreIfExists: Boolean = false, + replace: Boolean = false): CreateUserDefinedFunction = { + // scalastyle:on argcount + CreateUserDefinedFunction( + UnresolvedIdentifier(nameParts), + inputParamText = inputParamText, + returnTypeText = returnTypeText, + exprText = exprText, + queryText = queryText, + comment = comment, + isDeterministic = isDeterministic, + containsSQL = containsSQL, + language = LanguageSQL, + isTableFunc = isTableFunc, + ignoreIfExists = ignoreIfExists, + replace = replace) + } + + // scalastyle:off argcount + private def createSQLFunctionCommand( + name: String, + inputParamText: Option[String] = None, + returnTypeText: String = "INT", + exprText: Option[String] = None, + queryText: Option[String] = None, + comment: Option[String] = None, + isDeterministic: Option[Boolean] = None, + containsSQL: Option[Boolean] = None, + isTableFunc: Boolean = false, + ignoreIfExists: Boolean = false, + replace: Boolean = false): CreateSQLFunctionCommand = { + // scalastyle:on argcount + CreateSQLFunctionCommand( + FunctionIdentifier(name), + inputParamText = inputParamText, + returnTypeText = returnTypeText, + exprText = exprText, + queryText = queryText, + comment = comment, + isDeterministic = isDeterministic, + containsSQL = containsSQL, + isTableFunc = isTableFunc, + isTemp = true, + ignoreIfExists = ignoreIfExists, + replace = replace) + } + + test("create temporary SQL functions") { + comparePlans( + parser.parsePlan("CREATE TEMPORARY FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunctionCommand("a", exprText = Some("1"))) + + comparePlans( + parser.parsePlan( + "CREATE TEMPORARY FUNCTION a(x INT) RETURNS TABLE (a INT) RETURN SELECT x"), + createSQLFunctionCommand( + name = "a", + inputParamText = Some("x INT"), + returnTypeText = "a INT", + queryText = Some("SELECT x"), + isTableFunc = true)) + + comparePlans( + parser.parsePlan("CREATE OR REPLACE TEMPORARY FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunctionCommand("a", exprText = Some("1"), replace = true)) + + checkParseError( + "CREATE TEMPORARY FUNCTION a.b() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_DATABASE", + parameters = Map("database" -> "`a`"), + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION a.b() RETURNS INT RETURN 1", 0, 51) + ) + ) + + checkParseError( + "CREATE TEMPORARY FUNCTION a.b.c() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.MULTI_PART_NAME", + parameters = Map( + "statement" -> "CREATE TEMPORARY FUNCTION", + "name" -> "`a`.`b`.`c`"), + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION a.b.c() RETURNS INT RETURN 1", 0, 53) + ) + ) + + checkParseError( + "CREATE TEMPORARY FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", + errorClass = "INVALID_SQL_SYNTAX.CREATE_TEMP_FUNC_WITH_IF_NOT_EXISTS", + parameters = Map.empty, + queryContext = Array( + ExpectedContext("CREATE TEMPORARY FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", 0, 63) + ) + ) + } + + test("create persistent SQL functions") { + comparePlans( + parser.parsePlan("CREATE FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"))) + + comparePlans( + parser.parsePlan("CREATE FUNCTION a.b(x INT) RETURNS INT RETURN x"), + createSQLFunction(Seq("a", "b"), Some("x INT"), exprText = Some("x"))) + + comparePlans(parser.parsePlan( + "CREATE FUNCTION a.b.c(x INT) RETURNS TABLE (a INT) RETURN SELECT x"), + createSQLFunction(Seq("a", "b", "c"), Some("x INT"), returnTypeText = "a INT", None, + Some("SELECT x"), isTableFunc = true)) + + comparePlans(parser.parsePlan("CREATE FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"), ignoreIfExists = true) + ) + + comparePlans(parser.parsePlan("CREATE OR REPLACE FUNCTION a() RETURNS INT RETURN 1"), + createSQLFunction(Seq("a"), exprText = Some("1"), replace = true)) + + comparePlans( + parser.parsePlan( + """ + |CREATE FUNCTION a(x INT COMMENT 'x') RETURNS INT + |LANGUAGE SQL DETERMINISTIC CONTAINS SQL + |COMMENT 'function' + |RETURN x + |""".stripMargin), + createSQLFunction(Seq("a"), inputParamText = Some("x INT COMMENT 'x'"), + exprText = Some("x"), isDeterministic = Some(true), containsSQL = Some(true), + comment = Some("function")) + ) + + intercept("CREATE OR REPLACE FUNCTION IF NOT EXISTS a() RETURNS INT RETURN 1", + "Cannot create a routine with both IF NOT EXISTS and REPLACE specified") + } + + test("create SQL functions with unsupported routine characteristics") { + intercept("CREATE FUNCTION foo() RETURNS INT LANGUAGE blah RETURN 1", + "Operation not allowed: Unsupported language for user defined functions: blah") + + intercept("CREATE FUNCTION foo() RETURNS INT SPECIFIC foo1 RETURN 1", + "Operation not allowed: SQL function with SPECIFIC name is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT NO SQL RETURN 1", + "Operation not allowed: SQL function with NO SQL is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT NO SQL CONTAINS SQL RETURN 1", + "Found duplicate clauses: SQL DATA ACCESS") + + intercept("CREATE FUNCTION foo() RETURNS INT RETURNS NULL ON NULL INPUT RETURN 1", + "Operation not allowed: SQL function with RETURNS NULL ON NULL INPUT is not supported") + + intercept("CREATE FUNCTION foo() RETURNS INT SQL SECURITY INVOKER RETURN 1", + "Operation not allowed: SQL function with SQL SECURITY INVOKER is not supported") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala index d38708ab3745c..3dea8593b428d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLParserSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.command import org.apache.spark.SparkThrowable -import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, GlobalTempView, LocalTempView, SchemaCompensation, UnresolvedAttribute, UnresolvedFunctionName, UnresolvedIdentifier} import org.apache.spark.sql.catalyst.catalog.{ArchiveResource, FileResource, FunctionResource, JarResource} import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -37,9 +36,6 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { super.parseException(parser.parsePlan)(sqlText) } - private def intercept(sqlCommand: String, messages: String*): Unit = - interceptParseException(parser.parsePlan)(sqlCommand, messages: _*)() - private def compareTransformQuery(sql: String, expected: LogicalPlan): Unit = { val plan = parser.parsePlan(sql).asInstanceOf[ScriptTransformation].copy(ioschema = null) comparePlans(plan, expected, checkAnalysis = false) @@ -827,44 +823,4 @@ class DDLParserSuite extends AnalysisTest with SharedSparkSession { parser.parsePlan("SHOW CATALOGS LIKE 'defau*'"), ShowCatalogsCommand(Some("defau*"))) } - - test("Create SQL functions") { - comparePlans( - parser.parsePlan("CREATE TEMP FUNCTION foo() RETURNS INT RETURN 1"), - CreateSQLFunctionCommand( - FunctionIdentifier("foo"), - inputParamText = None, - returnTypeText = "INT", - exprText = Some("1"), - queryText = None, - comment = None, - isDeterministic = None, - containsSQL = None, - isTableFunc = false, - isTemp = true, - ignoreIfExists = false, - replace = false)) - intercept("CREATE FUNCTION foo() RETURNS INT RETURN 1", - "Operation not allowed: creating persistent SQL functions is not supported") - } - - test("create SQL functions with unsupported routine characteristics") { - intercept("CREATE FUNCTION foo() RETURNS INT LANGUAGE blah RETURN 1", - "Operation not allowed: Unsupported language for user defined functions: blah") - - intercept("CREATE FUNCTION foo() RETURNS INT SPECIFIC foo1 RETURN 1", - "Operation not allowed: SQL function with SPECIFIC name is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT NO SQL RETURN 1", - "Operation not allowed: SQL function with NO SQL is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT NO SQL CONTAINS SQL RETURN 1", - "Found duplicate clauses: SQL DATA ACCESS") - - intercept("CREATE FUNCTION foo() RETURNS INT RETURNS NULL ON NULL INPUT RETURN 1", - "Operation not allowed: SQL function with RETURNS NULL ON NULL INPUT is not supported") - - intercept("CREATE FUNCTION foo() RETURNS INT SQL SECURITY INVOKER RETURN 1", - "Operation not allowed: SQL function with SQL SECURITY INVOKER is not supported") - } } From e68b98a35cfed0930f59f30b2f8a542936a6a480 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Tue, 7 Jan 2025 10:16:43 +0800 Subject: [PATCH 57/60] [MINOR][PYTHON][TESTS] Restore daily coverage build ### What changes were proposed in this pull request? Skip `test_value_state_ttl_expiration` in daily coverage build ### Why are the changes needed? to restore daily coverage build https://github.com/apache/spark/actions/runs/12630507164 ### Does this PR introduce _any_ user-facing change? no ### How was this patch tested? PR build with ``` default: '{"PYSPARK_IMAGE_TO_TEST": "python-311", "PYTHON_TO_TEST": "python3.11", "PYSPARK_CODECOV": "true"}' default: '{"pyspark": "true"}' ``` ### Was this patch authored or co-authored using generative AI tooling? no Closes #49388 from zhengruifeng/py_fix_cov. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../sql/tests/pandas/test_pandas_transform_with_state.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py index 7a00f2f43d149..516a95a91a5e7 100644 --- a/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py +++ b/python/pyspark/sql/tests/pandas/test_pandas_transform_with_state.py @@ -314,6 +314,9 @@ def check_results(batch_df, batch_id): SimpleTTLStatefulProcessor(), check_results, False, "processingTime" ) + @unittest.skipIf( + "COVERAGE_PROCESS_START" in os.environ, "Flaky with coverage enabled, skipping for now." + ) def test_value_state_ttl_expiration(self): def check_results(batch_df, batch_id): if batch_id == 0: From 0467aca97120e493a3b02c5ff9460042dcdf233d Mon Sep 17 00:00:00 2001 From: Junfan Zhang Date: Mon, 6 Jan 2025 20:00:08 -0800 Subject: [PATCH 58/60] [SPARK-49783][YARN] Fix resource leak of yarn allocator ### What changes were proposed in this pull request? Fix the resource leak of yarn allocator ### Why are the changes needed? When the target < running containers number, the assigned containers from the resource manager will be skipped, but these containers are not released by invoking the amClient.releaseAssignedContainer , that will make these containers reserved into the Yarn resourceManager at least 10 minutes. And so, the cluster resource will be wasted at a high ratio. And this will reflect that the vcore * seconds statistics from yarn side will be greater than the result from the spark event logs. From my statistics, the cluster resource waste ratio is ~25% if the spark jobs are exclusive in this cluster. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? In our internal hadoop cluster ### Was this patch authored or co-authored using generative AI tooling? No Closes #48238 from zuston/patch-1. Authored-by: Junfan Zhang Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index b593d73f85721..911ce2e27f96d 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -820,6 +820,7 @@ private[yarn] class YarnAllocator( logInfo(log"Skip launching executorRunnable as running executors count: " + log"${MDC(LogKeys.COUNT, rpRunningExecs)} reached target executors count: " + log"${MDC(LogKeys.NUM_EXECUTOR_TARGET, getOrUpdateTargetNumExecutorsForRPId(rpId))}.") + internalReleaseContainer(container) } } } From b3182e5521b9aca70e9dc029fcb47e1185602629 Mon Sep 17 00:00:00 2001 From: Vladimir Golubev Date: Tue, 7 Jan 2025 12:08:36 +0800 Subject: [PATCH 59/60] [SPARK-50744][SQL] Add a test case for view/CTE name resolution precedence MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Add an important test case for CTE resolution: ``` CREATE VIEW v1 AS SELECT 1; CREATE VIEW v2 AS SELECT * FROM v1; – The result is 1. – The `v2` body will be inlined in the main query tree during the analysis, but upper `v1` – CTE definition won't take precedence over the lower `v1` view. WITH v1 AS ( SELECT 2 ) SELECT * FROM v2; ``` This is an exception to the usual "CTE name takes precedence over the table/view name". ### Why are the changes needed? To harden Spark testing. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? New test case. ### Was this patch authored or co-authored using generative AI tooling? No. Closes #49378 from vladimirg-db/vladimirg-db/add-cte-vs-view-test-case. Authored-by: Vladimir Golubev Signed-off-by: Wenchen Fan --- .../sql-tests/analyzer-results/cte.sql.out | 40 +++++++++++++++++++ .../test/resources/sql-tests/inputs/cte.sql | 5 +++ .../resources/sql-tests/results/cte.sql.out | 26 ++++++++++++ 3 files changed, 71 insertions(+) diff --git a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out index ea09573db51a5..cdd3698ce9af4 100644 --- a/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/analyzer-results/cte.sql.out @@ -17,6 +17,19 @@ CreateViewCommand `t2`, select * from values 0, 1 as t(id), false, false, LocalT +- LocalRelation [id#x] +-- !query +create temporary view t3 as select * from t +-- !query analysis +CreateViewCommand `t3`, select * from t, false, false, LocalTempView, UNSUPPORTED, true + +- Project [id#x] + +- SubqueryAlias t + +- View (`t`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + -- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query analysis @@ -76,6 +89,27 @@ WithCTE +- CTERelationRef xxxx, true, [1#x], false, false +-- !query +WITH t AS (SELECT 1) SELECT * FROM t3 +-- !query analysis +WithCTE +:- CTERelationDef xxxx, false +: +- SubqueryAlias t +: +- Project [1 AS 1#x] +: +- OneRowRelation ++- Project [id#x] + +- SubqueryAlias t3 + +- View (`t3`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- View (`t`, [id#x]) + +- Project [cast(id#x as int) AS id#x] + +- Project [id#x] + +- SubqueryAlias t + +- LocalRelation [id#x] + + -- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query analysis @@ -778,3 +812,9 @@ DropTempViewCommand t DROP VIEW IF EXISTS t2 -- !query analysis DropTempViewCommand t2 + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query analysis +DropTempViewCommand t3 diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte.sql b/sql/core/src/test/resources/sql-tests/inputs/cte.sql index 67a94ce61617d..1e17529d545bf 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte.sql @@ -1,5 +1,6 @@ create temporary view t as select * from values 0, 1, 2 as t(id); create temporary view t2 as select * from values 0, 1 as t(id); +create temporary view t3 as select * from t; -- WITH clause should not fall into infinite loop by referencing self WITH s AS (SELECT 1 FROM s) SELECT * FROM s; @@ -10,6 +11,9 @@ SELECT * FROM r; -- WITH clause should reference the base table WITH t AS (SELECT 1 FROM t) SELECT * FROM t; +-- Table `t` referenced by a view should take precedence over the top CTE `t` +WITH t AS (SELECT 1) SELECT * FROM t3; + -- WITH clause should not allow cross reference WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2; @@ -175,3 +179,4 @@ with cte as (select * from cte) select * from cte; -- Clean up DROP VIEW IF EXISTS t; DROP VIEW IF EXISTS t2; +DROP VIEW IF EXISTS t3; diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index 754a8832ef6c5..4367ae1d5f2d0 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -15,6 +15,14 @@ struct<> +-- !query +create temporary view t3 as select * from t +-- !query schema +struct<> +-- !query output + + + -- !query WITH s AS (SELECT 1 FROM s) SELECT * FROM s -- !query schema @@ -70,6 +78,16 @@ struct<1:int> 1 +-- !query +WITH t AS (SELECT 1) SELECT * FROM t3 +-- !query schema +struct +-- !query output +0 +1 +2 + + -- !query WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 -- !query schema @@ -580,3 +598,11 @@ DROP VIEW IF EXISTS t2 struct<> -- !query output + + +-- !query +DROP VIEW IF EXISTS t3 +-- !query schema +struct<> +-- !query output + From 22cbb9694ca53efef1d57387e14976d3906c2b15 Mon Sep 17 00:00:00 2001 From: Chenhao Li Date: Tue, 7 Jan 2025 13:55:46 +0800 Subject: [PATCH 60/60] [SPARK-50746][SQL] Replace Either with VariantPathSegment ### What changes were proposed in this pull request? It replaces `type PathSegment = Either[String, Int]` with a dedicated class `VariantPathSegment`. There is no semantic change, but the code has clear naming. ### Why are the changes needed? To make the code easier to understand. ### 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 #49385 from chenhao-db/VariantPathSegment. Authored-by: Chenhao Li Signed-off-by: Wenchen Fan --- .../variant/variantExpressions.scala | 30 +++++++++++-------- .../datasources/PushVariantIntoScan.scala | 4 +-- .../parquet/SparkShreddingUtils.scala | 15 +++++----- 3 files changed, 26 insertions(+), 23 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala index ba910b8c7e5fd..ff8b168793b5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/variant/variantExpressions.scala @@ -184,33 +184,37 @@ case class ToVariantObject(child: Expression) } } -object VariantPathParser extends RegexParsers { - // A path segment in the `VariantGet` expression represents either an object key access or an - // array index access. - type PathSegment = Either[String, Int] +// A path segment in the `VariantGet` expression represents either an object key access or an array +// index access. +sealed abstract class VariantPathSegment extends Serializable + +case class ObjectExtraction(key: String) extends VariantPathSegment +case class ArrayExtraction(index: Int) extends VariantPathSegment + +object VariantPathParser extends RegexParsers { private def root: Parser[Char] = '$' // Parse index segment like `[123]`. - private def index: Parser[PathSegment] = + private def index: Parser[VariantPathSegment] = for { index <- '[' ~> "\\d+".r <~ ']' } yield { - scala.util.Right(index.toInt) + ArrayExtraction(index.toInt) } // Parse key segment like `.name`, `['name']`, or `["name"]`. - private def key: Parser[PathSegment] = + private def key: Parser[VariantPathSegment] = for { key <- '.' ~> "[^\\.\\[]+".r | "['" ~> "[^\\'\\?]+".r <~ "']" | "[\"" ~> "[^\\\"\\?]+".r <~ "\"]" } yield { - scala.util.Left(key) + ObjectExtraction(key) } - private val parser: Parser[List[PathSegment]] = phrase(root ~> rep(key | index)) + private val parser: Parser[List[VariantPathSegment]] = phrase(root ~> rep(key | index)) - def parse(str: String): Option[Array[PathSegment]] = { + def parse(str: String): Option[Array[VariantPathSegment]] = { this.parseAll(parser, str) match { case Success(result, _) => Some(result.toArray) case _ => None @@ -349,14 +353,14 @@ case object VariantGet { /** The actual implementation of the `VariantGet` expression. */ def variantGet( input: VariantVal, - parsedPath: Array[VariantPathParser.PathSegment], + parsedPath: Array[VariantPathSegment], dataType: DataType, castArgs: VariantCastArgs): Any = { var v = new Variant(input.getValue, input.getMetadata) for (path <- parsedPath) { v = path match { - case scala.util.Left(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) - case scala.util.Right(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) + case ObjectExtraction(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) + case ArrayExtraction(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) case _ => null } if (v == null) return null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala index 83d219c28983b..33ba4f772a13a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PushVariantIntoScan.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.HashMap import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.variant.{VariantGet, VariantPathParser} +import org.apache.spark.sql.catalyst.expressions.variant._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, Subquery} import org.apache.spark.sql.catalyst.rules.Rule @@ -54,7 +54,7 @@ case class VariantMetadata( .build() ).build() - def parsedPath(): Array[VariantPathParser.PathSegment] = { + def parsedPath(): Array[VariantPathSegment] = { VariantPathParser.parse(path).getOrElse { val name = if (failOnError) "variant_get" else "try_variant_get" throw QueryExecutionErrors.invalidVariantGetPath(path, name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala index c0c4900344154..ffb6704061e66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/SparkShreddingUtils.scala @@ -25,7 +25,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.expressions.variant._ -import org.apache.spark.sql.catalyst.expressions.variant.VariantPathParser.PathSegment import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils, GenericArrayData} import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.execution.RowToColumnConverter @@ -56,9 +55,9 @@ case class SparkShreddedRow(row: SpecializedGetters) extends ShreddingUtils.Shre override def numElements(): Int = row.asInstanceOf[ArrayData].numElements() } -// The search result of a `PathSegment` in a `VariantSchema`. +// The search result of a `VariantPathSegment` in a `VariantSchema`. case class SchemaPathSegment( - rawPath: PathSegment, + rawPath: VariantPathSegment, // Whether this path segment is an object or array extraction. isObject: Boolean, // `schema.typedIdx`, if the path exists in the schema (for object extraction, the schema @@ -714,11 +713,11 @@ case object SparkShreddingUtils { // found at a certain level of the file type, then `typedIdx` will be -1 starting from // this position, and the final `schema` will be null. for (i <- rawPath.indices) { - val isObject = rawPath(i).isLeft + val isObject = rawPath(i).isInstanceOf[ObjectExtraction] var typedIdx = -1 var extractionIdx = -1 rawPath(i) match { - case scala.util.Left(key) if schema != null && schema.objectSchema != null => + case ObjectExtraction(key) if schema != null && schema.objectSchema != null => val fieldIdx = schema.objectSchemaMap.get(key) if (fieldIdx != null) { typedIdx = schema.typedIdx @@ -727,7 +726,7 @@ case object SparkShreddingUtils { } else { schema = null } - case scala.util.Right(index) if schema != null && schema.arraySchema != null => + case ArrayExtraction(index) if schema != null && schema.arraySchema != null => typedIdx = schema.typedIdx extractionIdx = index schema = schema.arraySchema @@ -770,8 +769,8 @@ case object SparkShreddingUtils { var v = new Variant(row.getBinary(variantIdx), topLevelMetadata) while (pathIdx < pathLen) { v = pathList(pathIdx).rawPath match { - case scala.util.Left(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) - case scala.util.Right(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) + case ObjectExtraction(key) if v.getType == Type.OBJECT => v.getFieldByKey(key) + case ArrayExtraction(index) if v.getType == Type.ARRAY => v.getElementAtIndex(index) case _ => null } if (v == null) return null