From 2721a50234b05db3e59b5a37225f9358978a4590 Mon Sep 17 00:00:00 2001 From: Ruifeng Zheng Date: Fri, 17 Jan 2025 17:54:12 +0800 Subject: [PATCH] [SPARK-50851][ML][CONNECT][PYTHON] Express ML params with `proto.Expression.Literal` MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ### What changes were proposed in this pull request? Express ML params with `proto.Expression.Literal`: 1, introduce `Literal.SpecializedArray` for large primitive literal arrays (e.g. the initial model coefficients which can be large) ``` message SpecializedArray { oneof value_type { Bools bools = 1; Ints ints = 2; Longs longs = 3; Floats floats = 4; Doubles doubles = 5; Strings strings = 6; } message Bools { repeated bool values = 1; } message Ints { repeated int32 values = 1; } message Longs { repeated int64 values = 1; } message Floats { repeated float values = 1; } message Doubles { repeated double values = 1; } message Strings { repeated string values = 1; } } ``` 2, Replace `proto.Param ` with `proto.Expression` to be consistent with SQL side For `Param[Vector]` and `Param[Matrix]`, apply `proto.Expression.Literal.Struct` with the underlying schema of `VectorUDT` and `MatrixUDT`. E.g. for `Param[Vector]` with value `Vectors.sparse(4, [(1, 1.0), (3, 5.5)])`, the message is like: ``` literal { struct { struct_type { struct { ... <- schema of VectorUDT } } elements { byte: 0 } elements { integer: 4 } elements { specialized_array { ints { values: 1 values: 3 } } } elements { specialized_array { doubles { values: 1 values: 5.5 } } } } ``` ### Why are the changes needed? 1, to optimize large literal arrays, for both ML and SQL (we can apply it in SQL side later) 2, be consistent with SQL side, e.g. the parameterized SQL ``` // (Optional) A map of parameter names to expressions. // It cannot coexist with `pos_arguments`. map named_arguments = 4; // (Optional) A sequence of expressions for positional parameters in the SQL query text. // It cannot coexist with `named_arguments`. repeated Expression pos_arguments = 5; ``` 3, to minimize the protobuf change ### Does this PR introduce _any_ user-facing change? no, refactor-only ### How was this patch tested? existing tests ### Was this patch authored or co-authored using generative AI tooling? no Closes #49529 from zhengruifeng/ml_proto_expr. Authored-by: Ruifeng Zheng Signed-off-by: Ruifeng Zheng --- .../apache/spark/ml/linalg/MatrixUDT.scala | 39 ++- .../apache/spark/ml/linalg/VectorUDT.scala | 7 +- python/pyspark/ml/connect/serialize.py | 200 +++++++---- .../pyspark/sql/connect/proto/common_pb2.py | 14 +- .../pyspark/sql/connect/proto/common_pb2.pyi | 102 ++++++ .../sql/connect/proto/expressions_pb2.py | 124 +++---- .../sql/connect/proto/expressions_pb2.pyi | 85 +++++ .../sql/connect/proto/ml_common_pb2.py | 32 +- .../sql/connect/proto/ml_common_pb2.pyi | 278 +-------------- python/pyspark/sql/connect/proto/ml_pb2.py | 35 +- python/pyspark/sql/connect/proto/ml_pb2.pyi | 5 +- .../sql/connect/proto/relations_pb2.py | 316 +++++++++--------- .../sql/connect/proto/relations_pb2.pyi | 4 +- .../main/protobuf/spark/connect/common.proto | 24 ++ .../protobuf/spark/connect/expressions.proto | 13 + .../src/main/protobuf/spark/connect/ml.proto | 3 +- .../protobuf/spark/connect/ml_common.proto | 54 +-- .../protobuf/spark/connect/relations.proto | 2 +- .../spark/sql/connect/ml/MLHandler.scala | 6 +- .../apache/spark/sql/connect/ml/MLUtils.scala | 107 ++++-- .../spark/sql/connect/ml/Serializer.scala | 200 ++++++----- .../spark/sql/connect/ml/MLBackendSuite.scala | 15 +- .../apache/spark/sql/connect/ml/MLSuite.scala | 112 +++---- 23 files changed, 906 insertions(+), 871 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala index 27a67d561e357..89ddac889f944 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/MatrixUDT.scala @@ -27,23 +27,7 @@ import org.apache.spark.sql.types._ */ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { - override def sqlType: StructType = { - // type: 0 = sparse, 1 = dense - // the dense matrix is built by numRows, numCols, values and isTransposed, all of which are - // set as not nullable, except values since in the future, support for binary matrices might - // be added for which values are not needed. - // the sparse matrix needs colPtrs and rowIndices, which are set as - // null, while building the dense matrix. - StructType(Array( - StructField("type", ByteType, nullable = false), - StructField("numRows", IntegerType, nullable = false), - StructField("numCols", IntegerType, nullable = false), - StructField("colPtrs", ArrayType(IntegerType, containsNull = false), nullable = true), - StructField("rowIndices", ArrayType(IntegerType, containsNull = false), nullable = true), - StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true), - StructField("isTransposed", BooleanType, nullable = false) - )) - } + override def sqlType: StructType = MatrixUDT.sqlType override def serialize(obj: Matrix): InternalRow = { val row = new GenericInternalRow(7) @@ -108,3 +92,24 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { private[spark] override def asNullable: MatrixUDT = this } + +private[spark] object MatrixUDT { + + val sqlType: StructType = { + // type: 0 = sparse, 1 = dense + // the dense matrix is built by numRows, numCols, values and isTransposed, all of which are + // set as not nullable, except values since in the future, support for binary matrices might + // be added for which values are not needed. + // the sparse matrix needs colPtrs and rowIndices, which are set as + // null, while building the dense matrix. + StructType(Array( + StructField("type", ByteType, nullable = false), + StructField("numRows", IntegerType, nullable = false), + StructField("numCols", IntegerType, nullable = false), + StructField("colPtrs", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("rowIndices", ArrayType(IntegerType, containsNull = false), nullable = true), + StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true), + StructField("isTransposed", BooleanType, nullable = false) + )) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala index 302a94c87811e..d4648c1f0a1bb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/VectorUDT.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.types._ */ private[spark] class VectorUDT extends UserDefinedType[Vector] { - override final def sqlType: StructType = _sqlType + override final def sqlType: StructType = VectorUDT.sqlType override def serialize(obj: Vector): InternalRow = { obj match { @@ -86,8 +86,11 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { override def typeName: String = "vector" private[spark] override def asNullable: VectorUDT = this +} + +private[spark] object VectorUDT { - private[this] val _sqlType = { + val sqlType = { // type: 0 = sparse, 1 = dense // We only use "values" for dense vectors, and "size", "indices", and "values" for sparse // vectors. The "values" field is nullable because we might want to add binary vectors later, diff --git a/python/pyspark/ml/connect/serialize.py b/python/pyspark/ml/connect/serialize.py index 69e3af1f4c787..62b21460feb7c 100644 --- a/python/pyspark/ml/connect/serialize.py +++ b/python/pyspark/ml/connect/serialize.py @@ -18,54 +18,107 @@ import pyspark.sql.connect.proto as pb2 from pyspark.ml.linalg import ( - Vectors, - Matrices, + VectorUDT, + MatrixUDT, DenseVector, SparseVector, DenseMatrix, SparseMatrix, ) -from pyspark.sql.connect.expressions import LiteralExpression if TYPE_CHECKING: from pyspark.sql.connect.client import SparkConnectClient from pyspark.ml.param import Params -def serialize_param(value: Any, client: "SparkConnectClient") -> pb2.Param: - if isinstance(value, DenseVector): - return pb2.Param(vector=pb2.Vector(dense=pb2.Vector.Dense(value=value.values.tolist()))) - elif isinstance(value, SparseVector): - return pb2.Param( - vector=pb2.Vector( - sparse=pb2.Vector.Sparse( - size=value.size, index=value.indices.tolist(), value=value.values.tolist() - ) - ) - ) - elif isinstance(value, DenseMatrix): - return pb2.Param( - matrix=pb2.Matrix( - dense=pb2.Matrix.Dense( - num_rows=value.numRows, num_cols=value.numCols, value=value.values.tolist() - ) - ) - ) +def literal_null() -> pb2.Expression.Literal: + dt = pb2.DataType() + dt.null.CopyFrom(pb2.DataType.NULL()) + return pb2.Expression.Literal(null=dt) + + +def build_int_list(value: List[int]) -> pb2.Expression.Literal: + p = pb2.Expression.Literal() + p.specialized_array.ints.values.extend(value) + return p + + +def build_float_list(value: List[float]) -> pb2.Expression.Literal: + p = pb2.Expression.Literal() + p.specialized_array.doubles.values.extend(value) + return p + + +def serialize_param(value: Any, client: "SparkConnectClient") -> pb2.Expression.Literal: + from pyspark.sql.connect.types import pyspark_types_to_proto_types + from pyspark.sql.connect.expressions import LiteralExpression + + if isinstance(value, SparseVector): + p = pb2.Expression.Literal() + p.struct.struct_type.CopyFrom(pyspark_types_to_proto_types(VectorUDT.sqlType())) + # type = 0 + p.struct.elements.append(pb2.Expression.Literal(byte=0)) + # size + p.struct.elements.append(pb2.Expression.Literal(integer=value.size)) + # indices + p.struct.elements.append(build_int_list(value.indices.tolist())) + # values + p.struct.elements.append(build_float_list(value.values.tolist())) + return p + + elif isinstance(value, DenseVector): + p = pb2.Expression.Literal() + p.struct.struct_type.CopyFrom(pyspark_types_to_proto_types(VectorUDT.sqlType())) + # type = 1 + p.struct.elements.append(pb2.Expression.Literal(byte=1)) + # size = null + p.struct.elements.append(literal_null()) + # indices = null + p.struct.elements.append(literal_null()) + # values + p.struct.elements.append(build_float_list(value.values.tolist())) + return p + elif isinstance(value, SparseMatrix): - return pb2.Param( - matrix=pb2.Matrix( - sparse=pb2.Matrix.Sparse( - num_rows=value.numRows, - num_cols=value.numCols, - colptr=value.colPtrs.tolist(), - row_index=value.rowIndices.tolist(), - value=value.values.tolist(), - ) - ) - ) + p = pb2.Expression.Literal() + p.struct.struct_type.CopyFrom(pyspark_types_to_proto_types(MatrixUDT.sqlType())) + # type = 0 + p.struct.elements.append(pb2.Expression.Literal(byte=0)) + # numRows + p.struct.elements.append(pb2.Expression.Literal(integer=value.numRows)) + # numCols + p.struct.elements.append(pb2.Expression.Literal(integer=value.numCols)) + # colPtrs + p.struct.elements.append(build_int_list(value.colPtrs.tolist())) + # rowIndices + p.struct.elements.append(build_int_list(value.rowIndices.tolist())) + # values + p.struct.elements.append(build_float_list(value.values.tolist())) + # isTransposed + p.struct.elements.append(pb2.Expression.Literal(boolean=value.isTransposed)) + return p + + elif isinstance(value, DenseMatrix): + p = pb2.Expression.Literal() + p.struct.struct_type.CopyFrom(pyspark_types_to_proto_types(MatrixUDT.sqlType())) + # type = 1 + p.struct.elements.append(pb2.Expression.Literal(byte=1)) + # numRows + p.struct.elements.append(pb2.Expression.Literal(integer=value.numRows)) + # numCols + p.struct.elements.append(pb2.Expression.Literal(integer=value.numCols)) + # colPtrs = null + p.struct.elements.append(literal_null()) + # rowIndices = null + p.struct.elements.append(literal_null()) + # values + p.struct.elements.append(build_float_list(value.values.tolist())) + # isTransposed + p.struct.elements.append(pb2.Expression.Literal(boolean=value.isTransposed)) + return p + else: - literal = LiteralExpression._from_value(value).to_plan(client).literal - return pb2.Param(literal=literal) + return LiteralExpression._from_value(value).to_plan(client).literal def serialize(client: "SparkConnectClient", *args: Any) -> List[Any]: @@ -80,38 +133,51 @@ def serialize(client: "SparkConnectClient", *args: Any) -> List[Any]: return result -def deserialize_param(param: pb2.Param) -> Any: - if param.HasField("literal"): - return LiteralExpression._to_value(param.literal) - if param.HasField("vector"): - vector = param.vector - if vector.HasField("dense"): - return Vectors.dense(vector.dense.value) - elif vector.HasField("sparse"): - return Vectors.sparse(vector.sparse.size, vector.sparse.index, vector.sparse.value) - else: - raise ValueError("Unsupported vector type") - if param.HasField("matrix"): - matrix = param.matrix - if matrix.HasField("dense"): - return DenseMatrix( - matrix.dense.num_rows, - matrix.dense.num_cols, - matrix.dense.value, - matrix.dense.is_transposed, - ) - elif matrix.HasField("sparse"): - return Matrices.sparse( - matrix.sparse.num_rows, - matrix.sparse.num_cols, - matrix.sparse.colptr, - matrix.sparse.row_index, - matrix.sparse.value, - ) +def deserialize_param(literal: pb2.Expression.Literal) -> Any: + from pyspark.sql.connect.types import proto_schema_to_pyspark_data_type + from pyspark.sql.connect.expressions import LiteralExpression + + if literal.HasField("struct"): + s = literal.struct + schema = proto_schema_to_pyspark_data_type(s.struct_type) + + if schema == VectorUDT.sqlType(): + assert len(s.elements) == 4 + tpe = s.elements[0].byte + if tpe == 0: + size = s.elements[1].integer + indices = s.elements[2].specialized_array.ints.values + values = s.elements[3].specialized_array.doubles.values + return SparseVector(size, indices, values) + elif tpe == 1: + values = s.elements[3].specialized_array.doubles.values + return DenseVector(values) + else: + raise ValueError(f"Unknown Vector type {tpe}") + + elif schema == MatrixUDT.sqlType(): + assert len(s.elements) == 7 + tpe = s.elements[0].byte + if tpe == 0: + numRows = s.elements[1].integer + numCols = s.elements[2].integer + colPtrs = s.elements[3].specialized_array.ints.values + rowIndices = s.elements[4].specialized_array.ints.values + values = s.elements[5].specialized_array.doubles.values + isTransposed = s.elements[6].boolean + return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) + elif tpe == 1: + numRows = s.elements[1].integer + numCols = s.elements[2].integer + values = s.elements[5].specialized_array.doubles.values + isTransposed = s.elements[6].boolean + return DenseMatrix(numRows, numCols, values, isTransposed) + else: + raise ValueError(f"Unknown Matrix type {tpe}") else: - raise ValueError("Unsupported matrix type") - - raise ValueError("Unsupported param type") + raise ValueError(f"Unsupported parameter struct {schema}") + else: + return LiteralExpression._to_value(literal) def deserialize(ml_command_result_properties: Dict[str, Any]) -> Any: @@ -126,7 +192,7 @@ def deserialize(ml_command_result_properties: Dict[str, Any]) -> Any: def serialize_ml_params(instance: "Params", client: "SparkConnectClient") -> pb2.MlParams: - params: Mapping[str, pb2.Param] = { + params: Mapping[str, pb2.Expression.Literal] = { k.name: serialize_param(v, client) for k, v in instance._paramMap.items() } return pb2.MlParams(params=params) diff --git a/python/pyspark/sql/connect/proto/common_pb2.py b/python/pyspark/sql/connect/proto/common_pb2.py index 85f475b31a9e3..2f57ce2f73b4f 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.py +++ b/python/pyspark/sql/connect/proto/common_pb2.py @@ -35,7 +35,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1aspark/connect/common.proto\x12\rspark.connect"\xb0\x01\n\x0cStorageLevel\x12\x19\n\x08use_disk\x18\x01 \x01(\x08R\x07useDisk\x12\x1d\n\nuse_memory\x18\x02 \x01(\x08R\tuseMemory\x12 \n\x0cuse_off_heap\x18\x03 \x01(\x08R\nuseOffHeap\x12"\n\x0c\x64\x65serialized\x18\x04 \x01(\x08R\x0c\x64\x65serialized\x12 \n\x0breplication\x18\x05 \x01(\x05R\x0breplication"G\n\x13ResourceInformation\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1c\n\taddresses\x18\x02 \x03(\tR\taddresses"\xc3\x01\n\x17\x45xecutorResourceRequest\x12#\n\rresource_name\x18\x01 \x01(\tR\x0cresourceName\x12\x16\n\x06\x61mount\x18\x02 \x01(\x03R\x06\x61mount\x12.\n\x10\x64iscovery_script\x18\x03 \x01(\tH\x00R\x0f\x64iscoveryScript\x88\x01\x01\x12\x1b\n\x06vendor\x18\x04 \x01(\tH\x01R\x06vendor\x88\x01\x01\x42\x13\n\x11_discovery_scriptB\t\n\x07_vendor"R\n\x13TaskResourceRequest\x12#\n\rresource_name\x18\x01 \x01(\tR\x0cresourceName\x12\x16\n\x06\x61mount\x18\x02 \x01(\x01R\x06\x61mount"\xa5\x03\n\x0fResourceProfile\x12\x64\n\x12\x65xecutor_resources\x18\x01 \x03(\x0b\x32\x35.spark.connect.ResourceProfile.ExecutorResourcesEntryR\x11\x65xecutorResources\x12X\n\x0etask_resources\x18\x02 \x03(\x0b\x32\x31.spark.connect.ResourceProfile.TaskResourcesEntryR\rtaskResources\x1al\n\x16\x45xecutorResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12<\n\x05value\x18\x02 \x01(\x0b\x32&.spark.connect.ExecutorResourceRequestR\x05value:\x02\x38\x01\x1a\x64\n\x12TaskResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.TaskResourceRequestR\x05value:\x02\x38\x01"X\n\x06Origin\x12\x42\n\rpython_origin\x18\x01 \x01(\x0b\x32\x1b.spark.connect.PythonOriginH\x00R\x0cpythonOriginB\n\n\x08\x66unction"G\n\x0cPythonOrigin\x12\x1a\n\x08\x66ragment\x18\x01 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x02 \x01(\tR\x08\x63\x61llSiteB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1aspark/connect/common.proto\x12\rspark.connect"\xb0\x01\n\x0cStorageLevel\x12\x19\n\x08use_disk\x18\x01 \x01(\x08R\x07useDisk\x12\x1d\n\nuse_memory\x18\x02 \x01(\x08R\tuseMemory\x12 \n\x0cuse_off_heap\x18\x03 \x01(\x08R\nuseOffHeap\x12"\n\x0c\x64\x65serialized\x18\x04 \x01(\x08R\x0c\x64\x65serialized\x12 \n\x0breplication\x18\x05 \x01(\x05R\x0breplication"G\n\x13ResourceInformation\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x1c\n\taddresses\x18\x02 \x03(\tR\taddresses"\xc3\x01\n\x17\x45xecutorResourceRequest\x12#\n\rresource_name\x18\x01 \x01(\tR\x0cresourceName\x12\x16\n\x06\x61mount\x18\x02 \x01(\x03R\x06\x61mount\x12.\n\x10\x64iscovery_script\x18\x03 \x01(\tH\x00R\x0f\x64iscoveryScript\x88\x01\x01\x12\x1b\n\x06vendor\x18\x04 \x01(\tH\x01R\x06vendor\x88\x01\x01\x42\x13\n\x11_discovery_scriptB\t\n\x07_vendor"R\n\x13TaskResourceRequest\x12#\n\rresource_name\x18\x01 \x01(\tR\x0cresourceName\x12\x16\n\x06\x61mount\x18\x02 \x01(\x01R\x06\x61mount"\xa5\x03\n\x0fResourceProfile\x12\x64\n\x12\x65xecutor_resources\x18\x01 \x03(\x0b\x32\x35.spark.connect.ResourceProfile.ExecutorResourcesEntryR\x11\x65xecutorResources\x12X\n\x0etask_resources\x18\x02 \x03(\x0b\x32\x31.spark.connect.ResourceProfile.TaskResourcesEntryR\rtaskResources\x1al\n\x16\x45xecutorResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12<\n\x05value\x18\x02 \x01(\x0b\x32&.spark.connect.ExecutorResourceRequestR\x05value:\x02\x38\x01\x1a\x64\n\x12TaskResourcesEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x38\n\x05value\x18\x02 \x01(\x0b\x32".spark.connect.TaskResourceRequestR\x05value:\x02\x38\x01"X\n\x06Origin\x12\x42\n\rpython_origin\x18\x01 \x01(\x0b\x32\x1b.spark.connect.PythonOriginH\x00R\x0cpythonOriginB\n\n\x08\x66unction"G\n\x0cPythonOrigin\x12\x1a\n\x08\x66ragment\x18\x01 \x01(\tR\x08\x66ragment\x12\x1b\n\tcall_site\x18\x02 \x01(\tR\x08\x63\x61llSite"\x1f\n\x05\x42ools\x12\x16\n\x06values\x18\x01 \x03(\x08R\x06values"\x1e\n\x04Ints\x12\x16\n\x06values\x18\x01 \x03(\x05R\x06values"\x1f\n\x05Longs\x12\x16\n\x06values\x18\x01 \x03(\x03R\x06values" \n\x06\x46loats\x12\x16\n\x06values\x18\x01 \x03(\x02R\x06values"!\n\x07\x44oubles\x12\x16\n\x06values\x18\x01 \x03(\x01R\x06values"!\n\x07Strings\x12\x16\n\x06values\x18\x01 \x03(\tR\x06valuesB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -70,4 +70,16 @@ _globals["_ORIGIN"]._serialized_end = 1091 _globals["_PYTHONORIGIN"]._serialized_start = 1093 _globals["_PYTHONORIGIN"]._serialized_end = 1164 + _globals["_BOOLS"]._serialized_start = 1166 + _globals["_BOOLS"]._serialized_end = 1197 + _globals["_INTS"]._serialized_start = 1199 + _globals["_INTS"]._serialized_end = 1229 + _globals["_LONGS"]._serialized_start = 1231 + _globals["_LONGS"]._serialized_end = 1262 + _globals["_FLOATS"]._serialized_start = 1264 + _globals["_FLOATS"]._serialized_end = 1296 + _globals["_DOUBLES"]._serialized_start = 1298 + _globals["_DOUBLES"]._serialized_end = 1331 + _globals["_STRINGS"]._serialized_start = 1333 + _globals["_STRINGS"]._serialized_end = 1366 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/common_pb2.pyi b/python/pyspark/sql/connect/proto/common_pb2.pyi index eda172e26cf4e..fc5e5adef432e 100644 --- a/python/pyspark/sql/connect/proto/common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/common_pb2.pyi @@ -347,3 +347,105 @@ class PythonOrigin(google.protobuf.message.Message): ) -> None: ... global___PythonOrigin = PythonOrigin + +class Bools(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.bool]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.bool] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Bools = Bools + +class Ints(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.int] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Ints = Ints + +class Longs(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.int] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Longs = Longs + +class Floats(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Floats = Floats + +class Doubles(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.float] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Doubles = Doubles + +class Strings(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + VALUES_FIELD_NUMBER: builtins.int + @property + def values( + self, + ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.str]: ... + def __init__( + self, + *, + values: collections.abc.Iterable[builtins.str] | None = ..., + ) -> None: ... + def ClearField(self, field_name: typing_extensions.Literal["values", b"values"]) -> None: ... + +global___Strings = Strings diff --git a/python/pyspark/sql/connect/proto/expressions_pb2.py b/python/pyspark/sql/connect/proto/expressions_pb2.py index 7edcbcac15c73..021bf4db2a122 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"\x97\x32\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\x9b\x0c\n\x07Literal\x12-\n\x04null\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04null\x12\x18\n\x06\x62inary\x18\x02 \x01(\x0cH\x00R\x06\x62inary\x12\x1a\n\x07\x62oolean\x18\x03 \x01(\x08H\x00R\x07\x62oolean\x12\x14\n\x04\x62yte\x18\x04 \x01(\x05H\x00R\x04\x62yte\x12\x16\n\x05short\x18\x05 \x01(\x05H\x00R\x05short\x12\x1a\n\x07integer\x18\x06 \x01(\x05H\x00R\x07integer\x12\x14\n\x04long\x18\x07 \x01(\x03H\x00R\x04long\x12\x16\n\x05\x66loat\x18\n \x01(\x02H\x00R\x05\x66loat\x12\x18\n\x06\x64ouble\x18\x0b \x01(\x01H\x00R\x06\x64ouble\x12\x45\n\x07\x64\x65\x63imal\x18\x0c \x01(\x0b\x32).spark.connect.Expression.Literal.DecimalH\x00R\x07\x64\x65\x63imal\x12\x18\n\x06string\x18\r \x01(\tH\x00R\x06string\x12\x14\n\x04\x64\x61te\x18\x10 \x01(\x05H\x00R\x04\x64\x61te\x12\x1e\n\ttimestamp\x18\x11 \x01(\x03H\x00R\ttimestamp\x12%\n\rtimestamp_ntz\x18\x12 \x01(\x03H\x00R\x0ctimestampNtz\x12\x61\n\x11\x63\x61lendar_interval\x18\x13 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.CalendarIntervalH\x00R\x10\x63\x61lendarInterval\x12\x30\n\x13year_month_interval\x18\x14 \x01(\x05H\x00R\x11yearMonthInterval\x12,\n\x11\x64\x61y_time_interval\x18\x15 \x01(\x03H\x00R\x0f\x64\x61yTimeInterval\x12?\n\x05\x61rray\x18\x16 \x01(\x0b\x32\'.spark.connect.Expression.Literal.ArrayH\x00R\x05\x61rray\x12\x39\n\x03map\x18\x17 \x01(\x0b\x32%.spark.connect.Expression.Literal.MapH\x00R\x03map\x12\x42\n\x06struct\x18\x18 \x01(\x0b\x32(.spark.connect.Expression.Literal.StructH\x00R\x06struct\x1au\n\x07\x44\x65\x63imal\x12\x14\n\x05value\x18\x01 \x01(\tR\x05value\x12!\n\tprecision\x18\x02 \x01(\x05H\x00R\tprecision\x88\x01\x01\x12\x19\n\x05scale\x18\x03 \x01(\x05H\x01R\x05scale\x88\x01\x01\x42\x0c\n\n_precisionB\x08\n\x06_scale\x1a\x62\n\x10\x43\x61lendarInterval\x12\x16\n\x06months\x18\x01 \x01(\x05R\x06months\x12\x12\n\x04\x64\x61ys\x18\x02 \x01(\x05R\x04\x64\x61ys\x12"\n\x0cmicroseconds\x18\x03 \x01(\x03R\x0cmicroseconds\x1a\x82\x01\n\x05\x41rray\x12:\n\x0c\x65lement_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x0b\x65lementType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lements\x1a\xe3\x01\n\x03Map\x12\x32\n\x08key_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x07keyType\x12\x36\n\nvalue_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\tvalueType\x12\x35\n\x04keys\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x04keys\x12\x39\n\x06values\x18\x04 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1a\x81\x01\n\x06Struct\x12\x38\n\x0bstruct_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\nstructType\x12=\n\x08\x65lements\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x08\x65lementsB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe1\x01\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType"]\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1fspark/connect/expressions.proto\x12\rspark.connect\x1a\x19google/protobuf/any.proto\x1a\x19spark/connect/types.proto\x1a\x1aspark/connect/common.proto"\xbd\x35\n\nExpression\x12\x37\n\x06\x63ommon\x18\x12 \x01(\x0b\x32\x1f.spark.connect.ExpressionCommonR\x06\x63ommon\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12\x62\n\x14unresolved_attribute\x18\x02 \x01(\x0b\x32-.spark.connect.Expression.UnresolvedAttributeH\x00R\x13unresolvedAttribute\x12_\n\x13unresolved_function\x18\x03 \x01(\x0b\x32,.spark.connect.Expression.UnresolvedFunctionH\x00R\x12unresolvedFunction\x12Y\n\x11\x65xpression_string\x18\x04 \x01(\x0b\x32*.spark.connect.Expression.ExpressionStringH\x00R\x10\x65xpressionString\x12S\n\x0funresolved_star\x18\x05 \x01(\x0b\x32(.spark.connect.Expression.UnresolvedStarH\x00R\x0eunresolvedStar\x12\x37\n\x05\x61lias\x18\x06 \x01(\x0b\x32\x1f.spark.connect.Expression.AliasH\x00R\x05\x61lias\x12\x34\n\x04\x63\x61st\x18\x07 \x01(\x0b\x32\x1e.spark.connect.Expression.CastH\x00R\x04\x63\x61st\x12V\n\x10unresolved_regex\x18\x08 \x01(\x0b\x32).spark.connect.Expression.UnresolvedRegexH\x00R\x0funresolvedRegex\x12\x44\n\nsort_order\x18\t \x01(\x0b\x32#.spark.connect.Expression.SortOrderH\x00R\tsortOrder\x12S\n\x0flambda_function\x18\n \x01(\x0b\x32(.spark.connect.Expression.LambdaFunctionH\x00R\x0elambdaFunction\x12:\n\x06window\x18\x0b \x01(\x0b\x32 .spark.connect.Expression.WindowH\x00R\x06window\x12l\n\x18unresolved_extract_value\x18\x0c \x01(\x0b\x32\x30.spark.connect.Expression.UnresolvedExtractValueH\x00R\x16unresolvedExtractValue\x12M\n\rupdate_fields\x18\r \x01(\x0b\x32&.spark.connect.Expression.UpdateFieldsH\x00R\x0cupdateFields\x12\x82\x01\n unresolved_named_lambda_variable\x18\x0e \x01(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableH\x00R\x1dunresolvedNamedLambdaVariable\x12~\n#common_inline_user_defined_function\x18\x0f \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionH\x00R\x1f\x63ommonInlineUserDefinedFunction\x12\x42\n\rcall_function\x18\x10 \x01(\x0b\x32\x1b.spark.connect.CallFunctionH\x00R\x0c\x63\x61llFunction\x12\x64\n\x19named_argument_expression\x18\x11 \x01(\x0b\x32&.spark.connect.NamedArgumentExpressionH\x00R\x17namedArgumentExpression\x12?\n\x0cmerge_action\x18\x13 \x01(\x0b\x32\x1a.spark.connect.MergeActionH\x00R\x0bmergeAction\x12g\n\x1atyped_aggregate_expression\x18\x14 \x01(\x0b\x32\'.spark.connect.TypedAggregateExpressionH\x00R\x18typedAggregateExpression\x12H\n\x0flazy_expression\x18\x15 \x01(\x0b\x32\x1d.spark.connect.LazyExpressionH\x00R\x0elazyExpression\x12T\n\x13subquery_expression\x18\x16 \x01(\x0b\x32!.spark.connect.SubqueryExpressionH\x00R\x12subqueryExpression\x12\x35\n\textension\x18\xe7\x07 \x01(\x0b\x32\x14.google.protobuf.AnyH\x00R\textension\x1a\x8f\x06\n\x06Window\x12\x42\n\x0fwindow_function\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0ewindowFunction\x12@\n\x0epartition_spec\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\rpartitionSpec\x12\x42\n\norder_spec\x18\x03 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\torderSpec\x12K\n\nframe_spec\x18\x04 \x01(\x0b\x32,.spark.connect.Expression.Window.WindowFrameR\tframeSpec\x1a\xed\x03\n\x0bWindowFrame\x12U\n\nframe_type\x18\x01 \x01(\x0e\x32\x36.spark.connect.Expression.Window.WindowFrame.FrameTypeR\tframeType\x12P\n\x05lower\x18\x02 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05lower\x12P\n\x05upper\x18\x03 \x01(\x0b\x32:.spark.connect.Expression.Window.WindowFrame.FrameBoundaryR\x05upper\x1a\x91\x01\n\rFrameBoundary\x12!\n\x0b\x63urrent_row\x18\x01 \x01(\x08H\x00R\ncurrentRow\x12\x1e\n\tunbounded\x18\x02 \x01(\x08H\x00R\tunbounded\x12\x31\n\x05value\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\x05valueB\n\n\x08\x62oundary"O\n\tFrameType\x12\x18\n\x14\x46RAME_TYPE_UNDEFINED\x10\x00\x12\x12\n\x0e\x46RAME_TYPE_ROW\x10\x01\x12\x14\n\x10\x46RAME_TYPE_RANGE\x10\x02\x1a\xa9\x03\n\tSortOrder\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12O\n\tdirection\x18\x02 \x01(\x0e\x32\x31.spark.connect.Expression.SortOrder.SortDirectionR\tdirection\x12U\n\rnull_ordering\x18\x03 \x01(\x0e\x32\x30.spark.connect.Expression.SortOrder.NullOrderingR\x0cnullOrdering"l\n\rSortDirection\x12\x1e\n\x1aSORT_DIRECTION_UNSPECIFIED\x10\x00\x12\x1c\n\x18SORT_DIRECTION_ASCENDING\x10\x01\x12\x1d\n\x19SORT_DIRECTION_DESCENDING\x10\x02"U\n\x0cNullOrdering\x12\x1a\n\x16SORT_NULLS_UNSPECIFIED\x10\x00\x12\x14\n\x10SORT_NULLS_FIRST\x10\x01\x12\x13\n\x0fSORT_NULLS_LAST\x10\x02\x1a\xbb\x02\n\x04\x43\x61st\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12-\n\x04type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x04type\x12\x1b\n\x08type_str\x18\x03 \x01(\tH\x00R\x07typeStr\x12\x44\n\teval_mode\x18\x04 \x01(\x0e\x32\'.spark.connect.Expression.Cast.EvalModeR\x08\x65valMode"b\n\x08\x45valMode\x12\x19\n\x15\x45VAL_MODE_UNSPECIFIED\x10\x00\x12\x14\n\x10\x45VAL_MODE_LEGACY\x10\x01\x12\x12\n\x0e\x45VAL_MODE_ANSI\x10\x02\x12\x11\n\rEVAL_MODE_TRY\x10\x03\x42\x0e\n\x0c\x63\x61st_to_type\x1a\xc1\x0f\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\x12\x61\n\x11specialized_array\x18\x19 \x01(\x0b\x32\x32.spark.connect.Expression.Literal.SpecializedArrayH\x00R\x10specializedArray\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\x65lements\x1a\xc0\x02\n\x10SpecializedArray\x12,\n\x05\x62ools\x18\x01 \x01(\x0b\x32\x14.spark.connect.BoolsH\x00R\x05\x62ools\x12)\n\x04ints\x18\x02 \x01(\x0b\x32\x13.spark.connect.IntsH\x00R\x04ints\x12,\n\x05longs\x18\x03 \x01(\x0b\x32\x14.spark.connect.LongsH\x00R\x05longs\x12/\n\x06\x66loats\x18\x04 \x01(\x0b\x32\x15.spark.connect.FloatsH\x00R\x06\x66loats\x12\x32\n\x07\x64oubles\x18\x05 \x01(\x0b\x32\x16.spark.connect.DoublesH\x00R\x07\x64oubles\x12\x32\n\x07strings\x18\x06 \x01(\x0b\x32\x16.spark.connect.StringsH\x00R\x07stringsB\x0c\n\nvalue_typeB\x0e\n\x0cliteral_type\x1a\xba\x01\n\x13UnresolvedAttribute\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12\x31\n\x12is_metadata_column\x18\x03 \x01(\x08H\x01R\x10isMetadataColumn\x88\x01\x01\x42\n\n\x08_plan_idB\x15\n\x13_is_metadata_column\x1a\x82\x02\n\x12UnresolvedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x1f\n\x0bis_distinct\x18\x03 \x01(\x08R\nisDistinct\x12\x37\n\x18is_user_defined_function\x18\x04 \x01(\x08R\x15isUserDefinedFunction\x12$\n\x0bis_internal\x18\x05 \x01(\x08H\x00R\nisInternal\x88\x01\x01\x42\x0e\n\x0c_is_internal\x1a\x32\n\x10\x45xpressionString\x12\x1e\n\nexpression\x18\x01 \x01(\tR\nexpression\x1a|\n\x0eUnresolvedStar\x12,\n\x0funparsed_target\x18\x01 \x01(\tH\x00R\x0eunparsedTarget\x88\x01\x01\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x01R\x06planId\x88\x01\x01\x42\x12\n\x10_unparsed_targetB\n\n\x08_plan_id\x1aV\n\x0fUnresolvedRegex\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x42\n\n\x08_plan_id\x1a\x84\x01\n\x16UnresolvedExtractValue\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild\x12\x39\n\nextraction\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\nextraction\x1a\xbb\x01\n\x0cUpdateFields\x12\x46\n\x11struct_expression\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x10structExpression\x12\x1d\n\nfield_name\x18\x02 \x01(\tR\tfieldName\x12\x44\n\x10value_expression\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x0fvalueExpression\x1ax\n\x05\x41lias\x12-\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x04\x65xpr\x12\x12\n\x04name\x18\x02 \x03(\tR\x04name\x12\x1f\n\x08metadata\x18\x03 \x01(\tH\x00R\x08metadata\x88\x01\x01\x42\x0b\n\t_metadata\x1a\x9e\x01\n\x0eLambdaFunction\x12\x35\n\x08\x66unction\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08\x66unction\x12U\n\targuments\x18\x02 \x03(\x0b\x32\x37.spark.connect.Expression.UnresolvedNamedLambdaVariableR\targuments\x1a>\n\x1dUnresolvedNamedLambdaVariable\x12\x1d\n\nname_parts\x18\x01 \x03(\tR\tnamePartsB\x0b\n\texpr_type"A\n\x10\x45xpressionCommon\x12-\n\x06origin\x18\x01 \x01(\x0b\x32\x15.spark.connect.OriginR\x06origin"\xec\x02\n\x1f\x43ommonInlineUserDefinedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12\x39\n\npython_udf\x18\x04 \x01(\x0b\x32\x18.spark.connect.PythonUDFH\x00R\tpythonUdf\x12I\n\x10scalar_scala_udf\x18\x05 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFH\x00R\x0escalarScalaUdf\x12\x33\n\x08java_udf\x18\x06 \x01(\x0b\x32\x16.spark.connect.JavaUDFH\x00R\x07javaUdfB\n\n\x08\x66unction"\xcc\x01\n\tPythonUDF\x12\x38\n\x0boutput_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVer\x12/\n\x13\x61\x64\x64itional_includes\x18\x05 \x03(\tR\x12\x61\x64\x64itionalIncludes"\xd6\x01\n\x0eScalarScalaUDF\x12\x18\n\x07payload\x18\x01 \x01(\x0cR\x07payload\x12\x37\n\ninputTypes\x18\x02 \x03(\x0b\x32\x17.spark.connect.DataTypeR\ninputTypes\x12\x37\n\noutputType\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeR\noutputType\x12\x1a\n\x08nullable\x18\x04 \x01(\x08R\x08nullable\x12\x1c\n\taggregate\x18\x05 \x01(\x08R\taggregate"\x95\x01\n\x07JavaUDF\x12\x1d\n\nclass_name\x18\x01 \x01(\tR\tclassName\x12=\n\x0boutput_type\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\noutputType\x88\x01\x01\x12\x1c\n\taggregate\x18\x03 \x01(\x08R\taggregateB\x0e\n\x0c_output_type"c\n\x18TypedAggregateExpression\x12G\n\x10scalar_scala_udf\x18\x01 \x01(\x0b\x32\x1d.spark.connect.ScalarScalaUDFR\x0escalarScalaUdf"l\n\x0c\x43\x61llFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"\\\n\x17NamedArgumentExpression\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\x80\x04\n\x0bMergeAction\x12\x46\n\x0b\x61\x63tion_type\x18\x01 \x01(\x0e\x32%.spark.connect.MergeAction.ActionTypeR\nactionType\x12<\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionH\x00R\tcondition\x88\x01\x01\x12G\n\x0b\x61ssignments\x18\x03 \x03(\x0b\x32%.spark.connect.MergeAction.AssignmentR\x0b\x61ssignments\x1aj\n\nAssignment\x12+\n\x03key\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value"\xa7\x01\n\nActionType\x12\x17\n\x13\x41\x43TION_TYPE_INVALID\x10\x00\x12\x16\n\x12\x41\x43TION_TYPE_DELETE\x10\x01\x12\x16\n\x12\x41\x43TION_TYPE_INSERT\x10\x02\x12\x1b\n\x17\x41\x43TION_TYPE_INSERT_STAR\x10\x03\x12\x16\n\x12\x41\x43TION_TYPE_UPDATE\x10\x04\x12\x1b\n\x17\x41\x43TION_TYPE_UPDATE_STAR\x10\x05\x42\x0c\n\n_condition"A\n\x0eLazyExpression\x12/\n\x05\x63hild\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05\x63hild"\xe1\x01\n\x12SubqueryExpression\x12\x17\n\x07plan_id\x18\x01 \x01(\x03R\x06planId\x12S\n\rsubquery_type\x18\x02 \x01(\x0e\x32..spark.connect.SubqueryExpression.SubqueryTypeR\x0csubqueryType"]\n\x0cSubqueryType\x12\x19\n\x15SUBQUERY_TYPE_UNKNOWN\x10\x00\x12\x18\n\x14SUBQUERY_TYPE_SCALAR\x10\x01\x12\x18\n\x14SUBQUERY_TYPE_EXISTS\x10\x02\x42\x36\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,7 +54,7 @@ "DESCRIPTOR" ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_EXPRESSION"]._serialized_start = 133 - _globals["_EXPRESSION"]._serialized_end = 6556 + _globals["_EXPRESSION"]._serialized_end = 6978 _globals["_EXPRESSION_WINDOW"]._serialized_start = 2060 _globals["_EXPRESSION_WINDOW"]._serialized_end = 2843 _globals["_EXPRESSION_WINDOW_WINDOWFRAME"]._serialized_start = 2350 @@ -74,63 +74,65 @@ _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_start = 3475 _globals["_EXPRESSION_CAST_EVALMODE"]._serialized_end = 3573 _globals["_EXPRESSION_LITERAL"]._serialized_start = 3592 - _globals["_EXPRESSION_LITERAL"]._serialized_end = 5155 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4427 - _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4544 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4546 - _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4644 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4647 - _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4777 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4780 - _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 5007 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 5010 - _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5139 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5158 - _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5344 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5347 - _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 5605 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 5607 - _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 5657 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 5659 - _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 5783 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 5785 - _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 5871 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 5874 - _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 6006 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 6009 - _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6196 - _globals["_EXPRESSION_ALIAS"]._serialized_start = 6198 - _globals["_EXPRESSION_ALIAS"]._serialized_end = 6318 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6321 - _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6479 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6481 - _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6543 - _globals["_EXPRESSIONCOMMON"]._serialized_start = 6558 - _globals["_EXPRESSIONCOMMON"]._serialized_end = 6623 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 6626 - _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 6990 - _globals["_PYTHONUDF"]._serialized_start = 6993 - _globals["_PYTHONUDF"]._serialized_end = 7197 - _globals["_SCALARSCALAUDF"]._serialized_start = 7200 - _globals["_SCALARSCALAUDF"]._serialized_end = 7414 - _globals["_JAVAUDF"]._serialized_start = 7417 - _globals["_JAVAUDF"]._serialized_end = 7566 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7568 - _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 7667 - _globals["_CALLFUNCTION"]._serialized_start = 7669 - _globals["_CALLFUNCTION"]._serialized_end = 7777 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 7779 - _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 7871 - _globals["_MERGEACTION"]._serialized_start = 7874 - _globals["_MERGEACTION"]._serialized_end = 8386 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8096 - _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8202 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8205 - _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8372 - _globals["_LAZYEXPRESSION"]._serialized_start = 8388 - _globals["_LAZYEXPRESSION"]._serialized_end = 8453 - _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8456 - _globals["_SUBQUERYEXPRESSION"]._serialized_end = 8681 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 8588 - _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 8681 + _globals["_EXPRESSION_LITERAL"]._serialized_end = 5577 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_start = 4526 + _globals["_EXPRESSION_LITERAL_DECIMAL"]._serialized_end = 4643 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_start = 4645 + _globals["_EXPRESSION_LITERAL_CALENDARINTERVAL"]._serialized_end = 4743 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_start = 4746 + _globals["_EXPRESSION_LITERAL_ARRAY"]._serialized_end = 4876 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_start = 4879 + _globals["_EXPRESSION_LITERAL_MAP"]._serialized_end = 5106 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_start = 5109 + _globals["_EXPRESSION_LITERAL_STRUCT"]._serialized_end = 5238 + _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_start = 5241 + _globals["_EXPRESSION_LITERAL_SPECIALIZEDARRAY"]._serialized_end = 5561 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_start = 5580 + _globals["_EXPRESSION_UNRESOLVEDATTRIBUTE"]._serialized_end = 5766 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_start = 5769 + _globals["_EXPRESSION_UNRESOLVEDFUNCTION"]._serialized_end = 6027 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_start = 6029 + _globals["_EXPRESSION_EXPRESSIONSTRING"]._serialized_end = 6079 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_start = 6081 + _globals["_EXPRESSION_UNRESOLVEDSTAR"]._serialized_end = 6205 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_start = 6207 + _globals["_EXPRESSION_UNRESOLVEDREGEX"]._serialized_end = 6293 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_start = 6296 + _globals["_EXPRESSION_UNRESOLVEDEXTRACTVALUE"]._serialized_end = 6428 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_start = 6431 + _globals["_EXPRESSION_UPDATEFIELDS"]._serialized_end = 6618 + _globals["_EXPRESSION_ALIAS"]._serialized_start = 6620 + _globals["_EXPRESSION_ALIAS"]._serialized_end = 6740 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_start = 6743 + _globals["_EXPRESSION_LAMBDAFUNCTION"]._serialized_end = 6901 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_start = 6903 + _globals["_EXPRESSION_UNRESOLVEDNAMEDLAMBDAVARIABLE"]._serialized_end = 6965 + _globals["_EXPRESSIONCOMMON"]._serialized_start = 6980 + _globals["_EXPRESSIONCOMMON"]._serialized_end = 7045 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_start = 7048 + _globals["_COMMONINLINEUSERDEFINEDFUNCTION"]._serialized_end = 7412 + _globals["_PYTHONUDF"]._serialized_start = 7415 + _globals["_PYTHONUDF"]._serialized_end = 7619 + _globals["_SCALARSCALAUDF"]._serialized_start = 7622 + _globals["_SCALARSCALAUDF"]._serialized_end = 7836 + _globals["_JAVAUDF"]._serialized_start = 7839 + _globals["_JAVAUDF"]._serialized_end = 7988 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_start = 7990 + _globals["_TYPEDAGGREGATEEXPRESSION"]._serialized_end = 8089 + _globals["_CALLFUNCTION"]._serialized_start = 8091 + _globals["_CALLFUNCTION"]._serialized_end = 8199 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_start = 8201 + _globals["_NAMEDARGUMENTEXPRESSION"]._serialized_end = 8293 + _globals["_MERGEACTION"]._serialized_start = 8296 + _globals["_MERGEACTION"]._serialized_end = 8808 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_start = 8518 + _globals["_MERGEACTION_ASSIGNMENT"]._serialized_end = 8624 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_start = 8627 + _globals["_MERGEACTION_ACTIONTYPE"]._serialized_end = 8794 + _globals["_LAZYEXPRESSION"]._serialized_start = 8810 + _globals["_LAZYEXPRESSION"]._serialized_end = 8875 + _globals["_SUBQUERYEXPRESSION"]._serialized_start = 8878 + _globals["_SUBQUERYEXPRESSION"]._serialized_end = 9103 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_start = 9010 + _globals["_SUBQUERYEXPRESSION_SUBQUERYTYPE"]._serialized_end = 9103 # @@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 1a8c60f673054..b5a844b8b43b6 100644 --- a/python/pyspark/sql/connect/proto/expressions_pb2.pyi +++ b/python/pyspark/sql/connect/proto/expressions_pb2.pyi @@ -578,6 +578,82 @@ class Expression(google.protobuf.message.Message): ], ) -> None: ... + class SpecializedArray(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + BOOLS_FIELD_NUMBER: builtins.int + INTS_FIELD_NUMBER: builtins.int + LONGS_FIELD_NUMBER: builtins.int + FLOATS_FIELD_NUMBER: builtins.int + DOUBLES_FIELD_NUMBER: builtins.int + STRINGS_FIELD_NUMBER: builtins.int + @property + def bools(self) -> pyspark.sql.connect.proto.common_pb2.Bools: ... + @property + def ints(self) -> pyspark.sql.connect.proto.common_pb2.Ints: ... + @property + def longs(self) -> pyspark.sql.connect.proto.common_pb2.Longs: ... + @property + def floats(self) -> pyspark.sql.connect.proto.common_pb2.Floats: ... + @property + def doubles(self) -> pyspark.sql.connect.proto.common_pb2.Doubles: ... + @property + def strings(self) -> pyspark.sql.connect.proto.common_pb2.Strings: ... + def __init__( + self, + *, + bools: pyspark.sql.connect.proto.common_pb2.Bools | None = ..., + ints: pyspark.sql.connect.proto.common_pb2.Ints | None = ..., + longs: pyspark.sql.connect.proto.common_pb2.Longs | None = ..., + floats: pyspark.sql.connect.proto.common_pb2.Floats | None = ..., + doubles: pyspark.sql.connect.proto.common_pb2.Doubles | None = ..., + strings: pyspark.sql.connect.proto.common_pb2.Strings | None = ..., + ) -> None: ... + def HasField( + self, + field_name: typing_extensions.Literal[ + "bools", + b"bools", + "doubles", + b"doubles", + "floats", + b"floats", + "ints", + b"ints", + "longs", + b"longs", + "strings", + b"strings", + "value_type", + b"value_type", + ], + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "bools", + b"bools", + "doubles", + b"doubles", + "floats", + b"floats", + "ints", + b"ints", + "longs", + b"longs", + "strings", + b"strings", + "value_type", + b"value_type", + ], + ) -> None: ... + def WhichOneof( + self, oneof_group: typing_extensions.Literal["value_type", b"value_type"] + ) -> ( + typing_extensions.Literal["bools", "ints", "longs", "floats", "doubles", "strings"] + | None + ): ... + NULL_FIELD_NUMBER: builtins.int BINARY_FIELD_NUMBER: builtins.int BOOLEAN_FIELD_NUMBER: builtins.int @@ -598,6 +674,7 @@ class Expression(google.protobuf.message.Message): ARRAY_FIELD_NUMBER: builtins.int MAP_FIELD_NUMBER: builtins.int STRUCT_FIELD_NUMBER: builtins.int + SPECIALIZED_ARRAY_FIELD_NUMBER: builtins.int @property def null(self) -> pyspark.sql.connect.proto.types_pb2.DataType: ... binary: builtins.bytes @@ -627,6 +704,8 @@ class Expression(google.protobuf.message.Message): def map(self) -> global___Expression.Literal.Map: ... @property def struct(self) -> global___Expression.Literal.Struct: ... + @property + def specialized_array(self) -> global___Expression.Literal.SpecializedArray: ... def __init__( self, *, @@ -650,6 +729,7 @@ class Expression(google.protobuf.message.Message): array: global___Expression.Literal.Array | None = ..., map: global___Expression.Literal.Map | None = ..., struct: global___Expression.Literal.Struct | None = ..., + specialized_array: global___Expression.Literal.SpecializedArray | None = ..., ) -> None: ... def HasField( self, @@ -686,6 +766,8 @@ class Expression(google.protobuf.message.Message): b"null", "short", b"short", + "specialized_array", + b"specialized_array", "string", b"string", "struct", @@ -733,6 +815,8 @@ class Expression(google.protobuf.message.Message): b"null", "short", b"short", + "specialized_array", + b"specialized_array", "string", b"string", "struct", @@ -769,6 +853,7 @@ class Expression(google.protobuf.message.Message): "array", "map", "struct", + "specialized_array", ] | None ): ... diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.py b/python/pyspark/sql/connect/proto/ml_common_pb2.py index 70e0e91652892..43d6a512f48f8 100644 --- a/python/pyspark/sql/connect/proto/ml_common_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.py @@ -38,7 +38,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x1dspark/connect/ml_common.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto"\x98\x01\n\x08MlParams\x12;\n\x06params\x18\x01 \x03(\x0b\x32#.spark.connect.MlParams.ParamsEntryR\x06params\x1aO\n\x0bParamsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x14.spark.connect.ParamR\x05value:\x02\x38\x01"\xb6\x01\n\x05Param\x12=\n\x07literal\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x07literal\x12/\n\x06vector\x18\x02 \x01(\x0b\x32\x15.spark.connect.VectorH\x00R\x06vector\x12/\n\x06matrix\x18\x03 \x01(\x0b\x32\x15.spark.connect.MatrixH\x00R\x06matrixB\x0c\n\nparam_type"\xc9\x01\n\nMlOperator\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x10\n\x03uid\x18\x02 \x01(\tR\x03uid\x12:\n\x04type\x18\x03 \x01(\x0e\x32&.spark.connect.MlOperator.OperatorTypeR\x04type"Y\n\x0cOperatorType\x12\x0f\n\x0bUNSPECIFIED\x10\x00\x12\r\n\tESTIMATOR\x10\x01\x12\x0f\n\x0bTRANSFORMER\x10\x02\x12\r\n\tEVALUATOR\x10\x03\x12\t\n\x05MODEL\x10\x04"\x1b\n\tObjectRef\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02id"\xed\x01\n\x06Vector\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Vector.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Vector.SparseH\x00R\x06sparse\x1a\x1d\n\x05\x44\x65nse\x12\x14\n\x05value\x18\x01 \x03(\x01R\x05value\x1aH\n\x06Sparse\x12\x12\n\x04size\x18\x01 \x01(\x05R\x04size\x12\x14\n\x05index\x18\x02 \x03(\x05R\x05index\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05valueB\r\n\x0bvector_type"\xaf\x03\n\x06Matrix\x12\x33\n\x05\x64\x65nse\x18\x01 \x01(\x0b\x32\x1b.spark.connect.Matrix.DenseH\x00R\x05\x64\x65nse\x12\x36\n\x06sparse\x18\x02 \x01(\x0b\x32\x1c.spark.connect.Matrix.SparseH\x00R\x06sparse\x1ax\n\x05\x44\x65nse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x14\n\x05value\x18\x03 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x04 \x01(\x08R\x0cisTransposed\x1a\xae\x01\n\x06Sparse\x12\x19\n\x08num_rows\x18\x01 \x01(\x05R\x07numRows\x12\x19\n\x08num_cols\x18\x02 \x01(\x05R\x07numCols\x12\x16\n\x06\x63olptr\x18\x03 \x03(\x05R\x06\x63olptr\x12\x1b\n\trow_index\x18\x04 \x03(\x05R\x08rowIndex\x12\x14\n\x05value\x18\x05 \x03(\x01R\x05value\x12#\n\ris_transposed\x18\x06 \x01(\x08R\x0cisTransposedB\r\n\x0bmatrix_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x1dspark/connect/ml_common.proto\x12\rspark.connect\x1a\x1fspark/connect/expressions.proto"\xa5\x01\n\x08MlParams\x12;\n\x06params\x18\x01 \x03(\x0b\x32#.spark.connect.MlParams.ParamsEntryR\x06params\x1a\\\n\x0bParamsEntry\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"\xc9\x01\n\nMlOperator\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12\x10\n\x03uid\x18\x02 \x01(\tR\x03uid\x12:\n\x04type\x18\x03 \x01(\x0e\x32&.spark.connect.MlOperator.OperatorTypeR\x04type"Y\n\x0cOperatorType\x12\x0f\n\x0bUNSPECIFIED\x10\x00\x12\r\n\tESTIMATOR\x10\x01\x12\x0f\n\x0bTRANSFORMER\x10\x02\x12\r\n\tEVALUATOR\x10\x03\x12\t\n\x05MODEL\x10\x04"\x1b\n\tObjectRef\x12\x0e\n\x02id\x18\x01 \x01(\tR\x02idB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -54,27 +54,13 @@ _globals["_MLPARAMS_PARAMSENTRY"]._loaded_options = None _globals["_MLPARAMS_PARAMSENTRY"]._serialized_options = b"8\001" _globals["_MLPARAMS"]._serialized_start = 82 - _globals["_MLPARAMS"]._serialized_end = 234 + _globals["_MLPARAMS"]._serialized_end = 247 _globals["_MLPARAMS_PARAMSENTRY"]._serialized_start = 155 - _globals["_MLPARAMS_PARAMSENTRY"]._serialized_end = 234 - _globals["_PARAM"]._serialized_start = 237 - _globals["_PARAM"]._serialized_end = 419 - _globals["_MLOPERATOR"]._serialized_start = 422 - _globals["_MLOPERATOR"]._serialized_end = 623 - _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_start = 534 - _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_end = 623 - _globals["_OBJECTREF"]._serialized_start = 625 - _globals["_OBJECTREF"]._serialized_end = 652 - _globals["_VECTOR"]._serialized_start = 655 - _globals["_VECTOR"]._serialized_end = 892 - _globals["_VECTOR_DENSE"]._serialized_start = 774 - _globals["_VECTOR_DENSE"]._serialized_end = 803 - _globals["_VECTOR_SPARSE"]._serialized_start = 805 - _globals["_VECTOR_SPARSE"]._serialized_end = 877 - _globals["_MATRIX"]._serialized_start = 895 - _globals["_MATRIX"]._serialized_end = 1326 - _globals["_MATRIX_DENSE"]._serialized_start = 1014 - _globals["_MATRIX_DENSE"]._serialized_end = 1134 - _globals["_MATRIX_SPARSE"]._serialized_start = 1137 - _globals["_MATRIX_SPARSE"]._serialized_end = 1311 + _globals["_MLPARAMS_PARAMSENTRY"]._serialized_end = 247 + _globals["_MLOPERATOR"]._serialized_start = 250 + _globals["_MLOPERATOR"]._serialized_end = 451 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_start = 362 + _globals["_MLOPERATOR_OPERATORTYPE"]._serialized_end = 451 + _globals["_OBJECTREF"]._serialized_start = 453 + _globals["_OBJECTREF"]._serialized_end = 480 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_common_pb2.pyi b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi index 64029b6679f19..f4688e94c3d55 100644 --- a/python/pyspark/sql/connect/proto/ml_common_pb2.pyi +++ b/python/pyspark/sql/connect/proto/ml_common_pb2.pyi @@ -62,12 +62,12 @@ class MlParams(google.protobuf.message.Message): VALUE_FIELD_NUMBER: builtins.int key: builtins.str @property - def value(self) -> global___Param: ... + def value(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: ... def __init__( self, *, key: builtins.str = ..., - value: global___Param | None = ..., + value: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal["value", b"value"] @@ -80,72 +80,22 @@ class MlParams(google.protobuf.message.Message): @property def params( self, - ) -> google.protobuf.internal.containers.MessageMap[builtins.str, global___Param]: + ) -> google.protobuf.internal.containers.MessageMap[ + builtins.str, pyspark.sql.connect.proto.expressions_pb2.Expression.Literal + ]: """User-supplied params""" def __init__( self, *, - params: collections.abc.Mapping[builtins.str, global___Param] | None = ..., + params: collections.abc.Mapping[ + builtins.str, pyspark.sql.connect.proto.expressions_pb2.Expression.Literal + ] + | None = ..., ) -> None: ... def ClearField(self, field_name: typing_extensions.Literal["params", b"params"]) -> None: ... global___MlParams = MlParams -class Param(google.protobuf.message.Message): - """Represents the parameter type of the ML instance, or the returned value - of the attribute - """ - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - LITERAL_FIELD_NUMBER: builtins.int - VECTOR_FIELD_NUMBER: builtins.int - MATRIX_FIELD_NUMBER: builtins.int - @property - def literal(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: ... - @property - def vector(self) -> global___Vector: ... - @property - def matrix(self) -> global___Matrix: ... - def __init__( - self, - *, - literal: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., - vector: global___Vector | None = ..., - matrix: global___Matrix | None = ..., - ) -> None: ... - def HasField( - self, - field_name: typing_extensions.Literal[ - "literal", - b"literal", - "matrix", - b"matrix", - "param_type", - b"param_type", - "vector", - b"vector", - ], - ) -> builtins.bool: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "literal", - b"literal", - "matrix", - b"matrix", - "param_type", - b"param_type", - "vector", - b"vector", - ], - ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["param_type", b"param_type"] - ) -> typing_extensions.Literal["literal", "vector", "matrix"] | None: ... - -global___Param = Param - class MlOperator(google.protobuf.message.Message): """MLOperator represents the ML operators like (Estimator, Transformer or Evaluator)""" @@ -215,213 +165,3 @@ class ObjectRef(google.protobuf.message.Message): def ClearField(self, field_name: typing_extensions.Literal["id", b"id"]) -> None: ... global___ObjectRef = ObjectRef - -class Vector(google.protobuf.message.Message): - """See pyspark.ml.linalg.Vector""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - class Dense(google.protobuf.message.Message): - """See pyspark.ml.linalg.DenseVector""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - VALUE_FIELD_NUMBER: builtins.int - @property - def value( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... - def __init__( - self, - *, - value: collections.abc.Iterable[builtins.float] | None = ..., - ) -> None: ... - def ClearField(self, field_name: typing_extensions.Literal["value", b"value"]) -> None: ... - - class Sparse(google.protobuf.message.Message): - """See pyspark.ml.linalg.SparseVector""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - SIZE_FIELD_NUMBER: builtins.int - INDEX_FIELD_NUMBER: builtins.int - VALUE_FIELD_NUMBER: builtins.int - size: builtins.int - @property - def index( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... - @property - def value( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... - def __init__( - self, - *, - size: builtins.int = ..., - index: collections.abc.Iterable[builtins.int] | None = ..., - value: collections.abc.Iterable[builtins.float] | None = ..., - ) -> None: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "index", b"index", "size", b"size", "value", b"value" - ], - ) -> None: ... - - DENSE_FIELD_NUMBER: builtins.int - SPARSE_FIELD_NUMBER: builtins.int - @property - def dense(self) -> global___Vector.Dense: ... - @property - def sparse(self) -> global___Vector.Sparse: ... - def __init__( - self, - *, - dense: global___Vector.Dense | None = ..., - sparse: global___Vector.Sparse | None = ..., - ) -> None: ... - def HasField( - self, - field_name: typing_extensions.Literal[ - "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" - ], - ) -> builtins.bool: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "dense", b"dense", "sparse", b"sparse", "vector_type", b"vector_type" - ], - ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["vector_type", b"vector_type"] - ) -> typing_extensions.Literal["dense", "sparse"] | None: ... - -global___Vector = Vector - -class Matrix(google.protobuf.message.Message): - """See pyspark.ml.linalg.Matrix""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - class Dense(google.protobuf.message.Message): - """See pyspark.ml.linalg.DenseMatrix""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - NUM_ROWS_FIELD_NUMBER: builtins.int - NUM_COLS_FIELD_NUMBER: builtins.int - VALUE_FIELD_NUMBER: builtins.int - IS_TRANSPOSED_FIELD_NUMBER: builtins.int - num_rows: builtins.int - num_cols: builtins.int - @property - def value( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... - is_transposed: builtins.bool - def __init__( - self, - *, - num_rows: builtins.int = ..., - num_cols: builtins.int = ..., - value: collections.abc.Iterable[builtins.float] | None = ..., - is_transposed: builtins.bool = ..., - ) -> None: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "is_transposed", - b"is_transposed", - "num_cols", - b"num_cols", - "num_rows", - b"num_rows", - "value", - b"value", - ], - ) -> None: ... - - class Sparse(google.protobuf.message.Message): - """See pyspark.ml.linalg.SparseMatrix""" - - DESCRIPTOR: google.protobuf.descriptor.Descriptor - - NUM_ROWS_FIELD_NUMBER: builtins.int - NUM_COLS_FIELD_NUMBER: builtins.int - COLPTR_FIELD_NUMBER: builtins.int - ROW_INDEX_FIELD_NUMBER: builtins.int - VALUE_FIELD_NUMBER: builtins.int - IS_TRANSPOSED_FIELD_NUMBER: builtins.int - num_rows: builtins.int - num_cols: builtins.int - @property - def colptr( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... - @property - def row_index( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.int]: ... - @property - def value( - self, - ) -> google.protobuf.internal.containers.RepeatedScalarFieldContainer[builtins.float]: ... - is_transposed: builtins.bool - def __init__( - self, - *, - num_rows: builtins.int = ..., - num_cols: builtins.int = ..., - colptr: collections.abc.Iterable[builtins.int] | None = ..., - row_index: collections.abc.Iterable[builtins.int] | None = ..., - value: collections.abc.Iterable[builtins.float] | None = ..., - is_transposed: builtins.bool = ..., - ) -> None: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "colptr", - b"colptr", - "is_transposed", - b"is_transposed", - "num_cols", - b"num_cols", - "num_rows", - b"num_rows", - "row_index", - b"row_index", - "value", - b"value", - ], - ) -> None: ... - - DENSE_FIELD_NUMBER: builtins.int - SPARSE_FIELD_NUMBER: builtins.int - @property - def dense(self) -> global___Matrix.Dense: ... - @property - def sparse(self) -> global___Matrix.Sparse: ... - def __init__( - self, - *, - dense: global___Matrix.Dense | None = ..., - sparse: global___Matrix.Sparse | None = ..., - ) -> None: ... - def HasField( - self, - field_name: typing_extensions.Literal[ - "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" - ], - ) -> builtins.bool: ... - def ClearField( - self, - field_name: typing_extensions.Literal[ - "dense", b"dense", "matrix_type", b"matrix_type", "sparse", b"sparse" - ], - ) -> None: ... - def WhichOneof( - self, oneof_group: typing_extensions.Literal["matrix_type", b"matrix_type"] - ) -> typing_extensions.Literal["dense", "sparse"] | None: ... - -global___Matrix = Matrix diff --git a/python/pyspark/sql/connect/proto/ml_pb2.py b/python/pyspark/sql/connect/proto/ml_pb2.py index 5005f82d5d533..c29d33db547cd 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.py +++ b/python/pyspark/sql/connect/proto/ml_pb2.py @@ -35,11 +35,12 @@ from pyspark.sql.connect.proto import relations_pb2 as spark_dot_connect_dot_relations__pb2 +from pyspark.sql.connect.proto import expressions_pb2 as spark_dot_connect_dot_expressions__pb2 from pyspark.sql.connect.proto import ml_common_pb2 as spark_dot_connect_dot_ml__common__pb2 DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1dspark/connect/relations.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.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\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xe9\x02\n\x0fMlCommandResult\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + b'\n\x16spark/connect/ml.proto\x12\rspark.connect\x1a\x1dspark/connect/relations.proto\x1a\x1fspark/connect/expressions.proto\x1a\x1dspark/connect/ml_common.proto"\xc6\x07\n\tMlCommand\x12\x30\n\x03\x66it\x18\x01 \x01(\x0b\x32\x1c.spark.connect.MlCommand.FitH\x00R\x03\x66it\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x12\x39\n\x06\x64\x65lete\x18\x03 \x01(\x0b\x32\x1f.spark.connect.MlCommand.DeleteH\x00R\x06\x64\x65lete\x12\x36\n\x05write\x18\x04 \x01(\x0b\x32\x1e.spark.connect.MlCommand.WriteH\x00R\x05write\x12\x33\n\x04read\x18\x05 \x01(\x0b\x32\x1d.spark.connect.MlCommand.ReadH\x00R\x04read\x1a\xa2\x01\n\x03\x46it\x12\x37\n\testimator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\testimator\x12/\n\x06params\x18\x02 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x31\n\x07\x64\x61taset\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x07\x64\x61taset\x1a;\n\x06\x44\x65lete\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x1a\xf0\x02\n\x05Write\x12\x37\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x08operator\x12\x33\n\x07obj_ref\x18\x02 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12/\n\x06params\x18\x03 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06params\x12\x12\n\x04path\x18\x04 \x01(\tR\x04path\x12)\n\x10should_overwrite\x18\x05 \x01(\x08R\x0fshouldOverwrite\x12\x45\n\x07options\x18\x06 \x03(\x0b\x32+.spark.connect.MlCommand.Write.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\x42\x06\n\x04type\x1aQ\n\x04Read\x12\x35\n\x08operator\x18\x01 \x01(\x0b\x32\x19.spark.connect.MlOperatorR\x08operator\x12\x12\n\x04path\x18\x02 \x01(\tR\x04pathB\t\n\x07\x63ommand"\xf6\x02\n\x0fMlCommandResult\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12\x1a\n\x07summary\x18\x02 \x01(\tH\x00R\x07summary\x12T\n\roperator_info\x18\x03 \x01(\x0b\x32-.spark.connect.MlCommandResult.MlOperatorInfoH\x00R\x0coperatorInfo\x1a\xa6\x01\n\x0eMlOperatorInfo\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12\x14\n\x04name\x18\x02 \x01(\tH\x00R\x04name\x12\x10\n\x03uid\x18\x03 \x01(\tR\x03uid\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\x06\n\x04typeB\r\n\x0bresult_typeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -52,20 +53,20 @@ ]._serialized_options = b"\n\036org.apache.spark.connect.protoP\001Z\022internal/generated" _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._loaded_options = None _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_options = b"8\001" - _globals["_MLCOMMAND"]._serialized_start = 104 - _globals["_MLCOMMAND"]._serialized_end = 1070 - _globals["_MLCOMMAND_FIT"]._serialized_start = 382 - _globals["_MLCOMMAND_FIT"]._serialized_end = 544 - _globals["_MLCOMMAND_DELETE"]._serialized_start = 546 - _globals["_MLCOMMAND_DELETE"]._serialized_end = 605 - _globals["_MLCOMMAND_WRITE"]._serialized_start = 608 - _globals["_MLCOMMAND_WRITE"]._serialized_end = 976 - _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 910 - _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 968 - _globals["_MLCOMMAND_READ"]._serialized_start = 978 - _globals["_MLCOMMAND_READ"]._serialized_end = 1059 - _globals["_MLCOMMANDRESULT"]._serialized_start = 1073 - _globals["_MLCOMMANDRESULT"]._serialized_end = 1434 - _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1253 - _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1419 + _globals["_MLCOMMAND"]._serialized_start = 137 + _globals["_MLCOMMAND"]._serialized_end = 1103 + _globals["_MLCOMMAND_FIT"]._serialized_start = 415 + _globals["_MLCOMMAND_FIT"]._serialized_end = 577 + _globals["_MLCOMMAND_DELETE"]._serialized_start = 579 + _globals["_MLCOMMAND_DELETE"]._serialized_end = 638 + _globals["_MLCOMMAND_WRITE"]._serialized_start = 641 + _globals["_MLCOMMAND_WRITE"]._serialized_end = 1009 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_start = 943 + _globals["_MLCOMMAND_WRITE_OPTIONSENTRY"]._serialized_end = 1001 + _globals["_MLCOMMAND_READ"]._serialized_start = 1011 + _globals["_MLCOMMAND_READ"]._serialized_end = 1092 + _globals["_MLCOMMANDRESULT"]._serialized_start = 1106 + _globals["_MLCOMMANDRESULT"]._serialized_end = 1480 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_start = 1299 + _globals["_MLCOMMANDRESULT_MLOPERATORINFO"]._serialized_end = 1465 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/ml_pb2.pyi b/python/pyspark/sql/connect/proto/ml_pb2.pyi index 95bfefb524e2a..6b950e4c67bb7 100644 --- a/python/pyspark/sql/connect/proto/ml_pb2.pyi +++ b/python/pyspark/sql/connect/proto/ml_pb2.pyi @@ -38,6 +38,7 @@ import collections.abc import google.protobuf.descriptor import google.protobuf.internal.containers import google.protobuf.message +import pyspark.sql.connect.proto.expressions_pb2 import pyspark.sql.connect.proto.ml_common_pb2 import pyspark.sql.connect.proto.relations_pb2 import sys @@ -346,7 +347,7 @@ class MlCommandResult(google.protobuf.message.Message): SUMMARY_FIELD_NUMBER: builtins.int OPERATOR_INFO_FIELD_NUMBER: builtins.int @property - def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: + def param(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: """The result of the attribute""" summary: builtins.str """Evaluate a Dataset in a model and return the cached ID of summary""" @@ -356,7 +357,7 @@ class MlCommandResult(google.protobuf.message.Message): def __init__( self, *, - param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + param: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., summary: builtins.str = ..., operator_info: global___MlCommandResult.MlOperatorInfo | None = ..., ) -> None: ... diff --git a/python/pyspark/sql/connect/proto/relations_pb2.py b/python/pyspark/sql/connect/proto/relations_pb2.py index 4327d0240b355..422addcf3dfb7 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.py +++ b/python/pyspark/sql/connect/proto/relations_pb2.py @@ -43,7 +43,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\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\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=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\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"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xbe\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xca\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1ar\n\x04\x41rgs\x12,\n\x05param\x18\x01 \x01(\x0b\x32\x14.spark.connect.ParamH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' + 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\x1a\x1dspark/connect/ml_common.proto"\x9c\x1d\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=\n\x0bml_relation\x18\xac\x02 \x01(\x0b\x32\x19.spark.connect.MlRelationH\x00R\nmlRelation\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"\xf8\x02\n\nMlRelation\x12\x43\n\ttransform\x18\x01 \x01(\x0b\x32#.spark.connect.MlRelation.TransformH\x00R\ttransform\x12,\n\x05\x66\x65tch\x18\x02 \x01(\x0b\x32\x14.spark.connect.FetchH\x00R\x05\x66\x65tch\x1a\xeb\x01\n\tTransform\x12\x33\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefH\x00R\x06objRef\x12=\n\x0btransformer\x18\x02 \x01(\x0b\x32\x19.spark.connect.MlOperatorH\x00R\x0btransformer\x12-\n\x05input\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06params\x18\x04 \x01(\x0b\x32\x17.spark.connect.MlParamsR\x06paramsB\n\n\x08operatorB\t\n\x07ml_type"\xcb\x02\n\x05\x46\x65tch\x12\x31\n\x07obj_ref\x18\x01 \x01(\x0b\x32\x18.spark.connect.ObjectRefR\x06objRef\x12\x35\n\x07methods\x18\x02 \x03(\x0b\x32\x1b.spark.connect.Fetch.MethodR\x07methods\x1a\xd7\x01\n\x06Method\x12\x16\n\x06method\x18\x01 \x01(\tR\x06method\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32 .spark.connect.Fetch.Method.ArgsR\x04\x61rgs\x1a\x7f\n\x04\x41rgs\x12\x39\n\x05param\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralH\x00R\x05param\x12/\n\x05input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationH\x00R\x05inputB\x0b\n\targs_type"\t\n\x07Unknown"\x8e\x01\n\x0eRelationCommon\x12#\n\x0bsource_info\x18\x01 \x01(\tB\x02\x18\x01R\nsourceInfo\x12\x1c\n\x07plan_id\x18\x02 \x01(\x03H\x00R\x06planId\x88\x01\x01\x12-\n\x06origin\x18\x03 \x01(\x0b\x32\x15.spark.connect.OriginR\x06originB\n\n\x08_plan_id"\xde\x03\n\x03SQL\x12\x14\n\x05query\x18\x01 \x01(\tR\x05query\x12\x34\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.spark.connect.SQL.ArgsEntryB\x02\x18\x01R\x04\x61rgs\x12@\n\x08pos_args\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralB\x02\x18\x01R\x07posArgs\x12O\n\x0fnamed_arguments\x18\x04 \x03(\x0b\x32&.spark.connect.SQL.NamedArgumentsEntryR\x0enamedArguments\x12>\n\rpos_arguments\x18\x05 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cposArguments\x1aZ\n\tArgsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x37\n\x05value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x05value:\x02\x38\x01\x1a\\\n\x13NamedArgumentsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12/\n\x05value\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x05value:\x02\x38\x01"u\n\rWithRelations\x12+\n\x04root\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04root\x12\x37\n\nreferences\x18\x02 \x03(\x0b\x32\x17.spark.connect.RelationR\nreferences"\x97\x05\n\x04Read\x12\x41\n\x0bnamed_table\x18\x01 \x01(\x0b\x32\x1e.spark.connect.Read.NamedTableH\x00R\nnamedTable\x12\x41\n\x0b\x64\x61ta_source\x18\x02 \x01(\x0b\x32\x1e.spark.connect.Read.DataSourceH\x00R\ndataSource\x12!\n\x0cis_streaming\x18\x03 \x01(\x08R\x0bisStreaming\x1a\xc0\x01\n\nNamedTable\x12/\n\x13unparsed_identifier\x18\x01 \x01(\tR\x12unparsedIdentifier\x12\x45\n\x07options\x18\x02 \x03(\x0b\x32+.spark.connect.Read.NamedTable.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x95\x02\n\nDataSource\x12\x1b\n\x06\x66ormat\x18\x01 \x01(\tH\x00R\x06\x66ormat\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x12\x45\n\x07options\x18\x03 \x03(\x0b\x32+.spark.connect.Read.DataSource.OptionsEntryR\x07options\x12\x14\n\x05paths\x18\x04 \x03(\tR\x05paths\x12\x1e\n\npredicates\x18\x05 \x03(\tR\npredicates\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x42\t\n\x07_formatB\t\n\x07_schemaB\x0b\n\tread_type"u\n\x07Project\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12;\n\x0b\x65xpressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0b\x65xpressions"p\n\x06\x46ilter\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x37\n\tcondition\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\tcondition"\x95\x05\n\x04Join\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinType\x12#\n\rusing_columns\x18\x05 \x03(\tR\x0cusingColumns\x12K\n\x0ejoin_data_type\x18\x06 \x01(\x0b\x32 .spark.connect.Join.JoinDataTypeH\x00R\x0cjoinDataType\x88\x01\x01\x1a\\\n\x0cJoinDataType\x12$\n\x0eis_left_struct\x18\x01 \x01(\x08R\x0cisLeftStruct\x12&\n\x0fis_right_struct\x18\x02 \x01(\x08R\risRightStruct"\xd0\x01\n\x08JoinType\x12\x19\n\x15JOIN_TYPE_UNSPECIFIED\x10\x00\x12\x13\n\x0fJOIN_TYPE_INNER\x10\x01\x12\x18\n\x14JOIN_TYPE_FULL_OUTER\x10\x02\x12\x18\n\x14JOIN_TYPE_LEFT_OUTER\x10\x03\x12\x19\n\x15JOIN_TYPE_RIGHT_OUTER\x10\x04\x12\x17\n\x13JOIN_TYPE_LEFT_ANTI\x10\x05\x12\x17\n\x13JOIN_TYPE_LEFT_SEMI\x10\x06\x12\x13\n\x0fJOIN_TYPE_CROSS\x10\x07\x42\x11\n\x0f_join_data_type"\xdf\x03\n\x0cSetOperation\x12\x36\n\nleft_input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\tleftInput\x12\x38\n\x0bright_input\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\nrightInput\x12\x45\n\x0bset_op_type\x18\x03 \x01(\x0e\x32%.spark.connect.SetOperation.SetOpTypeR\tsetOpType\x12\x1a\n\x06is_all\x18\x04 \x01(\x08H\x00R\x05isAll\x88\x01\x01\x12\x1c\n\x07\x62y_name\x18\x05 \x01(\x08H\x01R\x06\x62yName\x88\x01\x01\x12\x37\n\x15\x61llow_missing_columns\x18\x06 \x01(\x08H\x02R\x13\x61llowMissingColumns\x88\x01\x01"r\n\tSetOpType\x12\x1b\n\x17SET_OP_TYPE_UNSPECIFIED\x10\x00\x12\x19\n\x15SET_OP_TYPE_INTERSECT\x10\x01\x12\x15\n\x11SET_OP_TYPE_UNION\x10\x02\x12\x16\n\x12SET_OP_TYPE_EXCEPT\x10\x03\x42\t\n\x07_is_allB\n\n\x08_by_nameB\x18\n\x16_allow_missing_columns"L\n\x05Limit\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"O\n\x06Offset\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x16\n\x06offset\x18\x02 \x01(\x05R\x06offset"K\n\x04Tail\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05limit\x18\x02 \x01(\x05R\x05limit"\xfe\x05\n\tAggregate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x41\n\ngroup_type\x18\x02 \x01(\x0e\x32".spark.connect.Aggregate.GroupTypeR\tgroupType\x12L\n\x14grouping_expressions\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12N\n\x15\x61ggregate_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x14\x61ggregateExpressions\x12\x34\n\x05pivot\x18\x05 \x01(\x0b\x32\x1e.spark.connect.Aggregate.PivotR\x05pivot\x12J\n\rgrouping_sets\x18\x06 \x03(\x0b\x32%.spark.connect.Aggregate.GroupingSetsR\x0cgroupingSets\x1ao\n\x05Pivot\x12+\n\x03\x63ol\x18\x01 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x39\n\x06values\x18\x02 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values\x1aL\n\x0cGroupingSets\x12<\n\x0cgrouping_set\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0bgroupingSet"\x9f\x01\n\tGroupType\x12\x1a\n\x16GROUP_TYPE_UNSPECIFIED\x10\x00\x12\x16\n\x12GROUP_TYPE_GROUPBY\x10\x01\x12\x15\n\x11GROUP_TYPE_ROLLUP\x10\x02\x12\x13\n\x0fGROUP_TYPE_CUBE\x10\x03\x12\x14\n\x10GROUP_TYPE_PIVOT\x10\x04\x12\x1c\n\x18GROUP_TYPE_GROUPING_SETS\x10\x05"\xa0\x01\n\x04Sort\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x05order\x18\x02 \x03(\x0b\x32#.spark.connect.Expression.SortOrderR\x05order\x12 \n\tis_global\x18\x03 \x01(\x08H\x00R\x08isGlobal\x88\x01\x01\x42\x0c\n\n_is_global"\x8d\x01\n\x04\x44rop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x33\n\x07\x63olumns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07\x63olumns\x12!\n\x0c\x63olumn_names\x18\x03 \x03(\tR\x0b\x63olumnNames"\xf0\x01\n\x0b\x44\x65\x64uplicate\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames\x12\x32\n\x13\x61ll_columns_as_keys\x18\x03 \x01(\x08H\x00R\x10\x61llColumnsAsKeys\x88\x01\x01\x12.\n\x10within_watermark\x18\x04 \x01(\x08H\x01R\x0fwithinWatermark\x88\x01\x01\x42\x16\n\x14_all_columns_as_keysB\x13\n\x11_within_watermark"Y\n\rLocalRelation\x12\x17\n\x04\x64\x61ta\x18\x01 \x01(\x0cH\x00R\x04\x64\x61ta\x88\x01\x01\x12\x1b\n\x06schema\x18\x02 \x01(\tH\x01R\x06schema\x88\x01\x01\x42\x07\n\x05_dataB\t\n\x07_schema"H\n\x13\x43\x61\x63hedLocalRelation\x12\x12\n\x04hash\x18\x03 \x01(\tR\x04hashJ\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03R\x06userIdR\tsessionId"7\n\x14\x43\x61\x63hedRemoteRelation\x12\x1f\n\x0brelation_id\x18\x01 \x01(\tR\nrelationId"\x91\x02\n\x06Sample\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1f\n\x0blower_bound\x18\x02 \x01(\x01R\nlowerBound\x12\x1f\n\x0bupper_bound\x18\x03 \x01(\x01R\nupperBound\x12.\n\x10with_replacement\x18\x04 \x01(\x08H\x00R\x0fwithReplacement\x88\x01\x01\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x01R\x04seed\x88\x01\x01\x12/\n\x13\x64\x65terministic_order\x18\x06 \x01(\x08R\x12\x64\x65terministicOrderB\x13\n\x11_with_replacementB\x07\n\x05_seed"\x91\x01\n\x05Range\x12\x19\n\x05start\x18\x01 \x01(\x03H\x00R\x05start\x88\x01\x01\x12\x10\n\x03\x65nd\x18\x02 \x01(\x03R\x03\x65nd\x12\x12\n\x04step\x18\x03 \x01(\x03R\x04step\x12*\n\x0enum_partitions\x18\x04 \x01(\x05H\x01R\rnumPartitions\x88\x01\x01\x42\x08\n\x06_startB\x11\n\x0f_num_partitions"r\n\rSubqueryAlias\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x14\n\x05\x61lias\x18\x02 \x01(\tR\x05\x61lias\x12\x1c\n\tqualifier\x18\x03 \x03(\tR\tqualifier"\x8e\x01\n\x0bRepartition\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12%\n\x0enum_partitions\x18\x02 \x01(\x05R\rnumPartitions\x12\x1d\n\x07shuffle\x18\x03 \x01(\x08H\x00R\x07shuffle\x88\x01\x01\x42\n\n\x08_shuffle"\x8e\x01\n\nShowString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate\x12\x1a\n\x08vertical\x18\x04 \x01(\x08R\x08vertical"r\n\nHtmlString\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x19\n\x08num_rows\x18\x02 \x01(\x05R\x07numRows\x12\x1a\n\x08truncate\x18\x03 \x01(\x05R\x08truncate"\\\n\x0bStatSummary\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1e\n\nstatistics\x18\x02 \x03(\tR\nstatistics"Q\n\x0cStatDescribe\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols"e\n\x0cStatCrosstab\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"`\n\x07StatCov\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2"\x89\x01\n\x08StatCorr\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ol1\x18\x02 \x01(\tR\x04\x63ol1\x12\x12\n\x04\x63ol2\x18\x03 \x01(\tR\x04\x63ol2\x12\x1b\n\x06method\x18\x04 \x01(\tH\x00R\x06method\x88\x01\x01\x42\t\n\x07_method"\xa4\x01\n\x12StatApproxQuantile\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12$\n\rprobabilities\x18\x03 \x03(\x01R\rprobabilities\x12%\n\x0erelative_error\x18\x04 \x01(\x01R\rrelativeError"}\n\rStatFreqItems\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x1d\n\x07support\x18\x03 \x01(\x01H\x00R\x07support\x88\x01\x01\x42\n\n\x08_support"\xb5\x02\n\x0cStatSampleBy\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03\x63ol\x18\x02 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x03\x63ol\x12\x42\n\tfractions\x18\x03 \x03(\x0b\x32$.spark.connect.StatSampleBy.FractionR\tfractions\x12\x17\n\x04seed\x18\x05 \x01(\x03H\x00R\x04seed\x88\x01\x01\x1a\x63\n\x08\x46raction\x12;\n\x07stratum\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x07stratum\x12\x1a\n\x08\x66raction\x18\x02 \x01(\x01R\x08\x66ractionB\x07\n\x05_seed"\x86\x01\n\x06NAFill\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\x39\n\x06values\x18\x03 \x03(\x0b\x32!.spark.connect.Expression.LiteralR\x06values"\x86\x01\n\x06NADrop\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12\'\n\rmin_non_nulls\x18\x03 \x01(\x05H\x00R\x0bminNonNulls\x88\x01\x01\x42\x10\n\x0e_min_non_nulls"\xa8\x02\n\tNAReplace\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04\x63ols\x18\x02 \x03(\tR\x04\x63ols\x12H\n\x0creplacements\x18\x03 \x03(\x0b\x32$.spark.connect.NAReplace.ReplacementR\x0creplacements\x1a\x8d\x01\n\x0bReplacement\x12>\n\told_value\x18\x01 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08oldValue\x12>\n\tnew_value\x18\x02 \x01(\x0b\x32!.spark.connect.Expression.LiteralR\x08newValue"X\n\x04ToDF\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12!\n\x0c\x63olumn_names\x18\x02 \x03(\tR\x0b\x63olumnNames"\xfe\x02\n\x12WithColumnsRenamed\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12i\n\x12rename_columns_map\x18\x02 \x03(\x0b\x32\x37.spark.connect.WithColumnsRenamed.RenameColumnsMapEntryB\x02\x18\x01R\x10renameColumnsMap\x12\x42\n\x07renames\x18\x03 \x03(\x0b\x32(.spark.connect.WithColumnsRenamed.RenameR\x07renames\x1a\x43\n\x15RenameColumnsMapEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x45\n\x06Rename\x12\x19\n\x08\x63ol_name\x18\x01 \x01(\tR\x07\x63olName\x12 \n\x0cnew_col_name\x18\x02 \x01(\tR\nnewColName"w\n\x0bWithColumns\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x39\n\x07\x61liases\x18\x02 \x03(\x0b\x32\x1f.spark.connect.Expression.AliasR\x07\x61liases"\x86\x01\n\rWithWatermark\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x1d\n\nevent_time\x18\x02 \x01(\tR\teventTime\x12\'\n\x0f\x64\x65lay_threshold\x18\x03 \x01(\tR\x0e\x64\x65layThreshold"\x84\x01\n\x04Hint\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x39\n\nparameters\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\nparameters"\xc7\x02\n\x07Unpivot\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12+\n\x03ids\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x03ids\x12:\n\x06values\x18\x03 \x01(\x0b\x32\x1d.spark.connect.Unpivot.ValuesH\x00R\x06values\x88\x01\x01\x12\x30\n\x14variable_column_name\x18\x04 \x01(\tR\x12variableColumnName\x12*\n\x11value_column_name\x18\x05 \x01(\tR\x0fvalueColumnName\x1a;\n\x06Values\x12\x31\n\x06values\x18\x01 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x06valuesB\t\n\x07_values"z\n\tTranspose\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12>\n\rindex_columns\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0cindexColumns"}\n\x1dUnresolvedTableValuedFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12\x37\n\targuments\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments"j\n\x08ToSchema\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12/\n\x06schema\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x06schema"\xcb\x01\n\x17RepartitionByExpression\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x0fpartition_exprs\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x0epartitionExprs\x12*\n\x0enum_partitions\x18\x03 \x01(\x05H\x00R\rnumPartitions\x88\x01\x01\x42\x11\n\x0f_num_partitions"\xe8\x01\n\rMapPartitions\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x42\n\x04\x66unc\x18\x02 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12"\n\nis_barrier\x18\x03 \x01(\x08H\x00R\tisBarrier\x88\x01\x01\x12"\n\nprofile_id\x18\x04 \x01(\x05H\x01R\tprofileId\x88\x01\x01\x42\r\n\x0b_is_barrierB\r\n\x0b_profile_id"\xcd\x05\n\x08GroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12J\n\x13sorting_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x12sortingExpressions\x12<\n\rinitial_input\x18\x05 \x01(\x0b\x32\x17.spark.connect.RelationR\x0cinitialInput\x12[\n\x1cinitial_grouping_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x1ainitialGroupingExpressions\x12;\n\x18is_map_groups_with_state\x18\x07 \x01(\x08H\x00R\x14isMapGroupsWithState\x88\x01\x01\x12$\n\x0boutput_mode\x18\x08 \x01(\tH\x01R\noutputMode\x88\x01\x01\x12&\n\x0ctimeout_conf\x18\t \x01(\tH\x02R\x0btimeoutConf\x88\x01\x01\x12?\n\x0cstate_schema\x18\n \x01(\x0b\x32\x17.spark.connect.DataTypeH\x03R\x0bstateSchema\x88\x01\x01\x42\x1b\n\x19_is_map_groups_with_stateB\x0e\n\x0c_output_modeB\x0f\n\r_timeout_confB\x0f\n\r_state_schema"\x8e\x04\n\nCoGroupMap\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12W\n\x1ainput_grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18inputGroupingExpressions\x12-\n\x05other\x18\x03 \x01(\x0b\x32\x17.spark.connect.RelationR\x05other\x12W\n\x1aother_grouping_expressions\x18\x04 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x18otherGroupingExpressions\x12\x42\n\x04\x66unc\x18\x05 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12U\n\x19input_sorting_expressions\x18\x06 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17inputSortingExpressions\x12U\n\x19other_sorting_expressions\x18\x07 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x17otherSortingExpressions"\xe5\x02\n\x16\x41pplyInPandasWithState\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12L\n\x14grouping_expressions\x18\x02 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x13groupingExpressions\x12\x42\n\x04\x66unc\x18\x03 \x01(\x0b\x32..spark.connect.CommonInlineUserDefinedFunctionR\x04\x66unc\x12#\n\routput_schema\x18\x04 \x01(\tR\x0coutputSchema\x12!\n\x0cstate_schema\x18\x05 \x01(\tR\x0bstateSchema\x12\x1f\n\x0boutput_mode\x18\x06 \x01(\tR\noutputMode\x12!\n\x0ctimeout_conf\x18\x07 \x01(\tR\x0btimeoutConf"\xf4\x01\n$CommonInlineUserDefinedTableFunction\x12#\n\rfunction_name\x18\x01 \x01(\tR\x0c\x66unctionName\x12$\n\rdeterministic\x18\x02 \x01(\x08R\rdeterministic\x12\x37\n\targuments\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\targuments\x12<\n\x0bpython_udtf\x18\x04 \x01(\x0b\x32\x19.spark.connect.PythonUDTFH\x00R\npythonUdtfB\n\n\x08\x66unction"\xb1\x01\n\nPythonUDTF\x12=\n\x0breturn_type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\nreturnType\x88\x01\x01\x12\x1b\n\teval_type\x18\x02 \x01(\x05R\x08\x65valType\x12\x18\n\x07\x63ommand\x18\x03 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x04 \x01(\tR\tpythonVerB\x0e\n\x0c_return_type"\x97\x01\n!CommonInlineUserDefinedDataSource\x12\x12\n\x04name\x18\x01 \x01(\tR\x04name\x12O\n\x12python_data_source\x18\x02 \x01(\x0b\x32\x1f.spark.connect.PythonDataSourceH\x00R\x10pythonDataSourceB\r\n\x0b\x64\x61ta_source"K\n\x10PythonDataSource\x12\x18\n\x07\x63ommand\x18\x01 \x01(\x0cR\x07\x63ommand\x12\x1d\n\npython_ver\x18\x02 \x01(\tR\tpythonVer"\x88\x01\n\x0e\x43ollectMetrics\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x33\n\x07metrics\x18\x03 \x03(\x0b\x32\x19.spark.connect.ExpressionR\x07metrics"\x84\x03\n\x05Parse\x12-\n\x05input\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x05input\x12\x38\n\x06\x66ormat\x18\x02 \x01(\x0e\x32 .spark.connect.Parse.ParseFormatR\x06\x66ormat\x12\x34\n\x06schema\x18\x03 \x01(\x0b\x32\x17.spark.connect.DataTypeH\x00R\x06schema\x88\x01\x01\x12;\n\x07options\x18\x04 \x03(\x0b\x32!.spark.connect.Parse.OptionsEntryR\x07options\x1a:\n\x0cOptionsEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01"X\n\x0bParseFormat\x12\x1c\n\x18PARSE_FORMAT_UNSPECIFIED\x10\x00\x12\x14\n\x10PARSE_FORMAT_CSV\x10\x01\x12\x15\n\x11PARSE_FORMAT_JSON\x10\x02\x42\t\n\x07_schema"\xdb\x03\n\x08\x41sOfJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12\x37\n\nleft_as_of\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08leftAsOf\x12\x39\n\x0bright_as_of\x18\x04 \x01(\x0b\x32\x19.spark.connect.ExpressionR\trightAsOf\x12\x36\n\tjoin_expr\x18\x05 \x01(\x0b\x32\x19.spark.connect.ExpressionR\x08joinExpr\x12#\n\rusing_columns\x18\x06 \x03(\tR\x0cusingColumns\x12\x1b\n\tjoin_type\x18\x07 \x01(\tR\x08joinType\x12\x37\n\ttolerance\x18\x08 \x01(\x0b\x32\x19.spark.connect.ExpressionR\ttolerance\x12.\n\x13\x61llow_exact_matches\x18\t \x01(\x08R\x11\x61llowExactMatches\x12\x1c\n\tdirection\x18\n \x01(\tR\tdirection"\xe6\x01\n\x0bLateralJoin\x12+\n\x04left\x18\x01 \x01(\x0b\x32\x17.spark.connect.RelationR\x04left\x12-\n\x05right\x18\x02 \x01(\x0b\x32\x17.spark.connect.RelationR\x05right\x12@\n\x0ejoin_condition\x18\x03 \x01(\x0b\x32\x19.spark.connect.ExpressionR\rjoinCondition\x12\x39\n\tjoin_type\x18\x04 \x01(\x0e\x32\x1c.spark.connect.Join.JoinTypeR\x08joinTypeB6\n\x1eorg.apache.spark.connect.protoP\x01Z\x12internal/generatedb\x06proto3' ) _globals = globals() @@ -85,163 +85,163 @@ _globals["_MLRELATION_TRANSFORM"]._serialized_start = 4097 _globals["_MLRELATION_TRANSFORM"]._serialized_end = 4332 _globals["_FETCH"]._serialized_start = 4346 - _globals["_FETCH"]._serialized_end = 4664 + _globals["_FETCH"]._serialized_end = 4677 _globals["_FETCH_METHOD"]._serialized_start = 4462 - _globals["_FETCH_METHOD"]._serialized_end = 4664 + _globals["_FETCH_METHOD"]._serialized_end = 4677 _globals["_FETCH_METHOD_ARGS"]._serialized_start = 4550 - _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4664 - _globals["_UNKNOWN"]._serialized_start = 4666 - _globals["_UNKNOWN"]._serialized_end = 4675 - _globals["_RELATIONCOMMON"]._serialized_start = 4678 - _globals["_RELATIONCOMMON"]._serialized_end = 4820 - _globals["_SQL"]._serialized_start = 4823 - _globals["_SQL"]._serialized_end = 5301 - _globals["_SQL_ARGSENTRY"]._serialized_start = 5117 - _globals["_SQL_ARGSENTRY"]._serialized_end = 5207 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5209 - _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5301 - _globals["_WITHRELATIONS"]._serialized_start = 5303 - _globals["_WITHRELATIONS"]._serialized_end = 5420 - _globals["_READ"]._serialized_start = 5423 - _globals["_READ"]._serialized_end = 6086 - _globals["_READ_NAMEDTABLE"]._serialized_start = 5601 - _globals["_READ_NAMEDTABLE"]._serialized_end = 5793 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5735 - _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5793 - _globals["_READ_DATASOURCE"]._serialized_start = 5796 - _globals["_READ_DATASOURCE"]._serialized_end = 6073 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5735 - _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5793 - _globals["_PROJECT"]._serialized_start = 6088 - _globals["_PROJECT"]._serialized_end = 6205 - _globals["_FILTER"]._serialized_start = 6207 - _globals["_FILTER"]._serialized_end = 6319 - _globals["_JOIN"]._serialized_start = 6322 - _globals["_JOIN"]._serialized_end = 6983 - _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6661 - _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6753 - _globals["_JOIN_JOINTYPE"]._serialized_start = 6756 - _globals["_JOIN_JOINTYPE"]._serialized_end = 6964 - _globals["_SETOPERATION"]._serialized_start = 6986 - _globals["_SETOPERATION"]._serialized_end = 7465 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7302 - _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7416 - _globals["_LIMIT"]._serialized_start = 7467 - _globals["_LIMIT"]._serialized_end = 7543 - _globals["_OFFSET"]._serialized_start = 7545 - _globals["_OFFSET"]._serialized_end = 7624 - _globals["_TAIL"]._serialized_start = 7626 - _globals["_TAIL"]._serialized_end = 7701 - _globals["_AGGREGATE"]._serialized_start = 7704 - _globals["_AGGREGATE"]._serialized_end = 8470 - _globals["_AGGREGATE_PIVOT"]._serialized_start = 8119 - _globals["_AGGREGATE_PIVOT"]._serialized_end = 8230 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8232 - _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8308 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8311 - _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8470 - _globals["_SORT"]._serialized_start = 8473 - _globals["_SORT"]._serialized_end = 8633 - _globals["_DROP"]._serialized_start = 8636 - _globals["_DROP"]._serialized_end = 8777 - _globals["_DEDUPLICATE"]._serialized_start = 8780 - _globals["_DEDUPLICATE"]._serialized_end = 9020 - _globals["_LOCALRELATION"]._serialized_start = 9022 - _globals["_LOCALRELATION"]._serialized_end = 9111 - _globals["_CACHEDLOCALRELATION"]._serialized_start = 9113 - _globals["_CACHEDLOCALRELATION"]._serialized_end = 9185 - _globals["_CACHEDREMOTERELATION"]._serialized_start = 9187 - _globals["_CACHEDREMOTERELATION"]._serialized_end = 9242 - _globals["_SAMPLE"]._serialized_start = 9245 - _globals["_SAMPLE"]._serialized_end = 9518 - _globals["_RANGE"]._serialized_start = 9521 - _globals["_RANGE"]._serialized_end = 9666 - _globals["_SUBQUERYALIAS"]._serialized_start = 9668 - _globals["_SUBQUERYALIAS"]._serialized_end = 9782 - _globals["_REPARTITION"]._serialized_start = 9785 - _globals["_REPARTITION"]._serialized_end = 9927 - _globals["_SHOWSTRING"]._serialized_start = 9930 - _globals["_SHOWSTRING"]._serialized_end = 10072 - _globals["_HTMLSTRING"]._serialized_start = 10074 - _globals["_HTMLSTRING"]._serialized_end = 10188 - _globals["_STATSUMMARY"]._serialized_start = 10190 - _globals["_STATSUMMARY"]._serialized_end = 10282 - _globals["_STATDESCRIBE"]._serialized_start = 10284 - _globals["_STATDESCRIBE"]._serialized_end = 10365 - _globals["_STATCROSSTAB"]._serialized_start = 10367 - _globals["_STATCROSSTAB"]._serialized_end = 10468 - _globals["_STATCOV"]._serialized_start = 10470 - _globals["_STATCOV"]._serialized_end = 10566 - _globals["_STATCORR"]._serialized_start = 10569 - _globals["_STATCORR"]._serialized_end = 10706 - _globals["_STATAPPROXQUANTILE"]._serialized_start = 10709 - _globals["_STATAPPROXQUANTILE"]._serialized_end = 10873 - _globals["_STATFREQITEMS"]._serialized_start = 10875 - _globals["_STATFREQITEMS"]._serialized_end = 11000 - _globals["_STATSAMPLEBY"]._serialized_start = 11003 - _globals["_STATSAMPLEBY"]._serialized_end = 11312 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11204 - _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11303 - _globals["_NAFILL"]._serialized_start = 11315 - _globals["_NAFILL"]._serialized_end = 11449 - _globals["_NADROP"]._serialized_start = 11452 - _globals["_NADROP"]._serialized_end = 11586 - _globals["_NAREPLACE"]._serialized_start = 11589 - _globals["_NAREPLACE"]._serialized_end = 11885 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11744 - _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11885 - _globals["_TODF"]._serialized_start = 11887 - _globals["_TODF"]._serialized_end = 11975 - _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11978 - _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12360 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12222 - _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12289 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12291 - _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12360 - _globals["_WITHCOLUMNS"]._serialized_start = 12362 - _globals["_WITHCOLUMNS"]._serialized_end = 12481 - _globals["_WITHWATERMARK"]._serialized_start = 12484 - _globals["_WITHWATERMARK"]._serialized_end = 12618 - _globals["_HINT"]._serialized_start = 12621 - _globals["_HINT"]._serialized_end = 12753 - _globals["_UNPIVOT"]._serialized_start = 12756 - _globals["_UNPIVOT"]._serialized_end = 13083 - _globals["_UNPIVOT_VALUES"]._serialized_start = 13013 - _globals["_UNPIVOT_VALUES"]._serialized_end = 13072 - _globals["_TRANSPOSE"]._serialized_start = 13085 - _globals["_TRANSPOSE"]._serialized_end = 13207 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13209 - _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13334 - _globals["_TOSCHEMA"]._serialized_start = 13336 - _globals["_TOSCHEMA"]._serialized_end = 13442 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13445 - _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13648 - _globals["_MAPPARTITIONS"]._serialized_start = 13651 - _globals["_MAPPARTITIONS"]._serialized_end = 13883 - _globals["_GROUPMAP"]._serialized_start = 13886 - _globals["_GROUPMAP"]._serialized_end = 14603 - _globals["_COGROUPMAP"]._serialized_start = 14606 - _globals["_COGROUPMAP"]._serialized_end = 15132 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15135 - _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15492 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15495 - _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 15739 - _globals["_PYTHONUDTF"]._serialized_start = 15742 - _globals["_PYTHONUDTF"]._serialized_end = 15919 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15922 - _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16073 - _globals["_PYTHONDATASOURCE"]._serialized_start = 16075 - _globals["_PYTHONDATASOURCE"]._serialized_end = 16150 - _globals["_COLLECTMETRICS"]._serialized_start = 16153 - _globals["_COLLECTMETRICS"]._serialized_end = 16289 - _globals["_PARSE"]._serialized_start = 16292 - _globals["_PARSE"]._serialized_end = 16680 - _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5735 - _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5793 - _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16581 - _globals["_PARSE_PARSEFORMAT"]._serialized_end = 16669 - _globals["_ASOFJOIN"]._serialized_start = 16683 - _globals["_ASOFJOIN"]._serialized_end = 17158 - _globals["_LATERALJOIN"]._serialized_start = 17161 - _globals["_LATERALJOIN"]._serialized_end = 17391 + _globals["_FETCH_METHOD_ARGS"]._serialized_end = 4677 + _globals["_UNKNOWN"]._serialized_start = 4679 + _globals["_UNKNOWN"]._serialized_end = 4688 + _globals["_RELATIONCOMMON"]._serialized_start = 4691 + _globals["_RELATIONCOMMON"]._serialized_end = 4833 + _globals["_SQL"]._serialized_start = 4836 + _globals["_SQL"]._serialized_end = 5314 + _globals["_SQL_ARGSENTRY"]._serialized_start = 5130 + _globals["_SQL_ARGSENTRY"]._serialized_end = 5220 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_start = 5222 + _globals["_SQL_NAMEDARGUMENTSENTRY"]._serialized_end = 5314 + _globals["_WITHRELATIONS"]._serialized_start = 5316 + _globals["_WITHRELATIONS"]._serialized_end = 5433 + _globals["_READ"]._serialized_start = 5436 + _globals["_READ"]._serialized_end = 6099 + _globals["_READ_NAMEDTABLE"]._serialized_start = 5614 + _globals["_READ_NAMEDTABLE"]._serialized_end = 5806 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_start = 5748 + _globals["_READ_NAMEDTABLE_OPTIONSENTRY"]._serialized_end = 5806 + _globals["_READ_DATASOURCE"]._serialized_start = 5809 + _globals["_READ_DATASOURCE"]._serialized_end = 6086 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_start = 5748 + _globals["_READ_DATASOURCE_OPTIONSENTRY"]._serialized_end = 5806 + _globals["_PROJECT"]._serialized_start = 6101 + _globals["_PROJECT"]._serialized_end = 6218 + _globals["_FILTER"]._serialized_start = 6220 + _globals["_FILTER"]._serialized_end = 6332 + _globals["_JOIN"]._serialized_start = 6335 + _globals["_JOIN"]._serialized_end = 6996 + _globals["_JOIN_JOINDATATYPE"]._serialized_start = 6674 + _globals["_JOIN_JOINDATATYPE"]._serialized_end = 6766 + _globals["_JOIN_JOINTYPE"]._serialized_start = 6769 + _globals["_JOIN_JOINTYPE"]._serialized_end = 6977 + _globals["_SETOPERATION"]._serialized_start = 6999 + _globals["_SETOPERATION"]._serialized_end = 7478 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_start = 7315 + _globals["_SETOPERATION_SETOPTYPE"]._serialized_end = 7429 + _globals["_LIMIT"]._serialized_start = 7480 + _globals["_LIMIT"]._serialized_end = 7556 + _globals["_OFFSET"]._serialized_start = 7558 + _globals["_OFFSET"]._serialized_end = 7637 + _globals["_TAIL"]._serialized_start = 7639 + _globals["_TAIL"]._serialized_end = 7714 + _globals["_AGGREGATE"]._serialized_start = 7717 + _globals["_AGGREGATE"]._serialized_end = 8483 + _globals["_AGGREGATE_PIVOT"]._serialized_start = 8132 + _globals["_AGGREGATE_PIVOT"]._serialized_end = 8243 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_start = 8245 + _globals["_AGGREGATE_GROUPINGSETS"]._serialized_end = 8321 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_start = 8324 + _globals["_AGGREGATE_GROUPTYPE"]._serialized_end = 8483 + _globals["_SORT"]._serialized_start = 8486 + _globals["_SORT"]._serialized_end = 8646 + _globals["_DROP"]._serialized_start = 8649 + _globals["_DROP"]._serialized_end = 8790 + _globals["_DEDUPLICATE"]._serialized_start = 8793 + _globals["_DEDUPLICATE"]._serialized_end = 9033 + _globals["_LOCALRELATION"]._serialized_start = 9035 + _globals["_LOCALRELATION"]._serialized_end = 9124 + _globals["_CACHEDLOCALRELATION"]._serialized_start = 9126 + _globals["_CACHEDLOCALRELATION"]._serialized_end = 9198 + _globals["_CACHEDREMOTERELATION"]._serialized_start = 9200 + _globals["_CACHEDREMOTERELATION"]._serialized_end = 9255 + _globals["_SAMPLE"]._serialized_start = 9258 + _globals["_SAMPLE"]._serialized_end = 9531 + _globals["_RANGE"]._serialized_start = 9534 + _globals["_RANGE"]._serialized_end = 9679 + _globals["_SUBQUERYALIAS"]._serialized_start = 9681 + _globals["_SUBQUERYALIAS"]._serialized_end = 9795 + _globals["_REPARTITION"]._serialized_start = 9798 + _globals["_REPARTITION"]._serialized_end = 9940 + _globals["_SHOWSTRING"]._serialized_start = 9943 + _globals["_SHOWSTRING"]._serialized_end = 10085 + _globals["_HTMLSTRING"]._serialized_start = 10087 + _globals["_HTMLSTRING"]._serialized_end = 10201 + _globals["_STATSUMMARY"]._serialized_start = 10203 + _globals["_STATSUMMARY"]._serialized_end = 10295 + _globals["_STATDESCRIBE"]._serialized_start = 10297 + _globals["_STATDESCRIBE"]._serialized_end = 10378 + _globals["_STATCROSSTAB"]._serialized_start = 10380 + _globals["_STATCROSSTAB"]._serialized_end = 10481 + _globals["_STATCOV"]._serialized_start = 10483 + _globals["_STATCOV"]._serialized_end = 10579 + _globals["_STATCORR"]._serialized_start = 10582 + _globals["_STATCORR"]._serialized_end = 10719 + _globals["_STATAPPROXQUANTILE"]._serialized_start = 10722 + _globals["_STATAPPROXQUANTILE"]._serialized_end = 10886 + _globals["_STATFREQITEMS"]._serialized_start = 10888 + _globals["_STATFREQITEMS"]._serialized_end = 11013 + _globals["_STATSAMPLEBY"]._serialized_start = 11016 + _globals["_STATSAMPLEBY"]._serialized_end = 11325 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_start = 11217 + _globals["_STATSAMPLEBY_FRACTION"]._serialized_end = 11316 + _globals["_NAFILL"]._serialized_start = 11328 + _globals["_NAFILL"]._serialized_end = 11462 + _globals["_NADROP"]._serialized_start = 11465 + _globals["_NADROP"]._serialized_end = 11599 + _globals["_NAREPLACE"]._serialized_start = 11602 + _globals["_NAREPLACE"]._serialized_end = 11898 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_start = 11757 + _globals["_NAREPLACE_REPLACEMENT"]._serialized_end = 11898 + _globals["_TODF"]._serialized_start = 11900 + _globals["_TODF"]._serialized_end = 11988 + _globals["_WITHCOLUMNSRENAMED"]._serialized_start = 11991 + _globals["_WITHCOLUMNSRENAMED"]._serialized_end = 12373 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_start = 12235 + _globals["_WITHCOLUMNSRENAMED_RENAMECOLUMNSMAPENTRY"]._serialized_end = 12302 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_start = 12304 + _globals["_WITHCOLUMNSRENAMED_RENAME"]._serialized_end = 12373 + _globals["_WITHCOLUMNS"]._serialized_start = 12375 + _globals["_WITHCOLUMNS"]._serialized_end = 12494 + _globals["_WITHWATERMARK"]._serialized_start = 12497 + _globals["_WITHWATERMARK"]._serialized_end = 12631 + _globals["_HINT"]._serialized_start = 12634 + _globals["_HINT"]._serialized_end = 12766 + _globals["_UNPIVOT"]._serialized_start = 12769 + _globals["_UNPIVOT"]._serialized_end = 13096 + _globals["_UNPIVOT_VALUES"]._serialized_start = 13026 + _globals["_UNPIVOT_VALUES"]._serialized_end = 13085 + _globals["_TRANSPOSE"]._serialized_start = 13098 + _globals["_TRANSPOSE"]._serialized_end = 13220 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_start = 13222 + _globals["_UNRESOLVEDTABLEVALUEDFUNCTION"]._serialized_end = 13347 + _globals["_TOSCHEMA"]._serialized_start = 13349 + _globals["_TOSCHEMA"]._serialized_end = 13455 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_start = 13458 + _globals["_REPARTITIONBYEXPRESSION"]._serialized_end = 13661 + _globals["_MAPPARTITIONS"]._serialized_start = 13664 + _globals["_MAPPARTITIONS"]._serialized_end = 13896 + _globals["_GROUPMAP"]._serialized_start = 13899 + _globals["_GROUPMAP"]._serialized_end = 14616 + _globals["_COGROUPMAP"]._serialized_start = 14619 + _globals["_COGROUPMAP"]._serialized_end = 15145 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_start = 15148 + _globals["_APPLYINPANDASWITHSTATE"]._serialized_end = 15505 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_start = 15508 + _globals["_COMMONINLINEUSERDEFINEDTABLEFUNCTION"]._serialized_end = 15752 + _globals["_PYTHONUDTF"]._serialized_start = 15755 + _globals["_PYTHONUDTF"]._serialized_end = 15932 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_start = 15935 + _globals["_COMMONINLINEUSERDEFINEDDATASOURCE"]._serialized_end = 16086 + _globals["_PYTHONDATASOURCE"]._serialized_start = 16088 + _globals["_PYTHONDATASOURCE"]._serialized_end = 16163 + _globals["_COLLECTMETRICS"]._serialized_start = 16166 + _globals["_COLLECTMETRICS"]._serialized_end = 16302 + _globals["_PARSE"]._serialized_start = 16305 + _globals["_PARSE"]._serialized_end = 16693 + _globals["_PARSE_OPTIONSENTRY"]._serialized_start = 5748 + _globals["_PARSE_OPTIONSENTRY"]._serialized_end = 5806 + _globals["_PARSE_PARSEFORMAT"]._serialized_start = 16594 + _globals["_PARSE_PARSEFORMAT"]._serialized_end = 16682 + _globals["_ASOFJOIN"]._serialized_start = 16696 + _globals["_ASOFJOIN"]._serialized_end = 17171 + _globals["_LATERALJOIN"]._serialized_start = 17174 + _globals["_LATERALJOIN"]._serialized_end = 17404 # @@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 0c8cf8dd3eda8..d4f1233e45f9b 100644 --- a/python/pyspark/sql/connect/proto/relations_pb2.pyi +++ b/python/pyspark/sql/connect/proto/relations_pb2.pyi @@ -756,13 +756,13 @@ class Fetch(google.protobuf.message.Message): PARAM_FIELD_NUMBER: builtins.int INPUT_FIELD_NUMBER: builtins.int @property - def param(self) -> pyspark.sql.connect.proto.ml_common_pb2.Param: ... + def param(self) -> pyspark.sql.connect.proto.expressions_pb2.Expression.Literal: ... @property def input(self) -> global___Relation: ... def __init__( self, *, - param: pyspark.sql.connect.proto.ml_common_pb2.Param | None = ..., + param: pyspark.sql.connect.proto.expressions_pb2.Expression.Literal | None = ..., input: global___Relation | None = ..., ) -> None: ... def HasField( diff --git a/sql/connect/common/src/main/protobuf/spark/connect/common.proto b/sql/connect/common/src/main/protobuf/spark/connect/common.proto index b2848370b01dc..dfa30d151f88e 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/common.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/common.proto @@ -96,3 +96,27 @@ message PythonOrigin { // (Required) Callsite to show to end users, for example, stacktrace. string call_site = 2; } + +message Bools { + repeated bool values = 1; +} + +message Ints { + repeated int32 values = 1; +} + +message Longs { + repeated int64 values = 1; +} + +message Floats { + repeated float values = 1; +} + +message Doubles { + repeated double values = 1; +} + +message Strings { + repeated string values = 1; +} 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 bbe605a47f4ff..a5fadd913622c 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/expressions.proto @@ -194,6 +194,8 @@ message Expression { Array array = 22; Map map = 23; Struct struct = 24; + + SpecializedArray specialized_array = 25; } message Decimal { @@ -228,6 +230,17 @@ message Expression { DataType struct_type = 1; repeated Literal elements = 2; } + + message SpecializedArray { + oneof value_type { + Bools bools = 1; + Ints ints = 2; + Longs longs = 3; + Floats floats = 4; + Doubles doubles = 5; + Strings strings = 6; + } + } } // An unresolved attribute that is not explicitly bound to a specific column, but the column diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto index 48b04a6e14cd0..3198bbbe9c115 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/ml.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml.proto @@ -20,6 +20,7 @@ syntax = 'proto3'; package spark.connect; import "spark/connect/relations.proto"; +import "spark/connect/expressions.proto"; import "spark/connect/ml_common.proto"; option java_multiple_files = true; @@ -84,7 +85,7 @@ message MlCommand { message MlCommandResult { oneof result_type { // The result of the attribute - Param param = 1; + Expression.Literal param = 1; // Evaluate a Dataset in a model and return the cached ID of summary string summary = 2; // Operator information diff --git a/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto index f91c2489ed947..48b5fa8135cc9 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/ml_common.proto @@ -28,17 +28,7 @@ option go_package = "internal/generated"; // MlParams stores param settings for ML Estimator / Transformer / Evaluator message MlParams { // User-supplied params - map params = 1; -} - -// Represents the parameter type of the ML instance, or the returned value -// of the attribute -message Param { - oneof param_type { - Expression.Literal literal = 1; - Vector vector = 2; - Matrix matrix = 3; - } + map params = 1; } // MLOperator represents the ML operators like (Estimator, Transformer or Evaluator) @@ -64,45 +54,3 @@ message ObjectRef { // The ID is used to lookup the object on the server side. string id = 1; } - -// See pyspark.ml.linalg.Vector -message Vector { - oneof vector_type { - Dense dense = 1; - Sparse sparse = 2; - } - // See pyspark.ml.linalg.DenseVector - message Dense { - repeated double value = 1; - } - // See pyspark.ml.linalg.SparseVector - message Sparse { - int32 size = 1; - repeated int32 index = 2; - repeated double value = 3; - } -} - -// See pyspark.ml.linalg.Matrix -message Matrix { - oneof matrix_type { - Dense dense = 1; - Sparse sparse = 2; - } - // See pyspark.ml.linalg.DenseMatrix - message Dense { - int32 num_rows = 1; - int32 num_cols = 2; - repeated double value = 3; - bool is_transposed = 4; - } - // See pyspark.ml.linalg.SparseMatrix - message Sparse { - int32 num_rows = 1; - int32 num_cols = 2; - repeated int32 colptr = 3; - repeated int32 row_index = 4; - repeated double value = 5; - bool is_transposed = 6; - } -} 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 c2cbed0dd22ba..58ce057859787 100644 --- a/sql/connect/common/src/main/protobuf/spark/connect/relations.proto +++ b/sql/connect/common/src/main/protobuf/spark/connect/relations.proto @@ -151,7 +151,7 @@ message Fetch { message Args { oneof args_type { - Param param = 1; + Expression.Literal param = 1; Relation input = 2; } } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala index 1f0981e048afb..e53276dcf7df9 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLHandler.scala @@ -153,11 +153,7 @@ private[connect] object MLHandler extends Logging { } proto.MlCommandResult .newBuilder() - .setParam( - proto.Param - .newBuilder() - .setLiteral(LiteralValueProtoConverter.toLiteralProto(result)) - .build()) + .setParam(LiteralValueProtoConverter.toLiteralProto(result)) .build() case proto.MlCommand.CommandCase.WRITE => diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala index 2c006d86dfac8..4dbfb063eabbc 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/MLUtils.scala @@ -26,11 +26,11 @@ import org.apache.commons.lang3.reflect.MethodUtils.invokeMethod import org.apache.spark.connect.proto import org.apache.spark.ml.{Estimator, Transformer} -import org.apache.spark.ml.linalg.{Matrices, Matrix, Vector, Vectors} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.Params import org.apache.spark.ml.util.{MLReadable, MLWritable} import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.connect.plugin.SparkConnectPluginRegistry import org.apache.spark.sql.connect.service.SessionHolder @@ -53,28 +53,65 @@ private[ml] object MLUtils { providers.map(est => est.getClass.getName -> est.getClass).toMap } - def deserializeVector(vector: proto.Vector): Vector = { - if (vector.hasDense) { - val values = vector.getDense.getValueList.asScala.map(_.toDouble).toArray - Vectors.dense(values) - } else { - val size = vector.getSparse.getSize - val indices = vector.getSparse.getIndexList.asScala.map(_.toInt).toArray - val values = vector.getSparse.getValueList.asScala.map(_.toDouble).toArray - Vectors.sparse(size, indices, values) + private def parseInts(ints: proto.Ints): Array[Int] = { + val size = ints.getValuesCount + val values = Array.ofDim[Int](size) + var i = 0 + while (i < size) { + values(i) = ints.getValues(i) + i += 1 } + values } - def deserializeMatrix(matrix: proto.Matrix): Matrix = { - if (matrix.hasDense) { - val values = matrix.getDense.getValueList.asScala.map(_.toDouble).toArray - Matrices.dense(matrix.getDense.getNumRows, matrix.getDense.getNumCols, values) - } else { - val sparse = matrix.getSparse - val colPtrs = sparse.getColptrList.asScala.map(_.toInt).toArray - val rowIndices = sparse.getRowIndexList.asScala.map(_.toInt).toArray - val values = sparse.getValueList.asScala.map(_.toDouble).toArray - Matrices.sparse(sparse.getNumRows, sparse.getNumCols, colPtrs, rowIndices, values) + private def parseDoubles(doubles: proto.Doubles): Array[Double] = { + val size = doubles.getValuesCount + val values = Array.ofDim[Double](size) + var i = 0 + while (i < size) { + values(i) = doubles.getValues(i) + i += 1 + } + values + } + + def deserializeVector(s: proto.Expression.Literal.Struct): Vector = { + assert(s.getElementsCount == 4) + s.getElements(0).getByte match { + case 0 => + val size = s.getElements(1).getInteger + val indices = parseInts(s.getElements(2).getSpecializedArray.getInts) + val values = parseDoubles(s.getElements(3).getSpecializedArray.getDoubles) + Vectors.sparse(size, indices, values) + + case 1 => + val values = parseDoubles(s.getElements(3).getSpecializedArray.getDoubles) + Vectors.dense(values) + + case o => throw MlUnsupportedException(s"Unknown Vector type $o") + } + } + + def deserializeMatrix(s: proto.Expression.Literal.Struct): Matrix = { + assert(s.getElementsCount == 7) + s.getElements(0).getByte match { + case 0 => + val numRows = s.getElements(1).getInteger + val numCols = s.getElements(2).getInteger + val colPtrs = parseInts(s.getElements(3).getSpecializedArray.getInts) + val rowIndices = parseInts(s.getElements(4).getSpecializedArray.getInts) + val values = parseDoubles(s.getElements(5).getSpecializedArray.getDoubles) + val isTransposed = s.getElements(6).getBoolean + new SparseMatrix(numRows, numCols, colPtrs, rowIndices, values, isTransposed) + + case 1 => + val numRows = s.getElements(1).getInteger + val numCols = s.getElements(2).getInteger + val values = parseDoubles(s.getElements(5).getSpecializedArray.getDoubles) + val isTransposed = s.getElements(6).getBoolean + new DenseMatrix(numRows, numCols, values, isTransposed) + + case o => throw MlUnsupportedException(s"Unknown Matrix type $o") } } @@ -87,18 +124,24 @@ private[ml] object MLUtils { * the parameters of the ML operator */ def setInstanceParams(instance: Params, params: proto.MlParams): Unit = { - params.getParamsMap.asScala.foreach { case (name, paramProto) => + params.getParamsMap.asScala.foreach { case (name, literal) => val p = instance.getParam(name) - val value = if (paramProto.hasLiteral) { - reconcileParam( - p.paramValueClassTag.runtimeClass, - LiteralValueProtoConverter.toCatalystValue(paramProto.getLiteral)) - } else if (paramProto.hasVector) { - deserializeVector(paramProto.getVector) - } else if (paramProto.hasMatrix) { - deserializeMatrix(paramProto.getMatrix) - } else { - throw MlUnsupportedException(s"Unsupported parameter type for ${name}") + val value = literal.getLiteralTypeCase match { + case proto.Expression.Literal.LiteralTypeCase.STRUCT => + val s = literal.getStruct + val schema = DataTypeProtoConverter.toCatalystType(s.getStructType) + if (schema == VectorUDT.sqlType) { + deserializeVector(s) + } else if (schema == MatrixUDT.sqlType) { + deserializeMatrix(s) + } else { + throw MlUnsupportedException(s"Unsupported parameter struct ${schema} for ${name}") + } + + case _ => + reconcileParam( + p.paramValueClassTag.runtimeClass, + LiteralValueProtoConverter.toCatalystValue(literal)) } instance.set(p, value) } diff --git a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala index ad6735997f834..ee0812a1a98ca 100644 --- a/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala +++ b/sql/connect/server/src/main/scala/org/apache/spark/sql/connect/ml/Serializer.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.connect.ml import org.apache.spark.connect.proto -import org.apache.spark.ml.linalg.{DenseMatrix, DenseVector, SparseMatrix, SparseVector} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.Params import org.apache.spark.sql.Dataset -import org.apache.spark.sql.connect.common.LiteralValueProtoConverter +import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, LiteralValueProtoConverter, ProtoDataTypes} import org.apache.spark.sql.connect.service.SessionHolder private[ml] object Serializer { @@ -31,97 +31,141 @@ private[ml] object Serializer { * @param data * the value of parameter * @return - * proto.Param + * proto.Expression.Literal */ - def serializeParam(data: Any): proto.Param = { + def serializeParam(data: Any): proto.Expression.Literal = { data match { - case v: DenseVector => - val denseBuilder = proto.Vector.Dense.newBuilder() - v.values.foreach(denseBuilder.addValue) - proto.Param - .newBuilder() - .setVector(proto.Vector.newBuilder().setDense(denseBuilder)) - .build() case v: SparseVector => - val sparseBuilder = proto.Vector.Sparse.newBuilder().setSize(v.size) - v.indices.foreach(sparseBuilder.addIndex) - v.values.foreach(sparseBuilder.addValue) - proto.Param - .newBuilder() - .setVector(proto.Vector.newBuilder().setSparse(sparseBuilder)) - .build() - case v: DenseMatrix => - val denseBuilder = proto.Matrix.Dense.newBuilder() - v.values.foreach(denseBuilder.addValue) - denseBuilder.setNumCols(v.numCols) - denseBuilder.setNumRows(v.numRows) - denseBuilder.setIsTransposed(v.isTransposed) - proto.Param - .newBuilder() - .setMatrix(proto.Matrix.newBuilder().setDense(denseBuilder)) - .build() - case v: SparseMatrix => - val sparseBuilder = proto.Matrix.Sparse - .newBuilder() - .setNumCols(v.numCols) - .setNumRows(v.numRows) - v.values.foreach(sparseBuilder.addValue) - v.colPtrs.foreach(sparseBuilder.addColptr) - v.rowIndices.foreach(sparseBuilder.addRowIndex) - proto.Param - .newBuilder() - .setMatrix(proto.Matrix.newBuilder().setSparse(sparseBuilder)) - .build() + val builder = proto.Expression.Literal.Struct.newBuilder() + builder.setStructType(DataTypeProtoConverter.toConnectProtoType(VectorUDT.sqlType)) + // type = 0 + builder.addElements(proto.Expression.Literal.newBuilder().setByte(0)) + // size + builder.addElements(proto.Expression.Literal.newBuilder().setInteger(v.size)) + // indices + builder.addElements(buildIntArray(v.indices)) + // values + builder.addElements(buildDoubleArray(v.values)) + proto.Expression.Literal.newBuilder().setStruct(builder).build() + + case v: DenseVector => + val builder = proto.Expression.Literal.Struct.newBuilder() + builder.setStructType(DataTypeProtoConverter.toConnectProtoType(VectorUDT.sqlType)) + // type = 1 + builder.addElements(proto.Expression.Literal.newBuilder().setByte(1)) + // size = null + builder.addElements( + proto.Expression.Literal.newBuilder().setNull(ProtoDataTypes.NullType)) + // indices = null + builder.addElements( + proto.Expression.Literal.newBuilder().setNull(ProtoDataTypes.NullType)) + // values + builder.addElements(buildDoubleArray(v.values)) + proto.Expression.Literal.newBuilder().setStruct(builder).build() + + case m: SparseMatrix => + val builder = proto.Expression.Literal.Struct.newBuilder() + builder.setStructType(DataTypeProtoConverter.toConnectProtoType(MatrixUDT.sqlType)) + // type = 0 + builder.addElements(proto.Expression.Literal.newBuilder().setByte(0)) + // numRows + builder.addElements(proto.Expression.Literal.newBuilder().setInteger(m.numRows)) + // numCols + builder.addElements(proto.Expression.Literal.newBuilder().setInteger(m.numCols)) + // colPtrs + builder.addElements(buildIntArray(m.colPtrs)) + // rowIndices + builder.addElements(buildIntArray(m.rowIndices)) + // values + builder.addElements(buildDoubleArray(m.values)) + // isTransposed + builder.addElements(proto.Expression.Literal.newBuilder().setBoolean(m.isTransposed)) + proto.Expression.Literal.newBuilder().setStruct(builder).build() + + case m: DenseMatrix => + val builder = proto.Expression.Literal.Struct.newBuilder() + builder.setStructType(DataTypeProtoConverter.toConnectProtoType(MatrixUDT.sqlType)) + // type = 1 + builder.addElements(proto.Expression.Literal.newBuilder().setByte(1)) + // numRows + builder.addElements(proto.Expression.Literal.newBuilder().setInteger(m.numRows)) + // numCols + builder.addElements(proto.Expression.Literal.newBuilder().setInteger(m.numCols)) + // colPtrs = null + builder.addElements( + proto.Expression.Literal.newBuilder().setNull(ProtoDataTypes.NullType)) + // rowIndices = null + builder.addElements( + proto.Expression.Literal.newBuilder().setNull(ProtoDataTypes.NullType)) + // values + builder.addElements(buildDoubleArray(m.values)) + // isTransposed + builder.addElements(proto.Expression.Literal.newBuilder().setBoolean(m.isTransposed)) + proto.Expression.Literal.newBuilder().setStruct(builder).build() + case _: Byte | _: Short | _: Int | _: Long | _: Float | _: Double | _: Boolean | _: String | _: Array[_] => - proto.Param - .newBuilder() - .setLiteral(LiteralValueProtoConverter.toLiteralProto(data)) - .build() + LiteralValueProtoConverter.toLiteralProto(data) case other => throw MlUnsupportedException(s"$other not supported") } } + private def buildIntArray(values: Array[Int]): proto.Expression.Literal = { + val builder = proto.Ints.newBuilder() + values.foreach(builder.addValues) + proto.Expression.Literal + .newBuilder() + .setSpecializedArray( + proto.Expression.Literal.SpecializedArray + .newBuilder() + .setInts(builder) + .build()) + .build() + } + + private def buildDoubleArray(values: Array[Double]): proto.Expression.Literal = { + val builder = proto.Doubles.newBuilder() + values.foreach(builder.addValues) + proto.Expression.Literal + .newBuilder() + .setSpecializedArray( + proto.Expression.Literal.SpecializedArray + .newBuilder() + .setDoubles(builder) + .build()) + .build() + } + def deserializeMethodArguments( args: Array[proto.Fetch.Method.Args], sessionHolder: SessionHolder): Array[(Object, Class[_])] = { args.map { arg => if (arg.hasParam) { - val param = arg.getParam - if (param.hasLiteral) { - param.getLiteral.getLiteralTypeCase match { - case proto.Expression.Literal.LiteralTypeCase.INTEGER => - (param.getLiteral.getInteger.asInstanceOf[Object], classOf[Int]) - case proto.Expression.Literal.LiteralTypeCase.FLOAT => - (param.getLiteral.getFloat.toDouble.asInstanceOf[Object], classOf[Double]) - case proto.Expression.Literal.LiteralTypeCase.STRING => - (param.getLiteral.getString, classOf[String]) - case proto.Expression.Literal.LiteralTypeCase.DOUBLE => - (param.getLiteral.getDouble.asInstanceOf[Object], classOf[Double]) - case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => - (param.getLiteral.getBoolean.asInstanceOf[Object], classOf[Boolean]) - case other => - throw MlUnsupportedException(s"$other not supported") - } - } else if (param.hasVector) { - val vector = MLUtils.deserializeVector(param.getVector) - val vectorType = if (param.getVector.hasDense) { - classOf[DenseVector] - } else { - classOf[SparseVector] - } - (vector, vectorType) - } else if (param.hasMatrix) { - val matrix = MLUtils.deserializeMatrix(param.getMatrix) - val matrixType = if (param.getMatrix.hasDense) { - classOf[DenseMatrix] - } else { - classOf[SparseMatrix] - } - (matrix, matrixType) - } else { - throw MlUnsupportedException(s"$param not supported") + val literal = arg.getParam + literal.getLiteralTypeCase match { + case proto.Expression.Literal.LiteralTypeCase.STRUCT => + val struct = literal.getStruct + val schema = DataTypeProtoConverter.toCatalystType(struct.getStructType) + if (schema == VectorUDT.sqlType) { + (MLUtils.deserializeVector(struct), classOf[Vector]) + } else if (schema == MatrixUDT.sqlType) { + (MLUtils.deserializeMatrix(struct), classOf[Matrix]) + } else { + throw MlUnsupportedException(s"$schema not supported") + } + case proto.Expression.Literal.LiteralTypeCase.INTEGER => + (literal.getInteger.asInstanceOf[Object], classOf[Int]) + case proto.Expression.Literal.LiteralTypeCase.FLOAT => + (literal.getFloat.toDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.STRING => + (literal.getString, classOf[String]) + case proto.Expression.Literal.LiteralTypeCase.DOUBLE => + (literal.getDouble.asInstanceOf[Object], classOf[Double]) + case proto.Expression.Literal.LiteralTypeCase.BOOLEAN => + (literal.getBoolean.asInstanceOf[Object], classOf[Boolean]) + case other => + throw MlUnsupportedException(s"$other not supported") } } else if (arg.hasInput) { (MLUtils.parseRelationProto(arg.getInput, sessionHolder), classOf[Dataset[_]]) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLBackendSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLBackendSuite.scala index 2e1d3d95c486c..f4db4077d1f3c 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLBackendSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLBackendSuite.scala @@ -51,12 +51,9 @@ class MLBackendSuite extends MLHelper { .newBuilder() .putParams( "maxIter", - proto.Param + proto.Expression.Literal .newBuilder() - .setLiteral( - proto.Expression.Literal - .newBuilder() - .setInteger(2)) + .setInteger(2) .build()) } @@ -95,10 +92,10 @@ class MLBackendSuite extends MLHelper { assert(ret.getOperatorInfo.getParams.getParamsMap.containsKey("fakeParam")) assert(ret.getOperatorInfo.getParams.getParamsMap.containsKey("maxIter")) assert( - ret.getOperatorInfo.getParams.getParamsMap.get("maxIter").getLiteral.getInteger + ret.getOperatorInfo.getParams.getParamsMap.get("maxIter").getInteger == 2) assert( - ret.getOperatorInfo.getParams.getParamsMap.get("fakeParam").getLiteral.getInteger + ret.getOperatorInfo.getParams.getParamsMap.get("fakeParam").getInteger == 101010) } finally { Utils.deleteRecursively(tempDir) @@ -157,10 +154,10 @@ class MLBackendSuite extends MLHelper { assert(ret.getOperatorInfo.getParams.getParamsMap.containsKey("fakeParam")) assert(ret.getOperatorInfo.getParams.getParamsMap.containsKey("maxIter")) assert( - ret.getOperatorInfo.getParams.getParamsMap.get("maxIter").getLiteral.getInteger + ret.getOperatorInfo.getParams.getParamsMap.get("maxIter").getInteger == 2) assert( - ret.getOperatorInfo.getParams.getParamsMap.get("fakeParam").getLiteral.getInteger + ret.getOperatorInfo.getParams.getParamsMap.get("fakeParam").getInteger == 101010) } finally { Utils.deleteRecursively(tempDir) diff --git a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala index 280ced32a3f8c..b537f1c702bdf 100644 --- a/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala +++ b/sql/connect/server/src/test/scala/org/apache/spark/sql/connect/ml/MLSuite.scala @@ -72,85 +72,53 @@ class MLSuite extends MLHelper { val fakedML = new FakedML val params = proto.MlParams .newBuilder() - .putParams( - "boolean", - proto.Param - .newBuilder() - .setLiteral(proto.Expression.Literal.newBuilder().setBoolean(true)) - .build()) - .putParams( - "double", - proto.Param - .newBuilder() - .setLiteral(proto.Expression.Literal.newBuilder().setDouble(1.0)) - .build()) - .putParams( - "int", - proto.Param - .newBuilder() - .setLiteral(proto.Expression.Literal.newBuilder().setInteger(10)) - .build()) - .putParams( - "float", - proto.Param - .newBuilder() - .setLiteral(proto.Expression.Literal.newBuilder().setFloat(10.0f)) - .build()) + .putParams("boolean", proto.Expression.Literal.newBuilder().setBoolean(true).build()) + .putParams("double", proto.Expression.Literal.newBuilder().setDouble(1.0).build()) + .putParams("int", proto.Expression.Literal.newBuilder().setInteger(10).build()) + .putParams("float", proto.Expression.Literal.newBuilder().setFloat(10.0f).build()) .putParams( "arrayString", - proto.Param + proto.Expression.Literal .newBuilder() - .setLiteral( - proto.Expression.Literal + .setArray( + proto.Expression.Literal.Array .newBuilder() - .setArray( - proto.Expression.Literal.Array - .newBuilder() - .setElementType(proto.DataType - .newBuilder() - .setString(proto.DataType.String.getDefaultInstance) - .build()) - .addElements(proto.Expression.Literal.newBuilder().setString("hello")) - .addElements(proto.Expression.Literal.newBuilder().setString("world")) - .build()) + .setElementType(proto.DataType + .newBuilder() + .setString(proto.DataType.String.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setString("hello")) + .addElements(proto.Expression.Literal.newBuilder().setString("world")) .build()) .build()) .putParams( "arrayInt", - proto.Param + proto.Expression.Literal .newBuilder() - .setLiteral( - proto.Expression.Literal + .setArray( + proto.Expression.Literal.Array .newBuilder() - .setArray( - proto.Expression.Literal.Array - .newBuilder() - .setElementType(proto.DataType - .newBuilder() - .setInteger(proto.DataType.Integer.getDefaultInstance) - .build()) - .addElements(proto.Expression.Literal.newBuilder().setInteger(1)) - .addElements(proto.Expression.Literal.newBuilder().setInteger(2)) - .build()) + .setElementType(proto.DataType + .newBuilder() + .setInteger(proto.DataType.Integer.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setInteger(1)) + .addElements(proto.Expression.Literal.newBuilder().setInteger(2)) .build()) .build()) .putParams( "arrayDouble", - proto.Param + proto.Expression.Literal .newBuilder() - .setLiteral( - proto.Expression.Literal + .setArray( + proto.Expression.Literal.Array .newBuilder() - .setArray( - proto.Expression.Literal.Array - .newBuilder() - .setElementType(proto.DataType - .newBuilder() - .setDouble(proto.DataType.Double.getDefaultInstance) - .build()) - .addElements(proto.Expression.Literal.newBuilder().setDouble(11.0)) - .addElements(proto.Expression.Literal.newBuilder().setDouble(12.0)) - .build()) + .setElementType(proto.DataType + .newBuilder() + .setDouble(proto.DataType.Double.getDefaultInstance) + .build()) + .addElements(proto.Expression.Literal.newBuilder().setDouble(11.0)) + .addElements(proto.Expression.Literal.newBuilder().setDouble(12.0)) .build()) .build()) .build() @@ -193,11 +161,9 @@ class MLSuite extends MLHelper { .newBuilder() .putParams( "maxIter", - proto.Param + proto.Expression.Literal .newBuilder() - .setLiteral(proto.Expression.Literal - .newBuilder() - .setInteger(2)) + .setInteger(2) .build()))) .build() val fitResult = MLHandler.handleMlCommand(sessionHolder, fitCommand) @@ -218,13 +184,13 @@ class MLSuite extends MLHelper { // Fetch double attribute val interceptCommand = fetchCommand(modelId, "intercept") val interceptResult = MLHandler.handleMlCommand(sessionHolder, interceptCommand) - assert(interceptResult.getParam.getLiteral.getDouble === lrModel.intercept) + assert(interceptResult.getParam.getDouble === lrModel.intercept) // Fetch Vector attribute val coefficientsCommand = fetchCommand(modelId, "coefficients") val coefficientsResult = MLHandler.handleMlCommand(sessionHolder, coefficientsCommand) val deserializedCoefficients = - MLUtils.deserializeVector(coefficientsResult.getParam.getVector) + MLUtils.deserializeVector(coefficientsResult.getParam.getStruct) assert(deserializedCoefficients === lrModel.coefficients) // Fetch Matrix attribute @@ -232,7 +198,7 @@ class MLSuite extends MLHelper { val coefficientsMatrixResult = MLHandler.handleMlCommand(sessionHolder, coefficientsMatrixCommand) val deserializedCoefficientsMatrix = - MLUtils.deserializeMatrix(coefficientsMatrixResult.getParam.getMatrix) + MLUtils.deserializeMatrix(coefficientsMatrixResult.getParam.getStruct) assert(lrModel.coefficientMatrix === deserializedCoefficientsMatrix) // Predict with sparse vector @@ -252,7 +218,7 @@ class MLSuite extends MLHelper { .setParam(Serializer.serializeParam(sparseVector))))) .build() val predictResult = MLHandler.handleMlCommand(sessionHolder, predictCommand) - val predictValue = predictResult.getParam.getLiteral.getDouble + val predictValue = predictResult.getParam.getDouble assert(lrModel.predict(sparseVector) === predictValue) // The loaded model doesn't have summary @@ -268,7 +234,7 @@ class MLSuite extends MLHelper { .addMethods(proto.Fetch.Method.newBuilder().setMethod("accuracy"))) .build() val accuracyResult = MLHandler.handleMlCommand(sessionHolder, accuracyCommand) - assert(lrModel.summary.accuracy === accuracyResult.getParam.getLiteral.getDouble) + assert(lrModel.summary.accuracy === accuracyResult.getParam.getDouble) val weightedFMeasureCommand = proto.MlCommand .newBuilder() @@ -289,7 +255,7 @@ class MLSuite extends MLHelper { MLHandler.handleMlCommand(sessionHolder, weightedFMeasureCommand) assert( lrModel.summary.weightedFMeasure(2.5) === - weightedFMeasureResult.getParam.getLiteral.getDouble) + weightedFMeasureResult.getParam.getDouble) } }