diff --git a/connector/connect/src/main/protobuf/spark/connect/types.proto b/connector/connect/src/main/protobuf/spark/connect/types.proto index 98b0c48b1e016..ad043d8594700 100644 --- a/connector/connect/src/main/protobuf/spark/connect/types.proto +++ b/connector/connect/src/main/protobuf/spark/connect/types.proto @@ -55,134 +55,114 @@ message DataType { uint32 user_defined_type_reference = 31; } - enum Nullability { - NULLABILITY_UNSPECIFIED = 0; - NULLABILITY_NULLABLE = 1; - NULLABILITY_REQUIRED = 2; - } - message Boolean { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message I8 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message I16 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message I32 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message I64 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message FP32 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message FP64 { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message String { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message Binary { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message Timestamp { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message Date { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message Time { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message TimestampTZ { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message IntervalYear { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message IntervalDay { uint32 type_variation_reference = 1; - Nullability nullability = 2; } message UUID { uint32 type_variation_reference = 1; - Nullability nullability = 2; } // Start compound types. message FixedChar { int32 length = 1; uint32 type_variation_reference = 2; - Nullability nullability = 3; } message VarChar { int32 length = 1; uint32 type_variation_reference = 2; - Nullability nullability = 3; } message FixedBinary { int32 length = 1; uint32 type_variation_reference = 2; - Nullability nullability = 3; } message Decimal { int32 scale = 1; int32 precision = 2; uint32 type_variation_reference = 3; - Nullability nullability = 4; + } + + message StructField { + DataType type = 1; + string name = 2; + bool nullable = 3; + map metadata = 4; } message Struct { - repeated DataType types = 1; + repeated StructField fields = 1; uint32 type_variation_reference = 2; - Nullability nullability = 3; } message List { DataType DataType = 1; uint32 type_variation_reference = 2; - Nullability nullability = 3; + bool element_nullable = 3; } message Map { DataType key = 1; DataType value = 2; uint32 type_variation_reference = 3; - Nullability nullability = 4; + bool value_nullable = 4; } } diff --git a/connector/connect/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala b/connector/connect/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala index f6553f7e90b64..cb5e0cb46eaff 100644 --- a/connector/connect/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala +++ b/connector/connect/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala @@ -44,6 +44,32 @@ package object dsl { .addAllParts(identifier.asJava) .build()) .build() + + def struct( + attrs: proto.Expression.QualifiedAttribute*): proto.Expression.QualifiedAttribute = { + val structExpr = proto.DataType.Struct.newBuilder() + for (attr <- attrs) { + val structField = proto.DataType.StructField.newBuilder() + structField.setName(attr.getName) + structField.setType(attr.getType) + structExpr.addFields(structField) + } + proto.Expression.QualifiedAttribute.newBuilder() + .setName(s) + .setType(proto.DataType.newBuilder().setStruct(structExpr)) + .build() + } + + /** Creates a new AttributeReference of type int */ + def int: proto.Expression.QualifiedAttribute = protoQualifiedAttrWithType( + proto.DataType.newBuilder().setI32(proto.DataType.I32.newBuilder()).build()) + + private def protoQualifiedAttrWithType( + dataType: proto.DataType): proto.Expression.QualifiedAttribute = + proto.Expression.QualifiedAttribute.newBuilder() + .setName(s) + .setType(dataType) + .build() } implicit class DslExpression(val expr: proto.Expression) { diff --git a/connector/connect/src/main/scala/org/apache/spark/sql/connect/planner/DataTypeProtoConverter.scala b/connector/connect/src/main/scala/org/apache/spark/sql/connect/planner/DataTypeProtoConverter.scala index a0a5ea82d14ce..da3adce43ba98 100644 --- a/connector/connect/src/main/scala/org/apache/spark/sql/connect/planner/DataTypeProtoConverter.scala +++ b/connector/connect/src/main/scala/org/apache/spark/sql/connect/planner/DataTypeProtoConverter.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.connect.planner +import scala.collection.convert.ImplicitConversions._ + import org.apache.spark.connect.proto import org.apache.spark.sql.SaveMode -import org.apache.spark.sql.types.{DataType, IntegerType, StringType} +import org.apache.spark.sql.types.{DataType, IntegerType, StringType, StructField, StructType} /** * This object offers methods to convert to/from connect proto to catalyst types. @@ -29,11 +31,19 @@ object DataTypeProtoConverter { t.getKindCase match { case proto.DataType.KindCase.I32 => IntegerType case proto.DataType.KindCase.STRING => StringType + case proto.DataType.KindCase.STRUCT => convertProtoDataTypeToCatalyst(t.getStruct) case _ => throw InvalidPlanInput(s"Does not support convert ${t.getKindCase} to catalyst types.") } } + private def convertProtoDataTypeToCatalyst(t: proto.DataType.Struct): StructType = { + // TODO: handle nullability + val structFields = + t.getFieldsList.map(f => StructField(f.getName, toCatalystType(f.getType))).toList + StructType.apply(structFields) + } + def toConnectProtoType(t: DataType): proto.DataType = { t match { case IntegerType => diff --git a/connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala b/connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala index 7395307903ebd..d3f286d848a67 100644 --- a/connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala +++ b/connector/connect/src/test/scala/org/apache/spark/sql/connect/planner/SparkConnectProtoSuite.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.sql.connect.planner +import org.apache.spark.connect.proto import org.apache.spark.connect.proto.Join.JoinType import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -114,7 +115,26 @@ class SparkConnectProtoSuite extends PlanTest with SparkConnectPlanTest { import org.apache.spark.sql.connect.dsl.plans._ transform(connectTestRelation.as("target_table")) } + val sparkPlan = sparkTestRelation.as("target_table") comparePlans(connectPlan.analyze, sparkPlan.analyze, false) } + + test("Test StructType in LocalRelation") { + val connectPlan = { + import org.apache.spark.sql.connect.dsl.expressions._ + transform(createLocalRelationProtoByQualifiedAttributes(Seq("a".struct("id".int)))) + } + val sparkPlan = LocalRelation($"a".struct($"id".int)) + comparePlans(connectPlan.analyze, sparkPlan.analyze, false) + } + + private def createLocalRelationProtoByQualifiedAttributes( + attrs: Seq[proto.Expression.QualifiedAttribute]): proto.Relation = { + val localRelationBuilder = proto.LocalRelation.newBuilder() + for (attr <- attrs) { + localRelationBuilder.addAttributes(attr) + } + proto.Relation.newBuilder().setLocalRelation(localRelationBuilder.build()).build() + } } diff --git a/python/pyspark/sql/connect/proto/types_pb2.py b/python/pyspark/sql/connect/proto/types_pb2.py index dedc6b4b5785b..e904443714289 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.py +++ b/python/pyspark/sql/connect/proto/types_pb2.py @@ -29,7 +29,7 @@ DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile( - b'\n\x19spark/connect/types.proto\x12\rspark.connect"\xbe\'\n\x08\x44\x61taType\x12\x35\n\x04\x62ool\x18\x01 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x04\x62ool\x12,\n\x02i8\x18\x02 \x01(\x0b\x32\x1a.spark.connect.DataType.I8H\x00R\x02i8\x12/\n\x03i16\x18\x03 \x01(\x0b\x32\x1b.spark.connect.DataType.I16H\x00R\x03i16\x12/\n\x03i32\x18\x05 \x01(\x0b\x32\x1b.spark.connect.DataType.I32H\x00R\x03i32\x12/\n\x03i64\x18\x07 \x01(\x0b\x32\x1b.spark.connect.DataType.I64H\x00R\x03i64\x12\x32\n\x04\x66p32\x18\n \x01(\x0b\x32\x1c.spark.connect.DataType.FP32H\x00R\x04\x66p32\x12\x32\n\x04\x66p64\x18\x0b \x01(\x0b\x32\x1c.spark.connect.DataType.FP64H\x00R\x04\x66p64\x12\x38\n\x06string\x18\x0c \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x38\n\x06\x62inary\x18\r \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12\x41\n\ttimestamp\x18\x0e \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12\x32\n\x04\x64\x61te\x18\x10 \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x32\n\x04time\x18\x11 \x01(\x0b\x32\x1c.spark.connect.DataType.TimeH\x00R\x04time\x12K\n\rinterval_year\x18\x13 \x01(\x0b\x32$.spark.connect.DataType.IntervalYearH\x00R\x0cintervalYear\x12H\n\x0cinterval_day\x18\x14 \x01(\x0b\x32#.spark.connect.DataType.IntervalDayH\x00R\x0bintervalDay\x12H\n\x0ctimestamp_tz\x18\x1d \x01(\x0b\x32#.spark.connect.DataType.TimestampTZH\x00R\x0btimestampTz\x12\x32\n\x04uuid\x18 \x01(\x0b\x32\x1c.spark.connect.DataType.UUIDH\x00R\x04uuid\x12\x42\n\nfixed_char\x18\x15 \x01(\x0b\x32!.spark.connect.DataType.FixedCharH\x00R\tfixedChar\x12;\n\x07varchar\x18\x16 \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varchar\x12H\n\x0c\x66ixed_binary\x18\x17 \x01(\x0b\x32#.spark.connect.DataType.FixedBinaryH\x00R\x0b\x66ixedBinary\x12;\n\x07\x64\x65\x63imal\x18\x18 \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06struct\x18\x19 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12\x32\n\x04list\x18\x1b \x01(\x0b\x32\x1c.spark.connect.DataType.ListH\x00R\x04list\x12/\n\x03map\x18\x1c \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12?\n\x1buser_defined_type_reference\x18\x1f \x01(\rH\x00R\x18userDefinedTypeReference\x1a\x8a\x01\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x85\x01\n\x02I8\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x86\x01\n\x03I16\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x86\x01\n\x03I32\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x86\x01\n\x03I64\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x87\x01\n\x04\x46P32\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x87\x01\n\x04\x46P64\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x89\x01\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x89\x01\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x8c\x01\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x87\x01\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x87\x01\n\x04Time\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x8e\x01\n\x0bTimestampTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x8f\x01\n\x0cIntervalYear\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x8e\x01\n\x0bIntervalDay\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\x87\x01\n\x04UUID\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x02 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xa4\x01\n\tFixedChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x03 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xa2\x01\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x03 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xa6\x01\n\x0b\x46ixedBinary\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x03 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xbe\x01\n\x07\x44\x65\x63imal\x12\x14\n\x05scale\x18\x01 \x01(\x05R\x05scale\x12\x1c\n\tprecision\x18\x02 \x01(\x05R\tprecision\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x04 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xb8\x01\n\x06Struct\x12-\n\x05types\x18\x01 \x03(\x0b\x32\x17.spark.connect.DataTypeR\x05types\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x03 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xbc\x01\n\x04List\x12\x33\n\x08\x44\x61taType\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x44\x61taType\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x03 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability\x1a\xe0\x01\n\x03Map\x12)\n\x03key\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x03key\x12-\n\x05value\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x05value\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x12\x45\n\x0bnullability\x18\x04 \x01(\x0e\x32#.spark.connect.DataType.NullabilityR\x0bnullability"^\n\x0bNullability\x12\x1b\n\x17NULLABILITY_UNSPECIFIED\x10\x00\x12\x18\n\x14NULLABILITY_NULLABLE\x10\x01\x12\x18\n\x14NULLABILITY_REQUIRED\x10\x02\x42\x06\n\x04kindB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' + b'\n\x19spark/connect/types.proto\x12\rspark.connect"\xc1\x1c\n\x08\x44\x61taType\x12\x35\n\x04\x62ool\x18\x01 \x01(\x0b\x32\x1f.spark.connect.DataType.BooleanH\x00R\x04\x62ool\x12,\n\x02i8\x18\x02 \x01(\x0b\x32\x1a.spark.connect.DataType.I8H\x00R\x02i8\x12/\n\x03i16\x18\x03 \x01(\x0b\x32\x1b.spark.connect.DataType.I16H\x00R\x03i16\x12/\n\x03i32\x18\x05 \x01(\x0b\x32\x1b.spark.connect.DataType.I32H\x00R\x03i32\x12/\n\x03i64\x18\x07 \x01(\x0b\x32\x1b.spark.connect.DataType.I64H\x00R\x03i64\x12\x32\n\x04\x66p32\x18\n \x01(\x0b\x32\x1c.spark.connect.DataType.FP32H\x00R\x04\x66p32\x12\x32\n\x04\x66p64\x18\x0b \x01(\x0b\x32\x1c.spark.connect.DataType.FP64H\x00R\x04\x66p64\x12\x38\n\x06string\x18\x0c \x01(\x0b\x32\x1e.spark.connect.DataType.StringH\x00R\x06string\x12\x38\n\x06\x62inary\x18\r \x01(\x0b\x32\x1e.spark.connect.DataType.BinaryH\x00R\x06\x62inary\x12\x41\n\ttimestamp\x18\x0e \x01(\x0b\x32!.spark.connect.DataType.TimestampH\x00R\ttimestamp\x12\x32\n\x04\x64\x61te\x18\x10 \x01(\x0b\x32\x1c.spark.connect.DataType.DateH\x00R\x04\x64\x61te\x12\x32\n\x04time\x18\x11 \x01(\x0b\x32\x1c.spark.connect.DataType.TimeH\x00R\x04time\x12K\n\rinterval_year\x18\x13 \x01(\x0b\x32$.spark.connect.DataType.IntervalYearH\x00R\x0cintervalYear\x12H\n\x0cinterval_day\x18\x14 \x01(\x0b\x32#.spark.connect.DataType.IntervalDayH\x00R\x0bintervalDay\x12H\n\x0ctimestamp_tz\x18\x1d \x01(\x0b\x32#.spark.connect.DataType.TimestampTZH\x00R\x0btimestampTz\x12\x32\n\x04uuid\x18 \x01(\x0b\x32\x1c.spark.connect.DataType.UUIDH\x00R\x04uuid\x12\x42\n\nfixed_char\x18\x15 \x01(\x0b\x32!.spark.connect.DataType.FixedCharH\x00R\tfixedChar\x12;\n\x07varchar\x18\x16 \x01(\x0b\x32\x1f.spark.connect.DataType.VarCharH\x00R\x07varchar\x12H\n\x0c\x66ixed_binary\x18\x17 \x01(\x0b\x32#.spark.connect.DataType.FixedBinaryH\x00R\x0b\x66ixedBinary\x12;\n\x07\x64\x65\x63imal\x18\x18 \x01(\x0b\x32\x1f.spark.connect.DataType.DecimalH\x00R\x07\x64\x65\x63imal\x12\x38\n\x06struct\x18\x19 \x01(\x0b\x32\x1e.spark.connect.DataType.StructH\x00R\x06struct\x12\x32\n\x04list\x18\x1b \x01(\x0b\x32\x1c.spark.connect.DataType.ListH\x00R\x04list\x12/\n\x03map\x18\x1c \x01(\x0b\x32\x1b.spark.connect.DataType.MapH\x00R\x03map\x12?\n\x1buser_defined_type_reference\x18\x1f \x01(\rH\x00R\x18userDefinedTypeReference\x1a\x43\n\x07\x42oolean\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a>\n\x02I8\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a?\n\x03I16\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a?\n\x03I32\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a?\n\x03I64\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x46P32\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x46P64\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06String\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x42\n\x06\x42inary\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a\x45\n\tTimestamp\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04\x44\x61te\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04Time\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aG\n\x0bTimestampTZ\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aH\n\x0cIntervalYear\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1aG\n\x0bIntervalDay\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a@\n\x04UUID\x12\x38\n\x18type_variation_reference\x18\x01 \x01(\rR\x16typeVariationReference\x1a]\n\tFixedChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a[\n\x07VarChar\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a_\n\x0b\x46ixedBinary\x12\x16\n\x06length\x18\x01 \x01(\x05R\x06length\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1aw\n\x07\x44\x65\x63imal\x12\x14\n\x05scale\x18\x01 \x01(\x05R\x05scale\x12\x1c\n\tprecision\x18\x02 \x01(\x05R\tprecision\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x1a\xf6\x01\n\x0bStructField\x12+\n\x04type\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x04type\x12\x12\n\x04name\x18\x02 \x01(\tR\x04name\x12\x1a\n\x08nullable\x18\x03 \x01(\x08R\x08nullable\x12M\n\x08metadata\x18\x04 \x03(\x0b\x32\x31.spark.connect.DataType.StructField.MetadataEntryR\x08metadata\x1a;\n\rMetadataEntry\x12\x10\n\x03key\x18\x01 \x01(\tR\x03key\x12\x14\n\x05value\x18\x02 \x01(\tR\x05value:\x02\x38\x01\x1a\x7f\n\x06Struct\x12;\n\x06\x66ields\x18\x01 \x03(\x0b\x32#.spark.connect.DataType.StructFieldR\x06\x66ields\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x1a\xa0\x01\n\x04List\x12\x33\n\x08\x44\x61taType\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x08\x44\x61taType\x12\x38\n\x18type_variation_reference\x18\x02 \x01(\rR\x16typeVariationReference\x12)\n\x10\x65lement_nullable\x18\x03 \x01(\x08R\x0f\x65lementNullable\x1a\xc0\x01\n\x03Map\x12)\n\x03key\x18\x01 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x03key\x12-\n\x05value\x18\x02 \x01(\x0b\x32\x17.spark.connect.DataTypeR\x05value\x12\x38\n\x18type_variation_reference\x18\x03 \x01(\rR\x16typeVariationReference\x12%\n\x0evalue_nullable\x18\x04 \x01(\x08R\rvalueNullableB\x06\n\x04kindB"\n\x1eorg.apache.spark.connect.protoP\x01\x62\x06proto3' ) _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, globals()) @@ -38,54 +38,58 @@ DESCRIPTOR._options = None DESCRIPTOR._serialized_options = b"\n\036org.apache.spark.connect.protoP\001" + _DATATYPE_STRUCTFIELD_METADATAENTRY._options = None + _DATATYPE_STRUCTFIELD_METADATAENTRY._serialized_options = b"8\001" _DATATYPE._serialized_start = 45 - _DATATYPE._serialized_end = 5099 - _DATATYPE_BOOLEAN._serialized_start = 1462 - _DATATYPE_BOOLEAN._serialized_end = 1600 - _DATATYPE_I8._serialized_start = 1603 - _DATATYPE_I8._serialized_end = 1736 - _DATATYPE_I16._serialized_start = 1739 - _DATATYPE_I16._serialized_end = 1873 - _DATATYPE_I32._serialized_start = 1876 - _DATATYPE_I32._serialized_end = 2010 - _DATATYPE_I64._serialized_start = 2013 - _DATATYPE_I64._serialized_end = 2147 - _DATATYPE_FP32._serialized_start = 2150 - _DATATYPE_FP32._serialized_end = 2285 - _DATATYPE_FP64._serialized_start = 2288 - _DATATYPE_FP64._serialized_end = 2423 - _DATATYPE_STRING._serialized_start = 2426 - _DATATYPE_STRING._serialized_end = 2563 - _DATATYPE_BINARY._serialized_start = 2566 - _DATATYPE_BINARY._serialized_end = 2703 - _DATATYPE_TIMESTAMP._serialized_start = 2706 - _DATATYPE_TIMESTAMP._serialized_end = 2846 - _DATATYPE_DATE._serialized_start = 2849 - _DATATYPE_DATE._serialized_end = 2984 - _DATATYPE_TIME._serialized_start = 2987 - _DATATYPE_TIME._serialized_end = 3122 - _DATATYPE_TIMESTAMPTZ._serialized_start = 3125 - _DATATYPE_TIMESTAMPTZ._serialized_end = 3267 - _DATATYPE_INTERVALYEAR._serialized_start = 3270 - _DATATYPE_INTERVALYEAR._serialized_end = 3413 - _DATATYPE_INTERVALDAY._serialized_start = 3416 - _DATATYPE_INTERVALDAY._serialized_end = 3558 - _DATATYPE_UUID._serialized_start = 3561 - _DATATYPE_UUID._serialized_end = 3696 - _DATATYPE_FIXEDCHAR._serialized_start = 3699 - _DATATYPE_FIXEDCHAR._serialized_end = 3863 - _DATATYPE_VARCHAR._serialized_start = 3866 - _DATATYPE_VARCHAR._serialized_end = 4028 - _DATATYPE_FIXEDBINARY._serialized_start = 4031 - _DATATYPE_FIXEDBINARY._serialized_end = 4197 - _DATATYPE_DECIMAL._serialized_start = 4200 - _DATATYPE_DECIMAL._serialized_end = 4390 - _DATATYPE_STRUCT._serialized_start = 4393 - _DATATYPE_STRUCT._serialized_end = 4577 - _DATATYPE_LIST._serialized_start = 4580 - _DATATYPE_LIST._serialized_end = 4768 - _DATATYPE_MAP._serialized_start = 4771 - _DATATYPE_MAP._serialized_end = 4995 - _DATATYPE_NULLABILITY._serialized_start = 4997 - _DATATYPE_NULLABILITY._serialized_end = 5091 + _DATATYPE._serialized_end = 3694 + _DATATYPE_BOOLEAN._serialized_start = 1461 + _DATATYPE_BOOLEAN._serialized_end = 1528 + _DATATYPE_I8._serialized_start = 1530 + _DATATYPE_I8._serialized_end = 1592 + _DATATYPE_I16._serialized_start = 1594 + _DATATYPE_I16._serialized_end = 1657 + _DATATYPE_I32._serialized_start = 1659 + _DATATYPE_I32._serialized_end = 1722 + _DATATYPE_I64._serialized_start = 1724 + _DATATYPE_I64._serialized_end = 1787 + _DATATYPE_FP32._serialized_start = 1789 + _DATATYPE_FP32._serialized_end = 1853 + _DATATYPE_FP64._serialized_start = 1855 + _DATATYPE_FP64._serialized_end = 1919 + _DATATYPE_STRING._serialized_start = 1921 + _DATATYPE_STRING._serialized_end = 1987 + _DATATYPE_BINARY._serialized_start = 1989 + _DATATYPE_BINARY._serialized_end = 2055 + _DATATYPE_TIMESTAMP._serialized_start = 2057 + _DATATYPE_TIMESTAMP._serialized_end = 2126 + _DATATYPE_DATE._serialized_start = 2128 + _DATATYPE_DATE._serialized_end = 2192 + _DATATYPE_TIME._serialized_start = 2194 + _DATATYPE_TIME._serialized_end = 2258 + _DATATYPE_TIMESTAMPTZ._serialized_start = 2260 + _DATATYPE_TIMESTAMPTZ._serialized_end = 2331 + _DATATYPE_INTERVALYEAR._serialized_start = 2333 + _DATATYPE_INTERVALYEAR._serialized_end = 2405 + _DATATYPE_INTERVALDAY._serialized_start = 2407 + _DATATYPE_INTERVALDAY._serialized_end = 2478 + _DATATYPE_UUID._serialized_start = 2480 + _DATATYPE_UUID._serialized_end = 2544 + _DATATYPE_FIXEDCHAR._serialized_start = 2546 + _DATATYPE_FIXEDCHAR._serialized_end = 2639 + _DATATYPE_VARCHAR._serialized_start = 2641 + _DATATYPE_VARCHAR._serialized_end = 2732 + _DATATYPE_FIXEDBINARY._serialized_start = 2734 + _DATATYPE_FIXEDBINARY._serialized_end = 2829 + _DATATYPE_DECIMAL._serialized_start = 2831 + _DATATYPE_DECIMAL._serialized_end = 2950 + _DATATYPE_STRUCTFIELD._serialized_start = 2953 + _DATATYPE_STRUCTFIELD._serialized_end = 3199 + _DATATYPE_STRUCTFIELD_METADATAENTRY._serialized_start = 3140 + _DATATYPE_STRUCTFIELD_METADATAENTRY._serialized_end = 3199 + _DATATYPE_STRUCT._serialized_start = 3201 + _DATATYPE_STRUCT._serialized_end = 3328 + _DATATYPE_LIST._serialized_start = 3331 + _DATATYPE_LIST._serialized_end = 3491 + _DATATYPE_MAP._serialized_start = 3494 + _DATATYPE_MAP._serialized_end = 3686 # @@protoc_insertion_point(module_scope) diff --git a/python/pyspark/sql/connect/proto/types_pb2.pyi b/python/pyspark/sql/connect/proto/types_pb2.pyi index 8486053b435b2..3bf36fc790c67 100644 --- a/python/pyspark/sql/connect/proto/types_pb2.pyi +++ b/python/pyspark/sql/connect/proto/types_pb2.pyi @@ -37,12 +37,10 @@ import builtins import collections.abc import google.protobuf.descriptor import google.protobuf.internal.containers -import google.protobuf.internal.enum_type_wrapper import google.protobuf.message import sys -import typing -if sys.version_info >= (3, 10): +if sys.version_info >= (3, 8): import typing as typing_extensions else: import typing_extensions @@ -56,46 +54,20 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - class _Nullability: - ValueType = typing.NewType("ValueType", builtins.int) - V: typing_extensions.TypeAlias = ValueType - - class _NullabilityEnumTypeWrapper( - google.protobuf.internal.enum_type_wrapper._EnumTypeWrapper[ - DataType._Nullability.ValueType - ], - builtins.type, - ): # noqa: F821 - DESCRIPTOR: google.protobuf.descriptor.EnumDescriptor - NULLABILITY_UNSPECIFIED: DataType._Nullability.ValueType # 0 - NULLABILITY_NULLABLE: DataType._Nullability.ValueType # 1 - NULLABILITY_REQUIRED: DataType._Nullability.ValueType # 2 - - class Nullability(_Nullability, metaclass=_NullabilityEnumTypeWrapper): ... - NULLABILITY_UNSPECIFIED: DataType.Nullability.ValueType # 0 - NULLABILITY_NULLABLE: DataType.Nullability.ValueType # 1 - NULLABILITY_REQUIRED: DataType.Nullability.ValueType # 2 - class Boolean(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -103,22 +75,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -126,22 +92,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -149,22 +109,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -172,22 +126,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -195,22 +143,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -218,22 +160,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -241,22 +177,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -264,22 +194,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -287,22 +211,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -310,22 +228,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -333,22 +245,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -356,22 +262,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -379,22 +279,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -402,22 +296,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -425,22 +313,16 @@ class DataType(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -451,26 +333,18 @@ class DataType(google.protobuf.message.Message): LENGTH_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int length: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, length: builtins.int = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "length", - b"length", - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "length", b"length", "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -479,26 +353,18 @@ class DataType(google.protobuf.message.Message): LENGTH_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int length: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, length: builtins.int = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "length", - b"length", - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "length", b"length", "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -507,26 +373,18 @@ class DataType(google.protobuf.message.Message): LENGTH_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int length: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, length: builtins.int = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "length", - b"length", - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", + "length", b"length", "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -536,24 +394,19 @@ class DataType(google.protobuf.message.Message): SCALE_FIELD_NUMBER: builtins.int PRECISION_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int scale: builtins.int precision: builtins.int type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, scale: builtins.int = ..., precision: builtins.int = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", "precision", b"precision", "scale", @@ -563,36 +416,78 @@ class DataType(google.protobuf.message.Message): ], ) -> None: ... + class StructField(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + class MetadataEntry(google.protobuf.message.Message): + DESCRIPTOR: google.protobuf.descriptor.Descriptor + + KEY_FIELD_NUMBER: builtins.int + VALUE_FIELD_NUMBER: builtins.int + key: builtins.str + value: builtins.str + def __init__( + self, + *, + key: builtins.str = ..., + value: builtins.str = ..., + ) -> None: ... + def ClearField( + self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] + ) -> None: ... + + TYPE_FIELD_NUMBER: builtins.int + NAME_FIELD_NUMBER: builtins.int + NULLABLE_FIELD_NUMBER: builtins.int + METADATA_FIELD_NUMBER: builtins.int + @property + def type(self) -> global___DataType: ... + name: builtins.str + nullable: builtins.bool + @property + def metadata( + self, + ) -> google.protobuf.internal.containers.ScalarMap[builtins.str, builtins.str]: ... + def __init__( + self, + *, + type: global___DataType | None = ..., + name: builtins.str = ..., + nullable: builtins.bool = ..., + metadata: collections.abc.Mapping[builtins.str, builtins.str] | None = ..., + ) -> None: ... + def HasField( + self, field_name: typing_extensions.Literal["type", b"type"] + ) -> builtins.bool: ... + def ClearField( + self, + field_name: typing_extensions.Literal[ + "metadata", b"metadata", "name", b"name", "nullable", b"nullable", "type", b"type" + ], + ) -> None: ... + class Struct(google.protobuf.message.Message): DESCRIPTOR: google.protobuf.descriptor.Descriptor - TYPES_FIELD_NUMBER: builtins.int + FIELDS_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int @property - def types( + def fields( self, ) -> google.protobuf.internal.containers.RepeatedCompositeFieldContainer[ - global___DataType + global___DataType.StructField ]: ... type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType def __init__( self, *, - types: collections.abc.Iterable[global___DataType] | None = ..., + fields: collections.abc.Iterable[global___DataType.StructField] | None = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., ) -> None: ... def ClearField( self, field_name: typing_extensions.Literal[ - "nullability", - b"nullability", - "type_variation_reference", - b"type_variation_reference", - "types", - b"types", + "fields", b"fields", "type_variation_reference", b"type_variation_reference" ], ) -> None: ... @@ -601,17 +496,17 @@ class DataType(google.protobuf.message.Message): DATATYPE_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int + ELEMENT_NULLABLE_FIELD_NUMBER: builtins.int @property def DataType(self) -> global___DataType: ... type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType + element_nullable: builtins.bool def __init__( self, *, DataType: global___DataType | None = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., + element_nullable: builtins.bool = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal["DataType", b"DataType"] @@ -621,8 +516,8 @@ class DataType(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "DataType", b"DataType", - "nullability", - b"nullability", + "element_nullable", + b"element_nullable", "type_variation_reference", b"type_variation_reference", ], @@ -634,20 +529,20 @@ class DataType(google.protobuf.message.Message): KEY_FIELD_NUMBER: builtins.int VALUE_FIELD_NUMBER: builtins.int TYPE_VARIATION_REFERENCE_FIELD_NUMBER: builtins.int - NULLABILITY_FIELD_NUMBER: builtins.int + VALUE_NULLABLE_FIELD_NUMBER: builtins.int @property def key(self) -> global___DataType: ... @property def value(self) -> global___DataType: ... type_variation_reference: builtins.int - nullability: global___DataType.Nullability.ValueType + value_nullable: builtins.bool def __init__( self, *, key: global___DataType | None = ..., value: global___DataType | None = ..., type_variation_reference: builtins.int = ..., - nullability: global___DataType.Nullability.ValueType = ..., + value_nullable: builtins.bool = ..., ) -> None: ... def HasField( self, field_name: typing_extensions.Literal["key", b"key", "value", b"value"] @@ -657,12 +552,12 @@ class DataType(google.protobuf.message.Message): field_name: typing_extensions.Literal[ "key", b"key", - "nullability", - b"nullability", "type_variation_reference", b"type_variation_reference", "value", b"value", + "value_nullable", + b"value_nullable", ], ) -> None: ...