diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index eb166a1e8003b..ffaa8a70cc7c4 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -383,20 +383,44 @@ class Dataset[T] private[sql] ( } } - // TODO(SPARK-50134): Support Lateral Join API in Spark Connect - // scalastyle:off not.implemented.error.usage + private def lateralJoin( + right: DS[_], + joinExprs: Option[Column], + joinType: String): DataFrame = { + val joinTypeValue = toJoinType(joinType) + joinTypeValue match { + case proto.Join.JoinType.JOIN_TYPE_INNER | proto.Join.JoinType.JOIN_TYPE_LEFT_OUTER | + proto.Join.JoinType.JOIN_TYPE_CROSS => + case _ => + throw new IllegalArgumentException(s"Unsupported lateral join type $joinType") + } + sparkSession.newDataFrame { builder => + val lateralJoinBuilder = builder.getLateralJoinBuilder + lateralJoinBuilder.setLeft(plan.getRoot).setRight(right.plan.getRoot) + joinExprs.foreach(c => lateralJoinBuilder.setJoinCondition(c.expr)) + lateralJoinBuilder.setJoinType(joinTypeValue) + } + } + /** @inheritdoc */ - def lateralJoin(right: DS[_]): DataFrame = ??? + def lateralJoin(right: DS[_]): DataFrame = { + lateralJoin(right, None, "inner") + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = ??? + def lateralJoin(right: DS[_], joinExprs: Column): DataFrame = { + lateralJoin(right, Some(joinExprs), "inner") + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinType: String): DataFrame = ??? + def lateralJoin(right: DS[_], joinType: String): DataFrame = { + lateralJoin(right, None, joinType) + } /** @inheritdoc */ - def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = ??? - // scalastyle:on not.implemented.error.usage + def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = { + lateralJoin(right, Some(joinExprs), joinType) + } override protected def sortInternal(global: Boolean, sortCols: Seq[Column]): Dataset[T] = { val sortExprs = sortCols.map { c => diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 4690253da808b..993b09ace9139 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -17,7 +17,8 @@ package org.apache.spark.sql /** @inheritdoc */ -abstract class SQLImplicits private[sql] (override val session: SparkSession) - extends api.SQLImplicits { +abstract class SQLImplicits extends api.SQLImplicits { type DS[U] = Dataset[U] + + protected def session: SparkSession } diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala index 3183a155c16a3..939a1341a8911 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -301,7 +301,9 @@ class SparkSession private[sql] ( // scalastyle:off /** @inheritdoc */ - object implicits extends SQLImplicits(this) + object implicits extends SQLImplicits { + override protected def session: SparkSession = SparkSession.this + } // scalastyle:on /** @inheritdoc */ 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 34a8a91a0ddf8..0e8889e19de22 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 @@ -164,6 +164,9 @@ object ColumnNodeToProtoConverter extends (ColumnNode => proto.Expression) { b.addArguments(apply(value, e)) } + case LazyExpression(child, _) => + builder.getLazyExpressionBuilder.setChild(apply(child, e)) + case ProtoColumnNode(e, _) => return e 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 new file mode 100644 index 0000000000000..91f60b1fefb91 --- /dev/null +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/DataFrameSubquerySuite.scala @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql + +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} + +class DataFrameSubquerySuite extends QueryTest with RemoteSparkSession { + import testImplicits._ + + private def table1() = { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + spark.table("t1") + } + + private def table2() = { + sql("CREATE VIEW t2(c1, c2) AS VALUES (0, 2), (0, 3)") + spark.table("t2") + } + + private def table3() = { + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + spark.table("t3") + } + + test("lateral join with single column select") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(spark.range(1).select($"c1".outer())).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c1)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"c1")).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c1 FROM t2)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer())).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 FROM t2)").toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.select($"t1.c1".outer() + $"t2.c1")).toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT t1.c1 + t2.c1 FROM t2)").toDF("c1", "c2", "c3")) + } + } + + test("lateral join with different join types") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3"), + sql("SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3")) + checkAnswer( + t1.lateralJoin( + spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), + $"c2" === $"c3", + "left"), + sql("SELECT * FROM t1 LEFT JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3")) + checkAnswer( + t1.lateralJoin(spark.range(1).select(($"c1".outer() + $"c2".outer()).as("c3")), "cross"), + sql("SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)")) + } + } + + test("lateral join with correlated equality / non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() === $"t2.c1").select($"c2")) + .toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1)") + .toDF("c1", "c2", "c3")) + checkAnswer( + t1.lateralJoin(t2.where($"t1.c1".outer() < $"t2.c1").select($"c2")) + .toDF("c1", "c2", "c3"), + sql("SELECT * FROM t1, LATERAL (SELECT c2 FROM t2 WHERE t1.c1 < t2.c1)") + .toDF("c1", "c2", "c3")) + } + } + + test("lateral join with aggregation and correlated non-equality predicates") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.where($"t1.c2".outer() < $"t2.c2").select(max($"c2").as("m"))), + sql("SELECT * FROM t1, LATERAL (SELECT max(c2) AS m FROM t2 WHERE t1.c2 < t2.c2)")) + } + } + + test("lateral join can reference preceding FROM clause items") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.join(t2) + .lateralJoin(spark.range(1).select($"t1.c2".outer() + $"t2.c2".outer())) + .toDF("c1", "c2", "c3", "c4", "c5"), + sql("SELECT * FROM t1 JOIN t2 JOIN LATERAL (SELECT t1.c2 + t2.c2)") + .toDF("c1", "c2", "c3", "c4", "c5")) + } + } + + test("multiple lateral joins") { + withView("t1") { + val t1 = table1() + + checkAnswer( + t1.lateralJoin(spark.range(1).select(($"c1".outer() + $"c2".outer()).as("a"))) + .lateralJoin(spark.range(1).select(($"c1".outer() - $"c2".outer()).as("b"))) + .lateralJoin(spark.range(1).select(($"a".outer() * $"b".outer()).as("c"))), + sql(""" + |SELECT * FROM t1, + |LATERAL (SELECT c1 + c2 AS a), + |LATERAL (SELECT c1 - c2 AS b), + |LATERAL (SELECT a * b AS c) + |""".stripMargin)) + } + } + + test("lateral join in between regular joins") { + withView("t1", "t2") { + val t1 = table1() + 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") + .toDF("c1", "c2", "c3", "c4", "c5"), + sql(""" + |SELECT * FROM t1 + |LEFT OUTER JOIN LATERAL (SELECT c2 FROM t2 WHERE t1.c1 = t2.c1) s + |LEFT OUTER JOIN t1 t3 ON s.c2 = t3.c2 + |""".stripMargin) + .toDF("c1", "c2", "c3", "c4", "c5")) + } + } + + test("nested lateral joins") { + withView("t1", "t2") { + val t1 = table1() + val t2 = table2() + + checkAnswer( + t1.lateralJoin(t2.lateralJoin(spark.range(1).select($"c1".outer()))) + .toDF("c1", "c2", "c3", "c4", "c5"), + sql("SELECT * FROM t1, LATERAL (SELECT * FROM t2, LATERAL (SELECT c1))") + .toDF("c1", "c2", "c3", "c4", "c5")) + checkAnswer( + t1.lateralJoin( + spark + .range(1) + .select(($"c1".outer() + lit(1)).as("c1")) + .lateralJoin(spark.range(1).select($"c1".outer()))) + .toDF("c1", "c2", "c3", "c4"), + sql( + "SELECT * FROM t1, LATERAL (SELECT * FROM (SELECT c1 + 1 AS c1), LATERAL (SELECT c1))") + .toDF("c1", "c2", "c3", "c4")) + } + } + + test("lateral join with table-valued functions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer(t1.lateralJoin(spark.tvf.range(3)), sql("SELECT * FROM t1, LATERAL RANGE(3)")) + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + 1)), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + + test("lateral join with table-valued functions and join conditions") { + withView("t1", "t3") { + val t1 = table1() + val t3 = table3() + + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer())), $"c1" === $"col"), + sql("SELECT * FROM t1 JOIN LATERAL EXPLODE(ARRAY(c1, c2)) t(c3) ON t1.c1 = c3")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer()), $"c1" === $"col"), + sql("SELECT * FROM t3 JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer()), $"c1" === $"col", "left"), + sql("SELECT * FROM t3 LEFT JOIN LATERAL EXPLODE(c2) t(c3) ON t3.c1 = c3")) + } + } +} 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 f4a0d1bcf5d6f..aeef2e8f0fcf2 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 @@ -21,6 +21,7 @@ import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.{QueryTest, RemoteSparkSession} class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSession { + import testImplicits._ test("explode") { val actual1 = spark.tvf.explode(array(lit(1), lit(2))) @@ -50,6 +51,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("explode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + test("explode_outer") { val actual1 = spark.tvf.explode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM explode_outer(array(1, 2))") @@ -78,6 +103,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("explode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.explode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL EXPLODE_OUTER(ARRAY(c1, c2)) t2(c3)")) + checkAnswer( + t3.lateralJoin(spark.tvf.explode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL EXPLODE_OUTER(c2) t2(v)")) + checkAnswer( + spark.tvf + .explode_outer(array(lit(1), lit(2))) + .toDF("v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM EXPLODE_OUTER(ARRAY(1, 2)) t(v), LATERAL (SELECT v + 1)")) + } + } + test("inline") { val actual1 = spark.tvf.inline(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) val expected1 = spark.sql("SELECT * FROM inline(array(struct(1, 'a'), struct(2, 'b')))") @@ -98,6 +147,25 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual3, expected3) } + test("inline - lateral join") { + withView("array_struct") { + sql(""" + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + 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"), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE(arr) t(k, v) ON id = k")) + } + } + test("inline_outer") { val actual1 = spark.tvf.inline_outer(array(struct(lit(1), lit("a")), struct(lit(2), lit("b")))) @@ -119,6 +187,28 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual3, expected3) } + test("inline_outer - lateral join") { + withView("array_struct") { + sql(""" + |CREATE VIEW array_struct(id, arr) AS VALUES + | (1, ARRAY(STRUCT(1, 'a'), STRUCT(2, 'b'))), + | (2, ARRAY()), + | (3, ARRAY(STRUCT(3, 'c'))) + |""".stripMargin) + val arrayStruct = spark.table("array_struct") + + checkAnswer( + arrayStruct.lateralJoin(spark.tvf.inline_outer($"arr".outer())), + sql("SELECT * FROM array_struct JOIN LATERAL INLINE_OUTER(arr)")) + checkAnswer( + arrayStruct.lateralJoin( + spark.tvf.inline_outer($"arr".outer()), + $"id" === $"col1", + "left"), + sql("SELECT * FROM array_struct LEFT JOIN LATERAL INLINE_OUTER(arr) t(k, v) ON id = k")) + } + } + test("json_tuple") { val actual = spark.tvf.json_tuple(lit("""{"a":1,"b":2}"""), lit("a"), lit("b")) val expected = spark.sql("""SELECT * FROM json_tuple('{"a":1,"b":2}', 'a', 'b')""") @@ -131,6 +221,54 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi assert(ex.messageParameters("functionName") == "`json_tuple`") } + test("json_tuple - lateral join") { + withView("json_table") { + sql(""" + |CREATE OR REPLACE TEMP VIEW json_table(key, jstring) AS VALUES + | ('1', '{"f1": "1", "f2": "2", "f3": 3, "f5": 5.23}'), + | ('2', '{"f1": "1", "f3": "3", "f2": 2, "f4": 4.01}'), + | ('3', '{"f1": 3, "f4": "4", "f3": "3", "f2": 2, "f5": 5.01}'), + | ('4', cast(null as string)), + | ('5', '{"f1": null, "f5": ""}'), + | ('6', '[invalid JSON string]') + |""".stripMargin) + val jsonTable = spark.table("json_table") + + checkAnswer( + 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"), + 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"), + 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")) + } + } + test("posexplode") { val actual1 = spark.tvf.posexplode(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode(array(1, 2))") @@ -159,6 +297,30 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("posexplode - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE(ARRAY(c1, c2))")) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE(c2)")) + checkAnswer( + spark.tvf + .posexplode(array(lit(1), lit(2))) + .toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)")) + } + } + test("posexplode_outer") { val actual1 = spark.tvf.posexplode_outer(array(lit(1), lit(2))) val expected1 = spark.sql("SELECT * FROM posexplode_outer(array(1, 2))") @@ -187,12 +349,62 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("posexplode_outer - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin(spark.tvf.posexplode_outer(array($"c1".outer(), $"c2".outer()))), + sql("SELECT * FROM t1, LATERAL POSEXPLODE_OUTER(ARRAY(c1, c2))")) + checkAnswer( + t3.lateralJoin(spark.tvf.posexplode_outer($"c2".outer())), + sql("SELECT * FROM t3, LATERAL POSEXPLODE_OUTER(c2)")) + checkAnswer( + spark.tvf + .posexplode_outer(array(lit(1), lit(2))) + .toDF("p", "v") + .lateralJoin(spark.range(1).select($"v".outer() + lit(1))), + sql("SELECT * FROM POSEXPLODE_OUTER(ARRAY(1, 2)) t(p, v), LATERAL (SELECT v + 1)")) + } + } + test("stack") { val actual = spark.tvf.stack(lit(2), lit(1), lit(2), lit(3)) val expected = spark.sql("SELECT * FROM stack(2, 1, 2, 3)") checkAnswer(actual, expected) } + test("stack - lateral join") { + withView("t1", "t3") { + sql("CREATE VIEW t1(c1, c2) AS VALUES (0, 1), (1, 2)") + sql( + "CREATE VIEW t3(c1, c2) AS " + + "VALUES (0, ARRAY(0, 1)), (1, ARRAY(2)), (2, ARRAY()), (null, ARRAY(4))") + val t1 = spark.table("t1") + val t3 = spark.table("t3") + + checkAnswer( + t1.lateralJoin( + spark.tvf.stack(lit(2), lit("Key"), $"c1".outer(), lit("Value"), $"c2".outer())) + .select($"col0", $"col1"), + 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")), + 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"), + sql("SELECT t.* FROM t1 JOIN t3 ON t1.c1 = t3.c1 JOIN LATERAL stack(1, t1.c2, t3.c2) t")) + } + } + test("collations") { val actual = spark.tvf.collations() val expected = spark.sql("SELECT * FROM collations()") @@ -236,6 +448,27 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi checkAnswer(actual6, expected6) } + test("variant_explode - lateral join") { + withView("variant_table") { + sql(""" + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable + .as("t1") + .lateralJoin(spark.tvf.variant_explode($"v".outer())) + .select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode(v) AS t")) + } + } + test("variant_explode_outer") { val actual1 = spark.tvf.variant_explode_outer(parse_json(lit("""["hello", "world"]"""))) val expected1 = @@ -266,4 +499,25 @@ class DataFrameTableValuedFunctionsSuite extends QueryTest with RemoteSparkSessi val expected6 = spark.sql("SELECT * FROM variant_explode_outer(parse_json('1'))") checkAnswer(actual6, expected6) } + + test("variant_explode_outer - lateral join") { + withView("variant_table") { + sql(""" + |CREATE VIEW variant_table(id, v) AS + |SELECT id, parse_json(v) AS v FROM VALUES + |(0, '["hello", "world"]'), (1, '{"a": true, "b": 3.14}'), + |(2, '[]'), (3, '{}'), + |(4, NULL), (5, '1') + |AS t(id, v) + |""".stripMargin) + val variantTable = spark.table("variant_table") + + checkAnswer( + variantTable + .as("t1") + .lateralJoin(spark.tvf.variant_explode_outer($"v".outer())) + .select($"id", $"pos", $"key", $"value"), + sql("SELECT t1.id, t.* FROM variant_table AS t1, LATERAL variant_explode_outer(v) AS t")) + } + } } diff --git a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala index 4a574a15f7ab8..007d4f0648e44 100644 --- a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala +++ b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/test/SQLHelper.scala @@ -21,13 +21,27 @@ import java.util.UUID import org.scalatest.Assertions.fail -import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession, SQLImplicits} import org.apache.spark.util.{SparkErrorUtils, SparkFileUtils} trait SQLHelper { def spark: SparkSession + // Shorthand for running a query using our SparkSession + protected lazy val sql: String => DataFrame = spark.sql _ + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. This is + * because we create the `SparkSession` immediately before the first test is run, but the + * implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + override protected def session: SparkSession = spark + } + /** * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL * configurations. @@ -106,4 +120,13 @@ trait SQLHelper { } } } + + /** + * Drops view `viewName` after calling `f`. + */ + protected def withView(viewNames: String*)(f: => Unit): Unit = { + SparkErrorUtils.tryWithSafeFinally(f)(viewNames.foreach { name => + spark.sql(s"DROP VIEW IF EXISTS $name") + }) + } } diff --git a/python/pyspark/errors/error-conditions.json b/python/pyspark/errors/error-conditions.json index b2a68a83bfa70..dbb4bb0087e06 100644 --- a/python/pyspark/errors/error-conditions.json +++ b/python/pyspark/errors/error-conditions.json @@ -1075,7 +1075,7 @@ }, "UNSUPPORTED_JOIN_TYPE": { "message": [ - "Unsupported join type: . Supported join types include: 'inner', 'outer', 'full', 'fullouter', 'full_outer', 'leftouter', 'left', 'left_outer', 'rightouter', 'right', 'right_outer', 'leftsemi', 'left_semi', 'semi', 'leftanti', 'left_anti', 'anti', 'cross'." + "Unsupported join type: ''. Supported join types include: ." ] }, "UNSUPPORTED_LITERAL": { diff --git a/python/pyspark/sql/connect/column.py b/python/pyspark/sql/connect/column.py index 1440c4c2792bf..1e49afc0b158a 100644 --- a/python/pyspark/sql/connect/column.py +++ b/python/pyspark/sql/connect/column.py @@ -34,7 +34,6 @@ PySparkTypeError, PySparkAttributeError, PySparkValueError, - PySparkNotImplementedError, ) from pyspark.sql.types import DataType from pyspark.sql.utils import enum_to_value @@ -44,6 +43,7 @@ Expression, UnresolvedFunction, UnresolvedExtractValue, + LazyExpression, LiteralExpression, CaseWhen, SortOrder, @@ -466,11 +466,7 @@ def over(self, window: "WindowSpec") -> ParentColumn: # type: ignore[override] return Column(WindowExpression(windowFunction=self._expr, windowSpec=window)) def outer(self) -> ParentColumn: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "outer()"}, - ) + return Column(LazyExpression(self._expr)) def isin(self, *cols: Any) -> ParentColumn: if len(cols) == 1 and isinstance(cols[0], (list, set)): diff --git a/python/pyspark/sql/connect/dataframe.py b/python/pyspark/sql/connect/dataframe.py index 124ce5e0d39a5..ee1886b8ef290 100644 --- a/python/pyspark/sql/connect/dataframe.py +++ b/python/pyspark/sql/connect/dataframe.py @@ -692,10 +692,14 @@ def lateralJoin( on: Optional[Column] = None, how: Optional[str] = None, ) -> ParentDataFrame: - # TODO(SPARK-50134): Implement this method - raise PySparkNotImplementedError( - errorClass="NOT_IMPLEMENTED", - messageParameters={"feature": "lateralJoin()"}, + self._check_same_session(other) + if how is not None and isinstance(how, str): + how = how.lower().replace("_", "") + return DataFrame( + plan.LateralJoin( + left=self._plan, right=cast(plan.LogicalPlan, other._plan), on=on, how=how + ), + session=self._session, ) def _joinAsOf( @@ -2277,7 +2281,6 @@ def _test() -> None: # TODO(SPARK-50134): Support subquery in connect del pyspark.sql.dataframe.DataFrame.scalar.__doc__ del pyspark.sql.dataframe.DataFrame.exists.__doc__ - del pyspark.sql.dataframe.DataFrame.lateralJoin.__doc__ globs["spark"] = ( PySparkSession.builder.appName("sql.connect.dataframe tests") diff --git a/python/pyspark/sql/connect/expressions.py b/python/pyspark/sql/connect/expressions.py index 5a5320366f666..5d7b348f6d383 100644 --- a/python/pyspark/sql/connect/expressions.py +++ b/python/pyspark/sql/connect/expressions.py @@ -1153,3 +1153,18 @@ def to_plan(self, session: "SparkConnectClient") -> "proto.Expression": def __repr__(self) -> str: return f"{self._key} => {self._value}" + + +class LazyExpression(Expression): + def __init__(self, expr: Expression): + assert isinstance(expr, Expression) + super().__init__() + self._expr = expr + + def to_plan(self, session: "SparkConnectClient") -> proto.Expression: + expr = self._create_proto_expression() + expr.lazy_expression.child.CopyFrom(self._expr.to_plan(session)) + return expr + + def __repr__(self) -> str: + return f"lazy({self._expr})" diff --git a/python/pyspark/sql/connect/plan.py b/python/pyspark/sql/connect/plan.py index 34f11768bcbc0..c411baf17ce94 100644 --- a/python/pyspark/sql/connect/plan.py +++ b/python/pyspark/sql/connect/plan.py @@ -893,7 +893,35 @@ def __init__( else: raise AnalysisException( errorClass="UNSUPPORTED_JOIN_TYPE", - messageParameters={"join_type": how}, + messageParameters={ + "typ": how, + "supported": ( + "'" + + "', '".join( + [ + "inner", + "outer", + "full", + "fullouter", + "full_outer", + "leftouter", + "left", + "left_outer", + "rightouter", + "right", + "right_outer", + "leftsemi", + "left_semi", + "semi", + "leftanti", + "left_anti", + "anti", + "cross", + ] + ) + + "'" + ), + }, ) self.how = join_type @@ -1028,6 +1056,74 @@ def _repr_html_(self) -> str: """ +class LateralJoin(LogicalPlan): + def __init__( + self, + left: Optional[LogicalPlan], + right: LogicalPlan, + on: Optional[Column], + how: Optional[str], + ) -> None: + super().__init__(left) + self.left = cast(LogicalPlan, left) + self.right = right + self.on = on + if how is None: + join_type = proto.Join.JoinType.JOIN_TYPE_INNER + elif how == "inner": + join_type = proto.Join.JoinType.JOIN_TYPE_INNER + elif how in ["leftouter", "left"]: + join_type = proto.Join.JoinType.JOIN_TYPE_LEFT_OUTER + elif how == "cross": + join_type = proto.Join.JoinType.JOIN_TYPE_CROSS + else: + raise AnalysisException( + errorClass="UNSUPPORTED_JOIN_TYPE", + messageParameters={ + "typ": how, + "supported": ( + "'" + + "', '".join(["inner", "leftouter", "left", "left_outer", "cross"]) + + "'" + ), + }, + ) + self.how = join_type + + def plan(self, session: "SparkConnectClient") -> proto.Relation: + plan = self._create_proto_relation() + plan.lateral_join.left.CopyFrom(self.left.plan(session)) + plan.lateral_join.right.CopyFrom(self.right.plan(session)) + if self.on is not None: + plan.lateral_join.join_condition.CopyFrom(self.on.to_plan(session)) + plan.lateral_join.join_type = self.how + return plan + + @property + def observations(self) -> Dict[str, "Observation"]: + return dict(**super().observations, **self.right.observations) + + def print(self, indent: int = 0) -> str: + i = " " * indent + o = " " * (indent + LogicalPlan.INDENT) + n = indent + LogicalPlan.INDENT * 2 + return ( + f"{i}\n{o}" + f"left=\n{self.left.print(n)}\n{o}right=\n{self.right.print(n)}" + ) + + def _repr_html_(self) -> str: + return f""" +
    +
  • + LateralJoin
    + Left: {self.left._repr_html_()} + Right: {self.right._repr_html_()} +
  • +
+ """ + + class SetOperation(LogicalPlan): def __init__( self, diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 0d4730ac736e3..87070fd5ad3c5 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"\xc1\x30\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\xcc\x01\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_conditionB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\x8b\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\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\x63hildB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,77 +54,79 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_EXPRESSION"]._serialized_start = 133 - _globals["_EXPRESSION"]._serialized_end = 6342 - _globals["_EXPRESSION_WINDOW"]._serialized_start = 1900 - _globals["_EXPRESSION_WINDOW"]._serialized_end = 2683 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2190 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2683 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2457 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2602 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2604 - _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2683 - _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2686 - _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3111 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 2916 - _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3024 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3026 - _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3111 - _globals["_EXPRESSION_CAST"]._serialized_start = 3114 - _globals["_EXPRESSION_CAST"]._serialized_end = 3429 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3315 - _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3413 - _globals["_EXPRESSION_LITERAL"]._serialized_start = 3432 - _globals["_EXPRESSION_LITERAL"]._serialized_end = 4995 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4267 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4384 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4386 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4484 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4487 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4617 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4620 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 4847 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 4850 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 4979 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 4998 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5184 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5187 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5391 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5393 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5443 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5445 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5569 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5571 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5657 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5660 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5792 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5795 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 5982 - _globals["_EXPRESSION_ALIAS"]._serialized_start = 5984 - _globals["_EXPRESSION_ALIAS"]._serialized_end = 6104 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6107 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6265 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6267 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6329 - _globals["_EXPRESSIONCOMMON"]._serialized_start = 6344 - _globals["_EXPRESSIONCOMMON"]._serialized_end = 6409 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6412 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6776 - _globals["_PYTHONUDF"]._serialized_start = 6779 - _globals["_PYTHONUDF"]._serialized_end = 6983 - _globals["_SCALARSCALAUDF"]._serialized_start = 6986 - _globals["_SCALARSCALAUDF"]._serialized_end = 7200 - _globals["_JAVAUDF"]._serialized_start = 7203 - _globals["_JAVAUDF"]._serialized_end = 7352 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7354 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7453 - _globals["_CALLFUNCTION"]._serialized_start = 7455 - _globals["_CALLFUNCTION"]._serialized_end = 7563 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7565 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7657 - _globals["_MERGEACTION"]._serialized_start = 7660 - _globals["_MERGEACTION"]._serialized_end = 8172 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 7882 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 7988 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 7991 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8158 + _globals["_EXPRESSION"]._serialized_end = 6416 + _globals["_EXPRESSION_WINDOW"]._serialized_start = 1974 + _globals["_EXPRESSION_WINDOW"]._serialized_end = 2757 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2264 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_end = 2757 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_start = 2531 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMEBOUNDARY"]._serialized_end = 2676 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_start = 2678 + _globals["_EXPRESSION_WINDOW_WINDOWFRAME_FRAMETYPE"]._serialized_end = 2757 + _globals["_EXPRESSION_SORTORDER"]._serialized_start = 2760 + _globals["_EXPRESSION_SORTORDER"]._serialized_end = 3185 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_start = 2990 + _globals["_EXPRESSION_SORTORDER_SORTDIRECTION"]._serialized_end = 3098 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_start = 3100 + _globals["_EXPRESSION_SORTORDER_NULLORDERING"]._serialized_end = 3185 + _globals["_EXPRESSION_CAST"]._serialized_start = 3188 + _globals["_EXPRESSION_CAST"]._serialized_end = 3503 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3389 + _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3487 + _globals["_EXPRESSION_LITERAL"]._serialized_start = 3506 + _globals["_EXPRESSION_LITERAL"]._serialized_end = 5069 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4341 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4458 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4460 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4558 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4561 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4691 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4694 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 4921 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 4924 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5053 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5072 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5258 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5261 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5465 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5467 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5517 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5519 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5643 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5645 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5731 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5734 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 5866 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 5869 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6056 + _globals["_EXPRESSION_ALIAS"]._serialized_start = 6058 + _globals["_EXPRESSION_ALIAS"]._serialized_end = 6178 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6181 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6339 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6341 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6403 + _globals["_EXPRESSIONCOMMON"]._serialized_start = 6418 + _globals["_EXPRESSIONCOMMON"]._serialized_end = 6483 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6486 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6850 + _globals["_PYTHONUDF"]._serialized_start = 6853 + _globals["_PYTHONUDF"]._serialized_end = 7057 + _globals["_SCALARSCALAUDF"]._serialized_start = 7060 + _globals["_SCALARSCALAUDF"]._serialized_end = 7274 + _globals["_JAVAUDF"]._serialized_start = 7277 + _globals["_JAVAUDF"]._serialized_end = 7426 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7428 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7527 + _globals["_CALLFUNCTION"]._serialized_start = 7529 + _globals["_CALLFUNCTION"]._serialized_end = 7637 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7639 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7731 + _globals["_MERGEACTION"]._serialized_start = 7734 + _globals["_MERGEACTION"]._serialized_end = 8246 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 7956 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8062 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8065 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8232 + _globals["_LAZYEXPRESSION"]._serialized_start = 8248 + _globals["_LAZYEXPRESSION"]._serialized_end = 8313 # @@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 1566eb1b1e9e2..df4106cfc5f79 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -1184,6 +1184,7 @@ class Expression(google.protobuf.message.Message): NAMED_ARGUMENT_EXPRESSION_FIELD_NUMBER: builtins.int MERGE_ACTION_FIELD_NUMBER: builtins.int TYPED_AGGREGATE_EXPRESSION_FIELD_NUMBER: builtins.int + LAZY_EXPRESSION_FIELD_NUMBER: builtins.int EXTENSION_FIELD_NUMBER: builtins.int @property def common(self) -> global___ExpressionCommon: ... @@ -1228,6 +1229,8 @@ class Expression(google.protobuf.message.Message): @property def typed_aggregate_expression(self) -> global___TypedAggregateExpression: ... @property + def lazy_expression(self) -> global___LazyExpression: ... + @property def extension(self) -> google.protobuf.any_pb2.Any: """This field is used to mark extensions to the protocol. When plugins generate arbitrary relations they can add them here. During the planning the correct resolution is done. @@ -1256,6 +1259,7 @@ class Expression(google.protobuf.message.Message): named_argument_expression: global___NamedArgumentExpression | None = ..., merge_action: global___MergeAction | None = ..., typed_aggregate_expression: global___TypedAggregateExpression | None = ..., + lazy_expression: global___LazyExpression | None = ..., extension: google.protobuf.any_pb2.Any | None = ..., ) -> None: ... def HasField( @@ -1279,6 +1283,8 @@ class Expression(google.protobuf.message.Message): b"extension", "lambda_function", b"lambda_function", + "lazy_expression", + b"lazy_expression", "literal", b"literal", "merge_action", @@ -1328,6 +1334,8 @@ class Expression(google.protobuf.message.Message): b"extension", "lambda_function", b"lambda_function", + "lazy_expression", + b"lazy_expression", "literal", b"literal", "merge_action", @@ -1379,6 +1387,7 @@ class Expression(google.protobuf.message.Message): "named_argument_expression", "merge_action", "typed_aggregate_expression", + "lazy_expression", "extension", ] | None @@ -1801,3 +1810,22 @@ class MergeAction(google.protobuf.message.Message): ) -> typing_extensions.Literal["condition"] | None: ... global___MergeAction = MergeAction + +class LazyExpression(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + CHILD_FIELD_NUMBER: builtins.int + @property + def child(self) -> global___Expression: + """(Required) The expression to be marked as lazy.""" + def __init__( + self, + *, + child: global___Expression | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["child", b"child"] + ) -> builtins.bool: ... + def ClearField(self, field_name: typing_extensions.Literal["child", b"child"]) -> None: ... + +global___LazyExpression = LazyExpression diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 479abcfb597a1..506b266f60145 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"\x9c\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\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\tdirectionB6\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"\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' ) _globals = globals() @@ -78,157 +78,159 @@ _globals["_PARSE_OPTIONSENTRY"]._loaded_options = None _globals["_PARSE_OPTIONSENTRY"]._serialized_options = b"8\001" _globals["_RELATION"]._serialized_start = 193 - _globals["_RELATION"]._serialized_end = 3805 - _globals["_UNKNOWN"]._serialized_start = 3807 - _globals["_UNKNOWN"]._serialized_end = 3816 - _globals["_RELATIONCOMMON"]._serialized_start = 3819 - _globals["_RELATIONCOMMON"]._serialized_end = 3961 - _globals["_SQL"]._serialized_start = 3964 - _globals["_SQL"]._serialized_end = 4442 - _globals["_SQL_ARGSENTRY"]._serialized_start = 4258 - _globals["_SQL_ARGSENTRY"]._serialized_end = 4348 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4350 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4442 - _globals["_WITHRELATIONS"]._serialized_start = 4444 - _globals["_WITHRELATIONS"]._serialized_end = 4561 - _globals["_READ"]._serialized_start = 4564 - _globals["_READ"]._serialized_end = 5227 - _globals["_READ_NAMEDTABLE"]._serialized_start = 4742 - _globals["_READ_NAMEDTABLE"]._serialized_end = 4934 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_READ_DATASOURCE"]._serialized_start = 4937 - _globals["_READ_DATASOURCE"]._serialized_end = 5214 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_PROJECT"]._serialized_start = 5229 - _globals["_PROJECT"]._serialized_end = 5346 - _globals["_FILTER"]._serialized_start = 5348 - _globals["_FILTER"]._serialized_end = 5460 - _globals["_JOIN"]._serialized_start = 5463 - _globals["_JOIN"]._serialized_end = 6124 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5802 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5894 - _globals["_JOIN_JOINTYPE"]._serialized_start = 5897 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6105 - _globals["_SETOPERATION"]._serialized_start = 6127 - _globals["_SETOPERATION"]._serialized_end = 6606 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6443 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6557 - _globals["_LIMIT"]._serialized_start = 6608 - _globals["_LIMIT"]._serialized_end = 6684 - _globals["_OFFSET"]._serialized_start = 6686 - _globals["_OFFSET"]._serialized_end = 6765 - _globals["_TAIL"]._serialized_start = 6767 - _globals["_TAIL"]._serialized_end = 6842 - _globals["_AGGREGATE"]._serialized_start = 6845 - _globals["_AGGREGATE"]._serialized_end = 7611 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 7260 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 7371 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7373 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7449 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7452 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7611 - _globals["_SORT"]._serialized_start = 7614 - _globals["_SORT"]._serialized_end = 7774 - _globals["_DROP"]._serialized_start = 7777 - _globals["_DROP"]._serialized_end = 7918 - _globals["_DEDUPLICATE"]._serialized_start = 7921 - _globals["_DEDUPLICATE"]._serialized_end = 8161 - _globals["_LOCALRELATION"]._serialized_start = 8163 - _globals["_LOCALRELATION"]._serialized_end = 8252 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 8254 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 8326 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 8328 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 8383 - _globals["_SAMPLE"]._serialized_start = 8386 - _globals["_SAMPLE"]._serialized_end = 8659 - _globals["_RANGE"]._serialized_start = 8662 - _globals["_RANGE"]._serialized_end = 8807 - _globals["_SUBQUERYALIAS"]._serialized_start = 8809 - _globals["_SUBQUERYALIAS"]._serialized_end = 8923 - _globals["_REPARTITION"]._serialized_start = 8926 - _globals["_REPARTITION"]._serialized_end = 9068 - _globals["_SHOWSTRING"]._serialized_start = 9071 - _globals["_SHOWSTRING"]._serialized_end = 9213 - _globals["_HTMLSTRING"]._serialized_start = 9215 - _globals["_HTMLSTRING"]._serialized_end = 9329 - _globals["_STATSUMMARY"]._serialized_start = 9331 - _globals["_STATSUMMARY"]._serialized_end = 9423 - _globals["_STATDESCRIBE"]._serialized_start = 9425 - _globals["_STATDESCRIBE"]._serialized_end = 9506 - _globals["_STATCROSSTAB"]._serialized_start = 9508 - _globals["_STATCROSSTAB"]._serialized_end = 9609 - _globals["_STATCOV"]._serialized_start = 9611 - _globals["_STATCOV"]._serialized_end = 9707 - _globals["_STATCORR"]._serialized_start = 9710 - _globals["_STATCORR"]._serialized_end = 9847 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 9850 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10014 - _globals["_STATFREQITEMS"]._serialized_start = 10016 - _globals["_STATFREQITEMS"]._serialized_end = 10141 - _globals["_STATSAMPLEBY"]._serialized_start = 10144 - _globals["_STATSAMPLEBY"]._serialized_end = 10453 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10345 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10444 - _globals["_NAFILL"]._serialized_start = 10456 - _globals["_NAFILL"]._serialized_end = 10590 - _globals["_NADROP"]._serialized_start = 10593 - _globals["_NADROP"]._serialized_end = 10727 - _globals["_NAREPLACE"]._serialized_start = 10730 - _globals["_NAREPLACE"]._serialized_end = 11026 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10885 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11026 - _globals["_TODF"]._serialized_start = 11028 - _globals["_TODF"]._serialized_end = 11116 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11119 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11501 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11363 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11430 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11432 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11501 - _globals["_WITHCOLUMNS"]._serialized_start = 11503 - _globals["_WITHCOLUMNS"]._serialized_end = 11622 - _globals["_WITHWATERMARK"]._serialized_start = 11625 - _globals["_WITHWATERMARK"]._serialized_end = 11759 - _globals["_HINT"]._serialized_start = 11762 - _globals["_HINT"]._serialized_end = 11894 - _globals["_UNPIVOT"]._serialized_start = 11897 - _globals["_UNPIVOT"]._serialized_end = 12224 - _globals["_UNPIVOT_VALUES"]._serialized_start = 12154 - _globals["_UNPIVOT_VALUES"]._serialized_end = 12213 - _globals["_TRANSPOSE"]._serialized_start = 12226 - _globals["_TRANSPOSE"]._serialized_end = 12348 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12350 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12475 - _globals["_TOSCHEMA"]._serialized_start = 12477 - _globals["_TOSCHEMA"]._serialized_end = 12583 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12586 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12789 - _globals["_MAPPARTITIONS"]._serialized_start = 12792 - _globals["_MAPPARTITIONS"]._serialized_end = 13024 - _globals["_GROUPMAP"]._serialized_start = 13027 - _globals["_GROUPMAP"]._serialized_end = 13662 - _globals["_COGROUPMAP"]._serialized_start = 13665 - _globals["_COGROUPMAP"]._serialized_end = 14191 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 14194 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 14551 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 14554 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 14798 - _globals["_PYTHONUDTF"]._serialized_start = 14801 - _globals["_PYTHONUDTF"]._serialized_end = 14978 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 14981 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 15132 - _globals["_PYTHONDATASOURCE"]._serialized_start = 15134 - _globals["_PYTHONDATASOURCE"]._serialized_end = 15209 - _globals["_COLLECTMETRICS"]._serialized_start = 15212 - _globals["_COLLECTMETRICS"]._serialized_end = 15348 - _globals["_PARSE"]._serialized_start = 15351 - _globals["_PARSE"]._serialized_end = 15739 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 4876 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 4934 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 15640 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 15728 - _globals["_ASOFJOIN"]._serialized_start = 15742 - _globals["_ASOFJOIN"]._serialized_end = 16217 + _globals["_RELATION"]._serialized_end = 3870 + _globals["_UNKNOWN"]._serialized_start = 3872 + _globals["_UNKNOWN"]._serialized_end = 3881 + _globals["_RELATIONCOMMON"]._serialized_start = 3884 + _globals["_RELATIONCOMMON"]._serialized_end = 4026 + _globals["_SQL"]._serialized_start = 4029 + _globals["_SQL"]._serialized_end = 4507 + _globals["_SQL_ARGSENTRY"]._serialized_start = 4323 + _globals["_SQL_ARGSENTRY"]._serialized_end = 4413 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 4415 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 4507 + _globals["_WITHRELATIONS"]._serialized_start = 4509 + _globals["_WITHRELATIONS"]._serialized_end = 4626 + _globals["_READ"]._serialized_start = 4629 + _globals["_READ"]._serialized_end = 5292 + _globals["_READ_NAMEDTABLE"]._serialized_start = 4807 + _globals["_READ_NAMEDTABLE"]._serialized_end = 4999 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 4941 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 4999 + _globals["_READ_DATASOURCE"]._serialized_start = 5002 + _globals["_READ_DATASOURCE"]._serialized_end = 5279 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 4941 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 4999 + _globals["_PROJECT"]._serialized_start = 5294 + _globals["_PROJECT"]._serialized_end = 5411 + _globals["_FILTER"]._serialized_start = 5413 + _globals["_FILTER"]._serialized_end = 5525 + _globals["_JOIN"]._serialized_start = 5528 + _globals["_JOIN"]._serialized_end = 6189 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 5867 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 5959 + _globals["_JOIN_JOINTYPE"]._serialized_start = 5962 + _globals["_JOIN_JOINTYPE"]._serialized_end = 6170 + _globals["_SETOPERATION"]._serialized_start = 6192 + _globals["_SETOPERATION"]._serialized_end = 6671 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 6508 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 6622 + _globals["_LIMIT"]._serialized_start = 6673 + _globals["_LIMIT"]._serialized_end = 6749 + _globals["_OFFSET"]._serialized_start = 6751 + _globals["_OFFSET"]._serialized_end = 6830 + _globals["_TAIL"]._serialized_start = 6832 + _globals["_TAIL"]._serialized_end = 6907 + _globals["_AGGREGATE"]._serialized_start = 6910 + _globals["_AGGREGATE"]._serialized_end = 7676 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 7325 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 7436 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 7438 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 7514 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 7517 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 7676 + _globals["_SORT"]._serialized_start = 7679 + _globals["_SORT"]._serialized_end = 7839 + _globals["_DROP"]._serialized_start = 7842 + _globals["_DROP"]._serialized_end = 7983 + _globals["_DEDUPLICATE"]._serialized_start = 7986 + _globals["_DEDUPLICATE"]._serialized_end = 8226 + _globals["_LOCALRELATION"]._serialized_start = 8228 + _globals["_LOCALRELATION"]._serialized_end = 8317 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 8319 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 8391 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 8393 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 8448 + _globals["_SAMPLE"]._serialized_start = 8451 + _globals["_SAMPLE"]._serialized_end = 8724 + _globals["_RANGE"]._serialized_start = 8727 + _globals["_RANGE"]._serialized_end = 8872 + _globals["_SUBQUERYALIAS"]._serialized_start = 8874 + _globals["_SUBQUERYALIAS"]._serialized_end = 8988 + _globals["_REPARTITION"]._serialized_start = 8991 + _globals["_REPARTITION"]._serialized_end = 9133 + _globals["_SHOWSTRING"]._serialized_start = 9136 + _globals["_SHOWSTRING"]._serialized_end = 9278 + _globals["_HTMLSTRING"]._serialized_start = 9280 + _globals["_HTMLSTRING"]._serialized_end = 9394 + _globals["_STATSUMMARY"]._serialized_start = 9396 + _globals["_STATSUMMARY"]._serialized_end = 9488 + _globals["_STATDESCRIBE"]._serialized_start = 9490 + _globals["_STATDESCRIBE"]._serialized_end = 9571 + _globals["_STATCROSSTAB"]._serialized_start = 9573 + _globals["_STATCROSSTAB"]._serialized_end = 9674 + _globals["_STATCOV"]._serialized_start = 9676 + _globals["_STATCOV"]._serialized_end = 9772 + _globals["_STATCORR"]._serialized_start = 9775 + _globals["_STATCORR"]._serialized_end = 9912 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 9915 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10079 + _globals["_STATFREQITEMS"]._serialized_start = 10081 + _globals["_STATFREQITEMS"]._serialized_end = 10206 + _globals["_STATSAMPLEBY"]._serialized_start = 10209 + _globals["_STATSAMPLEBY"]._serialized_end = 10518 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 10410 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 10509 + _globals["_NAFILL"]._serialized_start = 10521 + _globals["_NAFILL"]._serialized_end = 10655 + _globals["_NADROP"]._serialized_start = 10658 + _globals["_NADROP"]._serialized_end = 10792 + _globals["_NAREPLACE"]._serialized_start = 10795 + _globals["_NAREPLACE"]._serialized_end = 11091 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 10950 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11091 + _globals["_TODF"]._serialized_start = 11093 + _globals["_TODF"]._serialized_end = 11181 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11184 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 11566 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 11428 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 11495 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 11497 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 11566 + _globals["_WITHCOLUMNS"]._serialized_start = 11568 + _globals["_WITHCOLUMNS"]._serialized_end = 11687 + _globals["_WITHWATERMARK"]._serialized_start = 11690 + _globals["_WITHWATERMARK"]._serialized_end = 11824 + _globals["_HINT"]._serialized_start = 11827 + _globals["_HINT"]._serialized_end = 11959 + _globals["_UNPIVOT"]._serialized_start = 11962 + _globals["_UNPIVOT"]._serialized_end = 12289 + _globals["_UNPIVOT_VALUES"]._serialized_start = 12219 + _globals["_UNPIVOT_VALUES"]._serialized_end = 12278 + _globals["_TRANSPOSE"]._serialized_start = 12291 + _globals["_TRANSPOSE"]._serialized_end = 12413 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 12415 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 12540 + _globals["_TOSCHEMA"]._serialized_start = 12542 + _globals["_TOSCHEMA"]._serialized_end = 12648 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 12651 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 12854 + _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["_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 # @@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 03753056c6bf1..e5a6bff9e430a 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -106,6 +106,7 @@ class Relation(google.protobuf.message.Message): WITH_RELATIONS_FIELD_NUMBER: builtins.int TRANSPOSE_FIELD_NUMBER: builtins.int UNRESOLVED_TABLE_VALUED_FUNCTION_FIELD_NUMBER: builtins.int + LATERAL_JOIN_FIELD_NUMBER: builtins.int FILL_NA_FIELD_NUMBER: builtins.int DROP_NA_FIELD_NUMBER: builtins.int REPLACE_FIELD_NUMBER: builtins.int @@ -211,6 +212,8 @@ class Relation(google.protobuf.message.Message): @property def unresolved_table_valued_function(self) -> global___UnresolvedTableValuedFunction: ... @property + def lateral_join(self) -> global___LateralJoin: ... + @property def fill_na(self) -> global___NAFill: """NA functions""" @property @@ -292,6 +295,7 @@ class Relation(google.protobuf.message.Message): with_relations: global___WithRelations | None = ..., transpose: global___Transpose | None = ..., unresolved_table_valued_function: global___UnresolvedTableValuedFunction | None = ..., + lateral_join: global___LateralJoin | None = ..., fill_na: global___NAFill | None = ..., drop_na: global___NADrop | None = ..., replace: global___NAReplace | None = ..., @@ -364,6 +368,8 @@ class Relation(google.protobuf.message.Message): b"html_string", "join", b"join", + "lateral_join", + b"lateral_join", "limit", b"limit", "local_relation", @@ -485,6 +491,8 @@ class Relation(google.protobuf.message.Message): b"html_string", "join", b"join", + "lateral_join", + b"lateral_join", "limit", b"limit", "local_relation", @@ -595,6 +603,7 @@ class Relation(google.protobuf.message.Message): "with_relations", "transpose", "unresolved_table_valued_function", + "lateral_join", "fill_na", "drop_na", "replace", @@ -4109,3 +4118,56 @@ class AsOfJoin(google.protobuf.message.Message): ) -> None: ... global___AsOfJoin = AsOfJoin + +class LateralJoin(google.protobuf.message.Message): + """Relation of type [[LateralJoin]]. + + `left` and `right` must be present. + """ + + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + LEFT_FIELD_NUMBER: builtins.int + RIGHT_FIELD_NUMBER: builtins.int + JOIN_CONDITION_FIELD_NUMBER: builtins.int + JOIN_TYPE_FIELD_NUMBER: builtins.int + @property + def left(self) -> global___Relation: + """(Required) Left input relation for a Join.""" + @property + def right(self) -> global___Relation: + """(Required) Right input relation for a Join.""" + @property + def join_condition(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression: + """(Optional) The join condition.""" + join_type: global___Join.JoinType.ValueType + """(Required) The join type.""" + def __init__( + self, + *, + left: global___Relation | None = ..., + right: global___Relation | None = ..., + join_condition: pyspark.sql.connect.proto.expressions_pb2.Expression | None = ..., + join_type: global___Join.JoinType.ValueType = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "join_condition", b"join_condition", "left", b"left", "right", b"right" + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "join_condition", + b"join_condition", + "join_type", + b"join_type", + "left", + b"left", + "right", + b"right", + ], + ) -> None: ... + +global___LateralJoin = LateralJoin diff --git a/python/pyspark/sql/tests/connect/test_parity_subquery.py b/python/pyspark/sql/tests/connect/test_parity_subquery.py index 1cba3a7d49956..cffb6fc39059d 100644 --- a/python/pyspark/sql/tests/connect/test_parity_subquery.py +++ b/python/pyspark/sql/tests/connect/test_parity_subquery.py @@ -21,9 +21,38 @@ from pyspark.testing.connectutils import ReusedConnectTestCase -@unittest.skip("TODO(SPARK-50134): Support subquery in connect") class SubqueryParityTests(SubqueryTestsMixin, ReusedConnectTestCase): - pass + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_simple_uncorrelated_scalar_subquery(self): + super().test_simple_uncorrelated_scalar_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_uncorrelated_scalar_subquery_with_view(self): + super().test_uncorrelated_scalar_subquery_with_view() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_against_local_relations(self): + super().test_scalar_subquery_against_local_relations() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_correlated_scalar_subquery(self): + super().test_correlated_scalar_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_exists_subquery(self): + super().test_exists_subquery() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_with_outer_reference_errors(self): + super().test_scalar_subquery_with_outer_reference_errors() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_scalar_subquery_inside_lateral_join(self): + super().test_scalar_subquery_inside_lateral_join() + + @unittest.skip("TODO(SPARK-50134): Support subquery in connect") + def test_lateral_join_inside_subquery(self): + super().test_lateral_join_inside_subquery() if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/connect/test_parity_tvf.py b/python/pyspark/sql/tests/connect/test_parity_tvf.py index c5edff02810ff..61e3decf562c3 100644 --- a/python/pyspark/sql/tests/connect/test_parity_tvf.py +++ b/python/pyspark/sql/tests/connect/test_parity_tvf.py @@ -21,45 +21,7 @@ class TVFParityTestsMixin(TVFTestsMixin, ReusedConnectTestCase): - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_explode_with_lateral_join(self): - super().test_explode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_explode_outer_with_lateral_join(self): - super().test_explode_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_inline_with_lateral_join(self): - super().test_inline_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_inline_outer_with_lateral_join(self): - super().test_inline_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_json_tuple_with_lateral_join(self): - super().test_json_tuple_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_posexplode_with_lateral_join(self): - super().test_posexplode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_posexplode_outer_with_lateral_join(self): - super().test_posexplode_outer_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_stack_with_lateral_join(self): - super().test_stack_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_variant_explode_with_lateral_join(self): - super().test_variant_explode_with_lateral_join() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_variant_explode_outer_with_lateral_join(self): - super().test_variant_explode_outer_with_lateral_join() + pass if __name__ == "__main__": diff --git a/python/pyspark/sql/tests/connect/test_parity_udtf.py b/python/pyspark/sql/tests/connect/test_parity_udtf.py index 29d1718fe3781..6955e7377b4c4 100644 --- a/python/pyspark/sql/tests/connect/test_parity_udtf.py +++ b/python/pyspark/sql/tests/connect/test_parity_udtf.py @@ -85,14 +85,6 @@ def _add_archive(self, path): def _add_file(self, path): self.spark.addArtifacts(path, file=True) - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_udtf_with_lateral_join_dataframe(self): - super().test_udtf_with_lateral_join_dataframe() - - @unittest.skip("SPARK-50134: Support Spark Connect") - def test_udtf_with_conditional_return_dataframe(self): - super().test_udtf_with_conditional_return_dataframe() - class ArrowUDTFParityTests(UDTFArrowTestsMixin, UDTFParityTests): @classmethod diff --git a/python/pyspark/sql/tests/test_subquery.py b/python/pyspark/sql/tests/test_subquery.py index 1b657e075c59e..91789f74d9da6 100644 --- a/python/pyspark/sql/tests/test_subquery.py +++ b/python/pyspark/sql/tests/test_subquery.py @@ -556,6 +556,23 @@ def test_lateral_join_with_different_join_types(self): self.spark.sql("""SELECT * FROM t1 CROSS JOIN LATERAL (SELECT c1 + c2 AS c3)"""), ) + with self.assertRaises(AnalysisException) as pe: + t1.lateralJoin( + self.spark.range(1).select( + (sf.col("c1").outer() + sf.col("c2").outer()).alias("c3") + ), + how="right", + ).collect() + + self.check_error( + pe.exception, + errorClass="UNSUPPORTED_JOIN_TYPE", + messageParameters={ + "typ": "right", + "supported": "'inner', 'leftouter', 'left', 'left_outer', 'cross'", + }, + ) + def test_lateral_join_with_correlated_predicates(self): with self.tempView("t1", "t2"): t1 = self.table1() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 41bba99673a2b..9f8c62fe58408 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -159,3 +159,25 @@ sealed abstract class AsOfJoinDirection case object Forward extends AsOfJoinDirection case object Backward extends AsOfJoinDirection case object Nearest extends AsOfJoinDirection + +object LateralJoinType { + + val supported = Seq( + "inner", + "leftouter", "left", "left_outer", + "cross" + ) + + def apply(typ: String): JoinType = typ.toLowerCase(Locale.ROOT).replace("_", "") match { + case "inner" => Inner + case "leftouter" | "left" => LeftOuter + case "cross" => Cross + case _ => + throw new AnalysisException( + errorClass = "UNSUPPORTED_JOIN_TYPE", + messageParameters = Map( + "typ" -> typ, + "supported" -> supported.mkString("'", "', '", "'")) + ) + } +} 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 3a91371fd3b25..811dd032aa419 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -52,6 +52,7 @@ message Expression { NamedArgumentExpression named_argument_expression = 17; MergeAction merge_action = 19; TypedAggregateExpression typed_aggregate_expression = 20; + LazyExpression lazy_expression = 21; // This field is used to mark extensions to the protocol. When plugins generate arbitrary // relations they can add them here. During the planning the correct resolution is done. @@ -451,3 +452,8 @@ message MergeAction { Expression value = 2; } } + +message LazyExpression { + // (Required) The expression to be marked as lazy. + Expression child = 1; +} 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 a7b9137c3400a..7a86db2799149 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -78,6 +78,7 @@ message Relation { WithRelations with_relations = 41; Transpose transpose = 42; UnresolvedTableValuedFunction unresolved_table_valued_function = 43; + LateralJoin lateral_join = 44; // NA functions NAFill fill_na = 90; @@ -1140,3 +1141,20 @@ message AsOfJoin { // (Required) Whether to search for prior, subsequent, or closest matches. string direction = 10; } + +// Relation of type [[LateralJoin]]. +// +// `left` and `right` must be present. +message LateralJoin { + // (Required) Left input relation for a Join. + Relation left = 1; + + // (Required) Right input relation for a Join. + Relation right = 2; + + // (Optional) The join condition. + Expression join_condition = 3; + + // (Required) The join type. + Join.JoinType join_type = 4; +} 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 4aa567f19e16c..ec67c57a37f57 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, LocalTempView, MultiAlias, NameParameterizedQuery, PosParameterizedQuery, UnresolvedAlias, UnresolvedAttribute, UnresolvedDataFrameStar, UnresolvedDeserializer, UnresolvedExtractValue, UnresolvedFunction, 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, UnresolvedRegex, UnresolvedRelation, UnresolvedStar, 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._ @@ -154,6 +154,7 @@ class SparkConnectPlanner( case proto.Relation.RelTypeCase.TAIL => transformTail(rel.getTail) case proto.Relation.RelTypeCase.JOIN => transformJoinOrJoinWith(rel.getJoin) case proto.Relation.RelTypeCase.AS_OF_JOIN => transformAsOfJoin(rel.getAsOfJoin) + case proto.Relation.RelTypeCase.LATERAL_JOIN => transformLateralJoin(rel.getLateralJoin) case proto.Relation.RelTypeCase.DEDUPLICATE => transformDeduplicate(rel.getDeduplicate) case proto.Relation.RelTypeCase.SET_OP => transformSetOperation(rel.getSetOp) case proto.Relation.RelTypeCase.SORT => transformSort(rel.getSort) @@ -1556,6 +1557,8 @@ class SparkConnectPlanner( transformMergeAction(exp.getMergeAction) case proto.Expression.ExprTypeCase.TYPED_AGGREGATE_EXPRESSION => transformTypedAggregateExpression(exp.getTypedAggregateExpression, baseRelationOpt) + case proto.Expression.ExprTypeCase.LAZY_EXPRESSION => + transformLazyExpression(exp.getLazyExpression) case _ => throw InvalidPlanInput( s"Expression with ID: ${exp.getExprTypeCase.getNumber} is not supported") @@ -2140,6 +2143,19 @@ class SparkConnectPlanner( joined.logicalPlan } + private def transformLateralJoin(rel: proto.LateralJoin): LogicalPlan = { + assert(rel.hasLeft && rel.hasRight, "Both join sides must be present") + val joinCondition = + if (rel.hasJoinCondition) Some(transformExpression(rel.getJoinCondition)) else None + val joinType = transformJoinType( + if (rel.getJoinType != null) rel.getJoinType else proto.Join.JoinType.JOIN_TYPE_INNER) + logical.LateralJoin( + left = transformRelation(rel.getLeft), + right = LateralSubquery(transformRelation(rel.getRight)), + joinType = joinType, + condition = joinCondition) + } + private def transformSort(sort: proto.Sort): LogicalPlan = { assert(sort.getOrderCount > 0, "'order' must be present and contain elements.") logical.Sort( @@ -3704,4 +3720,8 @@ class SparkConnectPlanner( session.catalog.listCatalogs().logicalPlan } } + + private def transformLazyExpression(getLazyExpression: proto.LazyExpression): Expression = { + LazyExpression(transformExpression(getLazyExpression.getChild)) + } } 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 8726ee268a477..c5c9dde901c9c 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 @@ -733,12 +733,12 @@ class Dataset[T] private[sql]( /** @inheritdoc */ def lateralJoin(right: DS[_], joinType: String): DataFrame = { - lateralJoin(right, None, JoinType(joinType)) + lateralJoin(right, None, LateralJoinType(joinType)) } /** @inheritdoc */ def lateralJoin(right: DS[_], joinExprs: Column, joinType: String): DataFrame = { - lateralJoin(right, Some(joinExprs), JoinType(joinType)) + lateralJoin(right, Some(joinExprs), LateralJoinType(joinType)) } // TODO(SPARK-22947): Fix the DataFrame API.