diff --git a/python/vegafusion/proto/datafusion.proto b/python/vegafusion/proto/datafusion.proto index a1caa4c6..7707ed38 100644 --- a/python/vegafusion/proto/datafusion.proto +++ b/python/vegafusion/proto/datafusion.proto @@ -73,6 +73,8 @@ message LogicalPlanNode { CustomTableScanNode custom_scan = 25; PrepareNode prepare = 26; DropViewNode drop_view = 27; + DistinctOnNode distinct_on = 28; + CopyToNode copy_to = 29; } } @@ -88,6 +90,10 @@ message ProjectionColumns { message CsvFormat { bool has_header = 1; string delimiter = 2; + string quote = 3; + oneof optional_escape { + string escape = 4; + } } message ParquetFormat { @@ -176,6 +182,25 @@ message EmptyRelationNode { bool produce_one_row = 1; } +message PrimaryKeyConstraint{ + repeated uint64 indices = 1; +} + +message UniqueConstraint{ + repeated uint64 indices = 1; +} + +message Constraint{ + oneof constraint_mode{ + PrimaryKeyConstraint primary_key = 1; + UniqueConstraint unique = 2; + } +} + +message Constraints{ + repeated Constraint constraints = 1; +} + message CreateExternalTableNode { reserved 1; // was string name OwnedTableReference name = 12; @@ -187,10 +212,13 @@ message CreateExternalTableNode { bool if_not_exists = 7; string delimiter = 8; string definition = 9; - string file_compression_type = 10; + reserved 10; // was string file_compression_type + CompressionTypeVariant file_compression_type = 17; repeated LogicalExprNodeCollection order_exprs = 13; bool unbounded = 14; map options = 11; + Constraints constraints = 15; + map column_defaults = 16; } message PrepareNode { @@ -284,6 +312,32 @@ message DistinctNode { LogicalPlanNode input = 1; } +message DistinctOnNode { + repeated LogicalExprNode on_expr = 1; + repeated LogicalExprNode select_expr = 2; + repeated LogicalExprNode sort_expr = 3; + LogicalPlanNode input = 4; +} + +message CopyToNode { + LogicalPlanNode input = 1; + string output_url = 2; + oneof CopyOptions { + SQLOptions sql_options = 4; + FileTypeWriterOptions writer_options = 5; + } + string file_type = 6; +} + +message SQLOptions { + repeated SQLOption option = 1; +} + +message SQLOption { + string key = 1; + string value = 2; +} + message UnionNode { repeated LogicalPlanNode inputs = 1; } @@ -339,7 +393,7 @@ message LogicalExprNode { SortExprNode sort = 12; NegativeNode negative = 13; InListNode in_list = 14; - bool wildcard = 15; + Wildcard wildcard = 15; ScalarFunctionNode scalar_function = 16; TryCastNode try_cast = 17; @@ -372,9 +426,15 @@ message LogicalExprNode { PlaceholderNode placeholder = 34; + Unnest unnest = 35; + } } +message Wildcard { + string qualifier = 1; +} + message PlaceholderNode { string id = 1; ArrowType data_type = 2; @@ -396,11 +456,27 @@ message RollupNode { repeated LogicalExprNode expr = 1; } +message NamedStructField { + ScalarValue name = 1; +} + +message ListIndex { + LogicalExprNode key = 1; +} +message ListRange { + LogicalExprNode start = 1; + LogicalExprNode stop = 2; + LogicalExprNode stride = 3; +} message GetIndexedField { LogicalExprNode expr = 1; - ScalarValue key = 2; + oneof field { + NamedStructField named_struct_field = 2; + ListIndex list_index = 3; + ListRange list_range = 4; + } } message IsNull { @@ -442,6 +518,7 @@ message Not { message AliasNode { LogicalExprNode expr = 1; string alias = 2; + repeated OwnedTableReference relation = 3; } message BinaryExprNode { @@ -456,6 +533,10 @@ message NegativeNode { LogicalExprNode expr = 1; } +message Unnest { + repeated LogicalExprNode exprs = 1; +} + message InListNode { LogicalExprNode expr = 1; repeated LogicalExprNode list = 2; @@ -499,7 +580,7 @@ enum ScalarFunction { Lower = 33; Ltrim = 34; MD5 = 35; - NullIf = 36; + // 36 was NullIf OctetLength = 37; Random = 38; RegexpReplace = 39; @@ -552,7 +633,7 @@ enum ScalarFunction { ArrayAppend = 86; ArrayConcat = 87; ArrayDims = 88; - ArrayFill = 89; + ArrayRepeat = 89; ArrayLength = 90; ArrayNdims = 91; ArrayPosition = 92; @@ -560,13 +641,44 @@ enum ScalarFunction { ArrayPrepend = 94; ArrayRemove = 95; ArrayReplace = 96; - ArrayToString = 97; + // 97 was ArrayToString Cardinality = 98; - TrimArray = 99; - ArrayContains = 100; - Encode = 101; - Decode = 102; + ArrayElement = 99; + ArraySlice = 100; Cot = 103; + ArrayHas = 104; + ArrayHasAny = 105; + ArrayHasAll = 106; + ArrayRemoveN = 107; + ArrayReplaceN = 108; + ArrayRemoveAll = 109; + ArrayReplaceAll = 110; + Nanvl = 111; + Flatten = 112; + // 113 was IsNan + Iszero = 114; + ArrayEmpty = 115; + ArrayPopBack = 116; + StringToArray = 117; + ToTimestampNanos = 118; + ArrayIntersect = 119; + ArrayUnion = 120; + OverLay = 121; + Range = 122; + ArrayExcept = 123; + ArrayPopFront = 124; + Levenshtein = 125; + SubstrIndex = 126; + FindInSet = 127; + ArraySort = 128; + ArrayDistinct = 129; + ArrayResize = 130; + EndsWith = 131; + InStr = 132; + MakeDate = 133; + ArrayReverse = 134; + RegexpLike = 135; + ToChar = 136; } message ScalarFunctionNode { @@ -603,6 +715,17 @@ enum AggregateFunction { // we append "_AGG" to obey name scoping rules. FIRST_VALUE_AGG = 24; LAST_VALUE_AGG = 25; + REGR_SLOPE = 26; + REGR_INTERCEPT = 27; + REGR_COUNT = 28; + REGR_R2 = 29; + REGR_AVGX = 30; + REGR_AVGY = 31; + REGR_SXX = 32; + REGR_SYY = 33; + REGR_SXY = 34; + STRING_AGG = 35; + NTH_VALUE_AGG = 36; } message AggregateExprNode { @@ -752,6 +875,8 @@ message Field { // for complex data types like structs, unions repeated Field children = 4; map metadata = 5; + int64 dict_id = 6; + bool dict_ordered = 7; } message FixedSizeBinary{ @@ -821,12 +946,11 @@ message Union{ repeated int32 type_ids = 3; } -message ScalarListValue{ - // encode null explicitly to distinguish a list with a null value - // from a list with no values) - bool is_null = 3; - Field field = 1; - repeated ScalarValue values = 2; +// Used for List/FixedSizeList/LargeList/Struct +message ScalarNestedValue { + bytes ipc_message = 1; + bytes arrow_data = 2; + Schema schema = 3; } message ScalarTime32Value { @@ -864,14 +988,6 @@ message IntervalMonthDayNanoValue { int64 nanos = 3; } -message StructValue { - // Note that a null struct value must have one or more fields, so we - // encode a null StructValue as one witth an empty field_values - // list. - repeated ScalarValue field_values = 2; - repeated Field fields = 3; -} - message ScalarFixedSizeBinary{ bytes values = 1; int32 length = 2; @@ -902,10 +1018,14 @@ message ScalarValue{ // Literal Date32 value always has a unit of day int32 date_32_value = 14; ScalarTime32Value time32_value = 15; - ScalarListValue list_value = 17; - //WAS: ScalarType null_list_value = 18; + ScalarNestedValue large_list_value = 16; + ScalarNestedValue list_value = 17; + ScalarNestedValue fixed_size_list_value = 18; + ScalarNestedValue struct_value = 32; Decimal128 decimal128_value = 20; + Decimal256 decimal256_value = 39; + int64 date_64_value = 21; int32 interval_yearmonth_value = 24; int64 interval_daytime_value = 25; @@ -921,7 +1041,6 @@ message ScalarValue{ bytes large_binary_value = 29; ScalarTime64Value time64_value = 30; IntervalMonthDayNanoValue interval_month_day_nano = 31; - StructValue struct_value = 32; ScalarFixedSizeBinary fixed_size_binary_value = 34; } } @@ -932,6 +1051,12 @@ message Decimal128{ int64 s = 3; } +message Decimal256{ + bytes value = 1; + int64 p = 2; + int64 s = 3; +} + // Serialized data type message ArrowType{ oneof arrow_type_enum { @@ -1002,8 +1127,10 @@ message PlanType { OptimizedLogicalPlanType OptimizedLogicalPlan = 2; EmptyMessage FinalLogicalPlan = 3; EmptyMessage InitialPhysicalPlan = 4; + EmptyMessage InitialPhysicalPlanWithStats = 9; OptimizedPhysicalPlanType OptimizedPhysicalPlan = 5; EmptyMessage FinalPhysicalPlan = 6; + EmptyMessage FinalPhysicalPlanWithStats = 10; } } @@ -1061,9 +1188,122 @@ message PhysicalPlanNode { ExplainExecNode explain = 20; SortPreservingMergeExecNode sort_preserving_merge = 21; NestedLoopJoinExecNode nested_loop_join = 22; + AnalyzeExecNode analyze = 23; + JsonSinkExecNode json_sink = 24; + SymmetricHashJoinExecNode symmetric_hash_join = 25; + InterleaveExecNode interleave = 26; + PlaceholderRowExecNode placeholder_row = 27; + CsvSinkExecNode csv_sink = 28; + ParquetSinkExecNode parquet_sink = 29; } } +enum CompressionTypeVariant { + GZIP = 0; + BZIP2 = 1; + XZ = 2; + ZSTD = 3; + UNCOMPRESSED = 4; +} + +message PartitionColumn { + string name = 1; + ArrowType arrow_type = 2; +} + +message FileTypeWriterOptions { + oneof FileType { + JsonWriterOptions json_options = 1; + ParquetWriterOptions parquet_options = 2; + CsvWriterOptions csv_options = 3; + ArrowWriterOptions arrow_options = 4; + } +} + +message JsonWriterOptions { + CompressionTypeVariant compression = 1; +} + +message ParquetWriterOptions { + WriterProperties writer_properties = 1; +} + +message CsvWriterOptions { + // Compression type + CompressionTypeVariant compression = 1; + // Optional column delimiter. Defaults to `b','` + string delimiter = 2; + // Whether to write column names as file headers. Defaults to `true` + bool has_header = 3; + // Optional date format for date arrays + string date_format = 4; + // Optional datetime format for datetime arrays + string datetime_format = 5; + // Optional timestamp format for timestamp arrays + string timestamp_format = 6; + // Optional time format for time arrays + string time_format = 7; + // Optional value to represent null + string null_value = 8; +} + +message ArrowWriterOptions {} + +message WriterProperties { + uint64 data_page_size_limit = 1; + uint64 dictionary_page_size_limit = 2; + uint64 data_page_row_count_limit = 3; + uint64 write_batch_size = 4; + uint64 max_row_group_size = 5; + string writer_version = 6; + string created_by = 7; +} + +message FileSinkConfig { + reserved 6; // writer_mode + + string object_store_url = 1; + repeated PartitionedFile file_groups = 2; + repeated string table_paths = 3; + Schema output_schema = 4; + repeated PartitionColumn table_partition_cols = 5; + bool overwrite = 8; + FileTypeWriterOptions file_type_writer_options = 9; +} + +message JsonSink { + FileSinkConfig config = 1; +} + +message JsonSinkExecNode { + PhysicalPlanNode input = 1; + JsonSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + +message CsvSink { + FileSinkConfig config = 1; +} + +message CsvSinkExecNode { + PhysicalPlanNode input = 1; + CsvSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + +message ParquetSink { + FileSinkConfig config = 1; +} + +message ParquetSinkExecNode { + PhysicalPlanNode input = 1; + ParquetSink sink = 2; + Schema sink_schema = 3; + PhysicalSortExprNodeCollection sort_order = 4; +} + message PhysicalExtensionNode { bytes node = 1; repeated PhysicalPlanNode inputs = 2; @@ -1071,6 +1311,9 @@ message PhysicalExtensionNode { // physical expressions message PhysicalExprNode { + // Was date_time_interval_expr + reserved 17; + oneof ExprType { // column references PhysicalColumn column = 1; @@ -1101,8 +1344,6 @@ message PhysicalExprNode { PhysicalScalarUdfNode scalar_udf = 16; - PhysicalDateTimeIntervalExprNode date_time_interval_expr = 17; - PhysicalLikeExprNode like_expr = 18; PhysicalGetIndexedFieldExprNode get_indexed_field_expr = 19; @@ -1131,7 +1372,11 @@ message PhysicalWindowExprNode { BuiltInWindowFunction built_in_function = 2; // udaf = 3 } - PhysicalExprNode expr = 4; + repeated PhysicalExprNode args = 4; + repeated PhysicalExprNode partition_by = 5; + repeated PhysicalSortExprNode order_by = 6; + WindowFrame window_frame = 7; + string name = 8; } message PhysicalIsNull { @@ -1217,6 +1462,7 @@ message PhysicalNegativeNode { message FilterExecNode { PhysicalPlanNode input = 1; PhysicalExprNode expr = 2; + uint32 default_filter_selectivity = 3; } message FileGroup { @@ -1259,6 +1505,10 @@ message CsvScanExecNode { FileScanExecConf base_conf = 1; bool has_header = 2; string delimiter = 3; + string quote = 4; + oneof optional_escape { + string escape = 5; + } } message AvroScanExecNode { @@ -1281,6 +1531,27 @@ message HashJoinExecNode { JoinFilter filter = 8; } +enum StreamPartitionMode { + SINGLE_PARTITION = 0; + PARTITIONED_EXEC = 1; +} + +message SymmetricHashJoinExecNode { + PhysicalPlanNode left = 1; + PhysicalPlanNode right = 2; + repeated JoinOn on = 3; + JoinType join_type = 4; + StreamPartitionMode partition_mode = 6; + bool null_equals_null = 7; + JoinFilter filter = 8; + repeated PhysicalSortExprNode left_sort_exprs = 9; + repeated PhysicalSortExprNode right_sort_exprs = 10; +} + +message InterleaveExecNode { + repeated PhysicalPlanNode inputs = 1; +} + message UnionExecNode { repeated PhysicalPlanNode inputs = 1; } @@ -1291,6 +1562,13 @@ message ExplainExecNode { bool verbose = 3; } +message AnalyzeExecNode { + bool verbose = 1; + bool show_statistics = 2; + PhysicalPlanNode input = 3; + Schema schema = 4; +} + message CrossJoinExecNode { PhysicalPlanNode left = 1; PhysicalPlanNode right = 2; @@ -1302,13 +1580,16 @@ message PhysicalColumn { } message JoinOn { - PhysicalColumn left = 1; - PhysicalColumn right = 2; + PhysicalExprNode left = 1; + PhysicalExprNode right = 2; } message EmptyExecNode { - bool produce_one_row = 1; - Schema schema = 2; + Schema schema = 1; +} + +message PlaceholderRowExecNode { + Schema schema = 1; } message ProjectionExecNode { @@ -1325,11 +1606,20 @@ enum AggregateMode { SINGLE_PARTITIONED = 4; } +message PartiallySortedInputOrderMode { + repeated uint64 columns = 6; +} + message WindowAggExecNode { PhysicalPlanNode input = 1; - repeated PhysicalExprNode window_expr = 2; - repeated string window_expr_name = 3; - Schema input_schema = 4; + repeated PhysicalWindowExprNode window_expr = 2; + repeated PhysicalExprNode partition_keys = 5; + // Set optional to `None` for `BoundedWindowAggExec`. + oneof input_order_mode { + EmptyMessage linear = 7; + PartiallySortedInputOrderMode partially_sorted = 8; + EmptyMessage sorted = 9; + } } message MaybeFilter { @@ -1352,7 +1642,6 @@ message AggregateExecNode { repeated PhysicalExprNode null_expr = 8; repeated bool groups = 9; repeated MaybeFilter filter_expr = 10; - repeated MaybePhysicalSortExprs order_by_expr = 11; } message GlobalLimitExecNode { @@ -1449,21 +1738,49 @@ message PartitionStats { repeated ColumnStats column_stats = 4; } +message Precision{ + PrecisionInfo precision_info = 1; + ScalarValue val = 2; +} + +enum PrecisionInfo { + EXACT = 0; + INEXACT = 1; + ABSENT = 2; +} + message Statistics { - int64 num_rows = 1; - int64 total_byte_size = 2; + Precision num_rows = 1; + Precision total_byte_size = 2; repeated ColumnStats column_stats = 3; - bool is_exact = 4; } message ColumnStats { - ScalarValue min_value = 1; - ScalarValue max_value = 2; - uint32 null_count = 3; - uint32 distinct_count = 4; + Precision min_value = 1; + Precision max_value = 2; + Precision null_count = 3; + Precision distinct_count = 4; +} + +message NamedStructFieldExpr { + ScalarValue name = 1; +} + +message ListIndexExpr { + PhysicalExprNode key = 1; +} + +message ListRangeExpr { + PhysicalExprNode start = 1; + PhysicalExprNode stop = 2; + PhysicalExprNode stride = 3; } message PhysicalGetIndexedFieldExprNode { PhysicalExprNode arg = 1; - ScalarValue key = 2; -} + oneof field { + NamedStructFieldExpr named_struct_field_expr = 2; + ListIndexExpr list_index_expr = 3; + ListRangeExpr list_range_expr = 4; + } +} \ No newline at end of file diff --git a/python/vegafusion/vegafusion/proto/datafusion_pb2.py b/python/vegafusion/vegafusion/proto/datafusion_pb2.py index 6cd5755e..53060149 100644 --- a/python/vegafusion/vegafusion/proto/datafusion_pb2.py +++ b/python/vegafusion/vegafusion/proto/datafusion_pb2.py @@ -13,7 +13,7 @@ -DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x16proto/datafusion.proto\x12\ndatafusion\"\"\n\x0e\x43olumnRelation\x12\x10\n\x08relation\x18\x01 \x01(\t\"D\n\x06\x43olumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12,\n\x08relation\x18\x02 \x01(\x0b\x32\x1a.datafusion.ColumnRelation\"Z\n\x07\x44\x66\x46ield\x12 \n\x05\x66ield\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12-\n\tqualifier\x18\x02 \x01(\x0b\x32\x1a.datafusion.ColumnRelation\"\x97\x01\n\x08\x44\x66Schema\x12$\n\x07\x63olumns\x18\x01 \x03(\x0b\x32\x13.datafusion.DfField\x12\x34\n\x08metadata\x18\x02 \x03(\x0b\x32\".datafusion.DfSchema.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xb3\n\n\x0fLogicalPlanNode\x12\x38\n\x0clisting_scan\x18\x01 \x01(\x0b\x32 .datafusion.ListingTableScanNodeH\x00\x12\x30\n\nprojection\x18\x03 \x01(\x0b\x32\x1a.datafusion.ProjectionNodeH\x00\x12.\n\tselection\x18\x04 \x01(\x0b\x32\x19.datafusion.SelectionNodeH\x00\x12&\n\x05limit\x18\x05 \x01(\x0b\x32\x15.datafusion.LimitNodeH\x00\x12.\n\taggregate\x18\x06 \x01(\x0b\x32\x19.datafusion.AggregateNodeH\x00\x12$\n\x04join\x18\x07 \x01(\x0b\x32\x14.datafusion.JoinNodeH\x00\x12$\n\x04sort\x18\x08 \x01(\x0b\x32\x14.datafusion.SortNodeH\x00\x12\x32\n\x0brepartition\x18\t \x01(\x0b\x32\x1b.datafusion.RepartitionNodeH\x00\x12\x37\n\x0e\x65mpty_relation\x18\n \x01(\x0b\x32\x1d.datafusion.EmptyRelationNodeH\x00\x12\x44\n\x15\x63reate_external_table\x18\x0b \x01(\x0b\x32#.datafusion.CreateExternalTableNodeH\x00\x12*\n\x07\x65xplain\x18\x0c \x01(\x0b\x32\x17.datafusion.ExplainNodeH\x00\x12(\n\x06window\x18\r \x01(\x0b\x32\x16.datafusion.WindowNodeH\x00\x12*\n\x07\x61nalyze\x18\x0e \x01(\x0b\x32\x17.datafusion.AnalyzeNodeH\x00\x12/\n\ncross_join\x18\x0f \x01(\x0b\x32\x19.datafusion.CrossJoinNodeH\x00\x12(\n\x06values\x18\x10 \x01(\x0b\x32\x16.datafusion.ValuesNodeH\x00\x12\x35\n\textension\x18\x11 \x01(\x0b\x32 .datafusion.LogicalExtensionNodeH\x00\x12\x44\n\x15\x63reate_catalog_schema\x18\x12 \x01(\x0b\x32#.datafusion.CreateCatalogSchemaNodeH\x00\x12&\n\x05union\x18\x13 \x01(\x0b\x32\x15.datafusion.UnionNodeH\x00\x12\x37\n\x0e\x63reate_catalog\x18\x14 \x01(\x0b\x32\x1d.datafusion.CreateCatalogNodeH\x00\x12\x37\n\x0esubquery_alias\x18\x15 \x01(\x0b\x32\x1d.datafusion.SubqueryAliasNodeH\x00\x12\x31\n\x0b\x63reate_view\x18\x16 \x01(\x0b\x32\x1a.datafusion.CreateViewNodeH\x00\x12,\n\x08\x64istinct\x18\x17 \x01(\x0b\x32\x18.datafusion.DistinctNodeH\x00\x12\x32\n\tview_scan\x18\x18 \x01(\x0b\x32\x1d.datafusion.ViewTableScanNodeH\x00\x12\x36\n\x0b\x63ustom_scan\x18\x19 \x01(\x0b\x32\x1f.datafusion.CustomTableScanNodeH\x00\x12*\n\x07prepare\x18\x1a \x01(\x0b\x32\x17.datafusion.PrepareNodeH\x00\x12-\n\tdrop_view\x18\x1b \x01(\x0b\x32\x18.datafusion.DropViewNodeH\x00\x42\x11\n\x0fLogicalPlanType\"Q\n\x14LogicalExtensionNode\x12\x0c\n\x04node\x18\x01 \x01(\x0c\x12+\n\x06inputs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalPlanNode\"$\n\x11ProjectionColumns\x12\x0f\n\x07\x63olumns\x18\x01 \x03(\t\"2\n\tCsvFormat\x12\x12\n\nhas_header\x18\x01 \x01(\x08\x12\x11\n\tdelimiter\x18\x02 \x01(\t\"\x15\n\rParquetFormatJ\x04\x08\x01\x10\x02\"\x0c\n\nAvroFormat\"T\n\x19LogicalExprNodeCollection\x12\x37\n\x12logical_expr_nodes\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\x9a\x04\n\x14ListingTableScanNode\x12\x33\n\ntable_name\x18\x0e \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\r\n\x05paths\x18\x02 \x03(\t\x12\x16\n\x0e\x66ile_extension\x18\x03 \x01(\t\x12\x31\n\nprojection\x18\x04 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\"\n\x06schema\x18\x05 \x01(\x0b\x32\x12.datafusion.Schema\x12,\n\x07\x66ilters\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x1c\n\x14table_partition_cols\x18\x07 \x03(\t\x12\x14\n\x0c\x63ollect_stat\x18\x08 \x01(\x08\x12\x19\n\x11target_partitions\x18\t \x01(\r\x12$\n\x03\x63sv\x18\n \x01(\x0b\x32\x15.datafusion.CsvFormatH\x00\x12,\n\x07parquet\x18\x0b \x01(\x0b\x32\x19.datafusion.ParquetFormatH\x00\x12&\n\x04\x61vro\x18\x0c \x01(\x0b\x32\x16.datafusion.AvroFormatH\x00\x12>\n\x0f\x66ile_sort_order\x18\r \x03(\x0b\x32%.datafusion.LogicalExprNodeCollectionB\x10\n\x0e\x46ileFormatTypeJ\x04\x08\x01\x10\x02\"\xe5\x01\n\x11ViewTableScanNode\x12\x33\n\ntable_name\x18\x06 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12*\n\x05input\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12\x31\n\nprojection\x18\x04 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\x12\n\ndefinition\x18\x05 \x01(\tJ\x04\x08\x01\x10\x02\"\xf0\x01\n\x13\x43ustomTableScanNode\x12\x33\n\ntable_name\x18\x06 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x31\n\nprojection\x18\x02 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12,\n\x07\x66ilters\x18\x04 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x19\n\x11\x63ustom_table_data\x18\x05 \x01(\x0cJ\x04\x08\x01\x10\x02\"\x8a\x01\n\x0eProjectionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x05\x61lias\x18\x03 \x01(\tH\x00\x42\x10\n\x0eoptional_alias\"f\n\rSelectionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"p\n\x08SortNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"\x95\x01\n\x0fRepartitionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x15\n\x0bround_robin\x18\x02 \x01(\x04H\x00\x12+\n\x04hash\x18\x03 \x01(\x0b\x32\x1b.datafusion.HashRepartitionH\x00\x42\x12\n\x10partition_method\"Z\n\x0fHashRepartition\x12.\n\thash_expr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x17\n\x0fpartition_count\x18\x02 \x01(\x04\",\n\x11\x45mptyRelationNode\x12\x17\n\x0fproduce_one_row\x18\x01 \x01(\x08\"\xea\x03\n\x17\x43reateExternalTableNode\x12-\n\x04name\x18\x0c \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x10\n\x08location\x18\x02 \x01(\t\x12\x11\n\tfile_type\x18\x03 \x01(\t\x12\x12\n\nhas_header\x18\x04 \x01(\x08\x12$\n\x06schema\x18\x05 \x01(\x0b\x32\x14.datafusion.DfSchema\x12\x1c\n\x14table_partition_cols\x18\x06 \x03(\t\x12\x15\n\rif_not_exists\x18\x07 \x01(\x08\x12\x11\n\tdelimiter\x18\x08 \x01(\t\x12\x12\n\ndefinition\x18\t \x01(\t\x12\x1d\n\x15\x66ile_compression_type\x18\n \x01(\t\x12:\n\x0border_exprs\x18\r \x03(\x0b\x32%.datafusion.LogicalExprNodeCollection\x12\x11\n\tunbounded\x18\x0e \x01(\x08\x12\x41\n\x07options\x18\x0b \x03(\x0b\x32\x30.datafusion.CreateExternalTableNode.OptionsEntry\x1a.\n\x0cOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01J\x04\x08\x01\x10\x02\"r\n\x0bPrepareNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12)\n\ndata_types\x18\x02 \x03(\x0b\x32\x15.datafusion.ArrowType\x12*\n\x05input\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"k\n\x17\x43reateCatalogSchemaNode\x12\x13\n\x0bschema_name\x18\x01 \x01(\t\x12\x15\n\rif_not_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"f\n\x11\x43reateCatalogNode\x12\x14\n\x0c\x63\x61talog_name\x18\x01 \x01(\t\x12\x15\n\rif_not_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"v\n\x0c\x44ropViewNode\x12-\n\x04name\x18\x01 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x11\n\tif_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"\x99\x01\n\x0e\x43reateViewNode\x12-\n\x04name\x18\x05 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12*\n\x05input\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x12\n\nor_replace\x18\x03 \x01(\x08\x12\x12\n\ndefinition\x18\x04 \x01(\tJ\x04\x08\x01\x10\x02\"N\n\nValuesNode\x12\x0e\n\x06n_cols\x18\x01 \x01(\x04\x12\x30\n\x0bvalues_list\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"J\n\x0b\x41nalyzeNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0f\n\x07verbose\x18\x02 \x01(\x08\"J\n\x0b\x45xplainNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0f\n\x07verbose\x18\x02 \x01(\x08\"\x9c\x01\n\rAggregateNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12/\n\ngroup_expr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12.\n\taggr_expr\x18\x03 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"j\n\nWindowNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x30\n\x0bwindow_expr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xef\x02\n\x08JoinNode\x12)\n\x04left\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12*\n\x05right\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\'\n\tjoin_type\x18\x03 \x01(\x0e\x32\x14.datafusion.JoinType\x12\x33\n\x0fjoin_constraint\x18\x04 \x01(\x0e\x32\x1a.datafusion.JoinConstraint\x12\x32\n\rleft_join_key\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x33\n\x0eright_join_key\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x18\n\x10null_equals_null\x18\x07 \x01(\x08\x12+\n\x06\x66ilter\x18\x08 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\":\n\x0c\x44istinctNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"8\n\tUnionNode\x12+\n\x06inputs\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalPlanNode\"f\n\rCrossJoinNode\x12)\n\x04left\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12*\n\x05right\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"T\n\tLimitNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0c\n\x04skip\x18\x02 \x01(\x03\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\">\n\x11SelectionExecNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"u\n\x11SubqueryAliasNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12.\n\x05\x61lias\x18\x03 \x01(\x0b\x32\x1f.datafusion.OwnedTableReferenceJ\x04\x08\x02\x10\x03\"\xad\x0c\n\x0fLogicalExprNode\x12$\n\x06\x63olumn\x18\x01 \x01(\x0b\x32\x12.datafusion.ColumnH\x00\x12&\n\x05\x61lias\x18\x02 \x01(\x0b\x32\x15.datafusion.AliasNodeH\x00\x12*\n\x07literal\x18\x03 \x01(\x0b\x32\x17.datafusion.ScalarValueH\x00\x12\x31\n\x0b\x62inary_expr\x18\x04 \x01(\x0b\x32\x1a.datafusion.BinaryExprNodeH\x00\x12\x37\n\x0e\x61ggregate_expr\x18\x05 \x01(\x0b\x32\x1d.datafusion.AggregateExprNodeH\x00\x12*\n\x0cis_null_expr\x18\x06 \x01(\x0b\x32\x12.datafusion.IsNullH\x00\x12\x31\n\x10is_not_null_expr\x18\x07 \x01(\x0b\x32\x15.datafusion.IsNotNullH\x00\x12#\n\x08not_expr\x18\x08 \x01(\x0b\x32\x0f.datafusion.NotH\x00\x12*\n\x07\x62\x65tween\x18\t \x01(\x0b\x32\x17.datafusion.BetweenNodeH\x00\x12%\n\x05\x63\x61se_\x18\n \x01(\x0b\x32\x14.datafusion.CaseNodeH\x00\x12$\n\x04\x63\x61st\x18\x0b \x01(\x0b\x32\x14.datafusion.CastNodeH\x00\x12(\n\x04sort\x18\x0c \x01(\x0b\x32\x18.datafusion.SortExprNodeH\x00\x12,\n\x08negative\x18\r \x01(\x0b\x32\x18.datafusion.NegativeNodeH\x00\x12)\n\x07in_list\x18\x0e \x01(\x0b\x32\x16.datafusion.InListNodeH\x00\x12\x12\n\x08wildcard\x18\x0f \x01(\x08H\x00\x12\x39\n\x0fscalar_function\x18\x10 \x01(\x0b\x32\x1e.datafusion.ScalarFunctionNodeH\x00\x12+\n\x08try_cast\x18\x11 \x01(\x0b\x32\x17.datafusion.TryCastNodeH\x00\x12\x31\n\x0bwindow_expr\x18\x12 \x01(\x0b\x32\x1a.datafusion.WindowExprNodeH\x00\x12>\n\x12\x61ggregate_udf_expr\x18\x13 \x01(\x0b\x32 .datafusion.AggregateUDFExprNodeH\x00\x12\x38\n\x0fscalar_udf_expr\x18\x14 \x01(\x0b\x32\x1d.datafusion.ScalarUDFExprNodeH\x00\x12\x38\n\x11get_indexed_field\x18\x15 \x01(\x0b\x32\x1b.datafusion.GetIndexedFieldH\x00\x12\x33\n\x0cgrouping_set\x18\x16 \x01(\x0b\x32\x1b.datafusion.GroupingSetNodeH\x00\x12$\n\x04\x63ube\x18\x17 \x01(\x0b\x32\x14.datafusion.CubeNodeH\x00\x12(\n\x06rollup\x18\x18 \x01(\x0b\x32\x16.datafusion.RollupNodeH\x00\x12%\n\x07is_true\x18\x19 \x01(\x0b\x32\x12.datafusion.IsTrueH\x00\x12\'\n\x08is_false\x18\x1a \x01(\x0b\x32\x13.datafusion.IsFalseH\x00\x12+\n\nis_unknown\x18\x1b \x01(\x0b\x32\x15.datafusion.IsUnknownH\x00\x12,\n\x0bis_not_true\x18\x1c \x01(\x0b\x32\x15.datafusion.IsNotTrueH\x00\x12.\n\x0cis_not_false\x18\x1d \x01(\x0b\x32\x16.datafusion.IsNotFalseH\x00\x12\x32\n\x0eis_not_unknown\x18\x1e \x01(\x0b\x32\x18.datafusion.IsNotUnknownH\x00\x12$\n\x04like\x18\x1f \x01(\x0b\x32\x14.datafusion.LikeNodeH\x00\x12&\n\x05ilike\x18 \x01(\x0b\x32\x15.datafusion.ILikeNodeH\x00\x12/\n\nsimilar_to\x18! \x01(\x0b\x32\x19.datafusion.SimilarToNodeH\x00\x12\x32\n\x0bplaceholder\x18\" \x01(\x0b\x32\x1b.datafusion.PlaceholderNodeH\x00\x42\n\n\x08\x45xprType\"G\n\x0fPlaceholderNode\x12\n\n\x02id\x18\x01 \x01(\t\x12(\n\tdata_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"<\n\x0fLogicalExprList\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"<\n\x0fGroupingSetNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprList\"5\n\x08\x43ubeNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"7\n\nRollupNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"b\n\x0fGetIndexedField\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12$\n\x03key\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"3\n\x06IsNull\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsNotNull\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"3\n\x06IsTrue\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"4\n\x07IsFalse\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsUnknown\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsNotTrue\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"7\n\nIsNotFalse\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"9\n\x0cIsNotUnknown\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"0\n\x03Not\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"E\n\tAliasNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\r\n\x05\x61lias\x18\x02 \x01(\t\"K\n\x0e\x42inaryExprNode\x12-\n\x08operands\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"9\n\x0cNegativeNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"s\n\nInListNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\x04list\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x07negated\x18\x03 \x01(\x08\"h\n\x12ScalarFunctionNode\x12\'\n\x03\x66un\x18\x01 \x01(\x0e\x32\x1a.datafusion.ScalarFunction\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xe2\x01\n\x11\x41ggregateExprNode\x12\x34\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunction\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x10\n\x08\x64istinct\x18\x03 \x01(\x08\x12+\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xaf\x01\n\x14\x41ggregateUDFExprNode\x12\x10\n\x08\x66un_name\x18\x01 \x01(\t\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12+\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x04 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"P\n\x11ScalarUDFExprNode\x12\x10\n\x08\x66un_name\x18\x01 \x01(\t\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xf7\x02\n\x0eWindowExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12>\n\x11\x62uilt_in_function\x18\x02 \x01(\x0e\x32!.datafusion.BuiltInWindowFunctionH\x00\x12\x0e\n\x04udaf\x18\x03 \x01(\tH\x00\x12\x0e\n\x04udwf\x18\t \x01(\tH\x00\x12)\n\x04\x65xpr\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x31\n\x0cpartition_by\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x0cwindow_frame\x18\x08 \x01(\x0b\x32\x17.datafusion.WindowFrameB\x11\n\x0fwindow_function\"\x9e\x01\n\x0b\x42\x65tweenNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x07negated\x18\x02 \x01(\x08\x12(\n\x03low\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\x04high\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"\x89\x01\n\x08LikeNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x8a\x01\n\tILikeNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x8e\x01\n\rSimilarToNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x93\x01\n\x08\x43\x61seNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x0ewhen_then_expr\x18\x02 \x03(\x0b\x32\x14.datafusion.WhenThen\x12.\n\telse_expr\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"j\n\x08WhenThen\x12.\n\twhen_expr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12.\n\tthen_expr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"`\n\x08\x43\x61stNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"c\n\x0bTryCastNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"[\n\x0cSortExprNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0b\n\x03\x61sc\x18\x02 \x01(\x08\x12\x13\n\x0bnulls_first\x18\x03 \x01(\x08\"\xb6\x01\n\x0bWindowFrame\x12\x38\n\x12window_frame_units\x18\x01 \x01(\x0e\x32\x1c.datafusion.WindowFrameUnits\x12\x31\n\x0bstart_bound\x18\x02 \x01(\x0b\x32\x1c.datafusion.WindowFrameBound\x12-\n\x05\x62ound\x18\x03 \x01(\x0b\x32\x1c.datafusion.WindowFrameBoundH\x00\x42\x0b\n\tend_bound\"\x83\x01\n\x10WindowFrameBound\x12\x41\n\x17window_frame_bound_type\x18\x01 \x01(\x0e\x32 .datafusion.WindowFrameBoundType\x12,\n\x0b\x62ound_value\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"\x91\x01\n\x06Schema\x12\"\n\x07\x63olumns\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\x12\x32\n\x08metadata\x18\x02 \x03(\x0b\x32 .datafusion.Schema.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\xdb\x01\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\x12\x10\n\x08nullable\x18\x03 \x01(\x08\x12#\n\x08\x63hildren\x18\x04 \x03(\x0b\x32\x11.datafusion.Field\x12\x31\n\x08metadata\x18\x05 \x03(\x0b\x32\x1f.datafusion.Field.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"!\n\x0f\x46ixedSizeBinary\x12\x0e\n\x06length\x18\x01 \x01(\x05\"F\n\tTimestamp\x12\'\n\ttime_unit\x18\x01 \x01(\x0e\x32\x14.datafusion.TimeUnit\x12\x10\n\x08timezone\x18\x02 \x01(\t\"7\n\x07\x44\x65\x63imal\x12\x11\n\tprecision\x18\x03 \x01(\r\x12\r\n\x05scale\x18\x04 \x01(\x05J\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03\"-\n\x04List\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\"I\n\rFixedSizeList\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12\x11\n\tlist_size\x18\x02 \x01(\x05\"V\n\nDictionary\x12\"\n\x03key\x18\x01 \x01(\x0b\x32\x15.datafusion.ArrowType\x12$\n\x05value\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"4\n\x06Struct\x12*\n\x0fsub_field_types\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\"A\n\x03Map\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12\x13\n\x0bkeys_sorted\x18\x02 \x01(\x08\"l\n\x05Union\x12&\n\x0bunion_types\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\x12)\n\nunion_mode\x18\x02 \x01(\x0e\x32\x15.datafusion.UnionMode\x12\x10\n\x08type_ids\x18\x03 \x03(\x05\"m\n\x0fScalarListValue\x12\x0f\n\x07is_null\x18\x03 \x01(\x08\x12 \n\x05\x66ield\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12\'\n\x06values\x18\x02 \x03(\x0b\x32\x17.datafusion.ScalarValue\"_\n\x11ScalarTime32Value\x12\x1d\n\x13time32_second_value\x18\x01 \x01(\x05H\x00\x12\"\n\x18time32_millisecond_value\x18\x02 \x01(\x05H\x00\x42\x07\n\x05value\"c\n\x11ScalarTime64Value\x12\"\n\x18time64_microsecond_value\x18\x01 \x01(\x03H\x00\x12!\n\x17time64_nanosecond_value\x18\x02 \x01(\x03H\x00\x42\x07\n\x05value\"\xb3\x01\n\x14ScalarTimestampValue\x12 \n\x16time_microsecond_value\x18\x01 \x01(\x03H\x00\x12\x1f\n\x15time_nanosecond_value\x18\x02 \x01(\x03H\x00\x12\x1b\n\x11time_second_value\x18\x03 \x01(\x03H\x00\x12 \n\x16time_millisecond_value\x18\x04 \x01(\x03H\x00\x12\x10\n\x08timezone\x18\x05 \x01(\tB\x07\n\x05value\"j\n\x15ScalarDictionaryValue\x12)\n\nindex_type\x18\x01 \x01(\x0b\x32\x15.datafusion.ArrowType\x12&\n\x05value\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"H\n\x19IntervalMonthDayNanoValue\x12\x0e\n\x06months\x18\x01 \x01(\x05\x12\x0c\n\x04\x64\x61ys\x18\x02 \x01(\x05\x12\r\n\x05nanos\x18\x03 \x01(\x03\"_\n\x0bStructValue\x12-\n\x0c\x66ield_values\x18\x02 \x03(\x0b\x32\x17.datafusion.ScalarValue\x12!\n\x06\x66ields\x18\x03 \x03(\x0b\x32\x11.datafusion.Field\"7\n\x15ScalarFixedSizeBinary\x12\x0e\n\x06values\x18\x01 \x01(\x0c\x12\x0e\n\x06length\x18\x02 \x01(\x05\"\xcf\t\n\x0bScalarValue\x12+\n\nnull_value\x18! \x01(\x0b\x32\x15.datafusion.ArrowTypeH\x00\x12\x14\n\nbool_value\x18\x01 \x01(\x08H\x00\x12\x14\n\nutf8_value\x18\x02 \x01(\tH\x00\x12\x1a\n\x10large_utf8_value\x18\x03 \x01(\tH\x00\x12\x14\n\nint8_value\x18\x04 \x01(\x05H\x00\x12\x15\n\x0bint16_value\x18\x05 \x01(\x05H\x00\x12\x15\n\x0bint32_value\x18\x06 \x01(\x05H\x00\x12\x15\n\x0bint64_value\x18\x07 \x01(\x03H\x00\x12\x15\n\x0buint8_value\x18\x08 \x01(\rH\x00\x12\x16\n\x0cuint16_value\x18\t \x01(\rH\x00\x12\x16\n\x0cuint32_value\x18\n \x01(\rH\x00\x12\x16\n\x0cuint64_value\x18\x0b \x01(\x04H\x00\x12\x17\n\rfloat32_value\x18\x0c \x01(\x02H\x00\x12\x17\n\rfloat64_value\x18\r \x01(\x01H\x00\x12\x17\n\rdate_32_value\x18\x0e \x01(\x05H\x00\x12\x35\n\x0ctime32_value\x18\x0f \x01(\x0b\x32\x1d.datafusion.ScalarTime32ValueH\x00\x12\x31\n\nlist_value\x18\x11 \x01(\x0b\x32\x1b.datafusion.ScalarListValueH\x00\x12\x32\n\x10\x64\x65\x63imal128_value\x18\x14 \x01(\x0b\x32\x16.datafusion.Decimal128H\x00\x12\x17\n\rdate_64_value\x18\x15 \x01(\x03H\x00\x12\"\n\x18interval_yearmonth_value\x18\x18 \x01(\x05H\x00\x12 \n\x16interval_daytime_value\x18\x19 \x01(\x03H\x00\x12\x1f\n\x15\x64uration_second_value\x18# \x01(\x03H\x00\x12$\n\x1a\x64uration_millisecond_value\x18$ \x01(\x03H\x00\x12$\n\x1a\x64uration_microsecond_value\x18% \x01(\x03H\x00\x12#\n\x19\x64uration_nanosecond_value\x18& \x01(\x03H\x00\x12;\n\x0ftimestamp_value\x18\x1a \x01(\x0b\x32 .datafusion.ScalarTimestampValueH\x00\x12=\n\x10\x64ictionary_value\x18\x1b \x01(\x0b\x32!.datafusion.ScalarDictionaryValueH\x00\x12\x16\n\x0c\x62inary_value\x18\x1c \x01(\x0cH\x00\x12\x1c\n\x12large_binary_value\x18\x1d \x01(\x0cH\x00\x12\x35\n\x0ctime64_value\x18\x1e \x01(\x0b\x32\x1d.datafusion.ScalarTime64ValueH\x00\x12H\n\x17interval_month_day_nano\x18\x1f \x01(\x0b\x32%.datafusion.IntervalMonthDayNanoValueH\x00\x12/\n\x0cstruct_value\x18 \x01(\x0b\x32\x17.datafusion.StructValueH\x00\x12\x44\n\x17\x66ixed_size_binary_value\x18\" \x01(\x0b\x32!.datafusion.ScalarFixedSizeBinaryH\x00\x42\x07\n\x05valueJ\x04\x08\x13\x10\x14\"1\n\nDecimal128\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\t\n\x01p\x18\x02 \x01(\x03\x12\t\n\x01s\x18\x03 \x01(\x03\"\x94\x0b\n\tArrowType\x12(\n\x04NONE\x18\x01 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04\x42OOL\x18\x02 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05UINT8\x18\x03 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04INT8\x18\x04 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT16\x18\x05 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT16\x18\x06 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT32\x18\x07 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT32\x18\x08 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT64\x18\t \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT64\x18\n \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT16\x18\x0b \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT32\x18\x0c \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT64\x18\r \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04UTF8\x18\x0e \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12.\n\nLARGE_UTF8\x18 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x42INARY\x18\x0f \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x1b\n\x11\x46IXED_SIZE_BINARY\x18\x10 \x01(\x05H\x00\x12\x30\n\x0cLARGE_BINARY\x18\x1f \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x44\x41TE32\x18\x11 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x44\x41TE64\x18\x12 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x08\x44URATION\x18\x13 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12*\n\tTIMESTAMP\x18\x14 \x01(\x0b\x32\x15.datafusion.TimestampH\x00\x12&\n\x06TIME32\x18\x15 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12&\n\x06TIME64\x18\x16 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12,\n\x08INTERVAL\x18\x17 \x01(\x0e\x32\x18.datafusion.IntervalUnitH\x00\x12&\n\x07\x44\x45\x43IMAL\x18\x18 \x01(\x0b\x32\x13.datafusion.DecimalH\x00\x12 \n\x04LIST\x18\x19 \x01(\x0b\x32\x10.datafusion.ListH\x00\x12&\n\nLARGE_LIST\x18\x1a \x01(\x0b\x32\x10.datafusion.ListH\x00\x12\x34\n\x0f\x46IXED_SIZE_LIST\x18\x1b \x01(\x0b\x32\x19.datafusion.FixedSizeListH\x00\x12$\n\x06STRUCT\x18\x1c \x01(\x0b\x32\x12.datafusion.StructH\x00\x12\"\n\x05UNION\x18\x1d \x01(\x0b\x32\x11.datafusion.UnionH\x00\x12,\n\nDICTIONARY\x18\x1e \x01(\x0b\x32\x16.datafusion.DictionaryH\x00\x12\x1e\n\x03MAP\x18! \x01(\x0b\x32\x0f.datafusion.MapH\x00\x42\x11\n\x0f\x61rrow_type_enum\"\x0e\n\x0c\x45mptyMessage\"0\n\x17\x41nalyzedLogicalPlanType\x12\x15\n\ranalyzer_name\x18\x01 \x01(\t\"2\n\x18OptimizedLogicalPlanType\x12\x16\n\x0eoptimizer_name\x18\x01 \x01(\t\"3\n\x19OptimizedPhysicalPlanType\x12\x16\n\x0eoptimizer_name\x18\x01 \x01(\t\"\x8a\x04\n\x08PlanType\x12\x36\n\x12InitialLogicalPlan\x18\x01 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x42\n\x13\x41nalyzedLogicalPlan\x18\x07 \x01(\x0b\x32#.datafusion.AnalyzedLogicalPlanTypeH\x00\x12<\n\x18\x46inalAnalyzedLogicalPlan\x18\x08 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x44\n\x14OptimizedLogicalPlan\x18\x02 \x01(\x0b\x32$.datafusion.OptimizedLogicalPlanTypeH\x00\x12\x34\n\x10\x46inalLogicalPlan\x18\x03 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x37\n\x13InitialPhysicalPlan\x18\x04 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x46\n\x15OptimizedPhysicalPlan\x18\x05 \x01(\x0b\x32%.datafusion.OptimizedPhysicalPlanTypeH\x00\x12\x35\n\x11\x46inalPhysicalPlan\x18\x06 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x42\x10\n\x0eplan_type_enum\"H\n\x0fStringifiedPlan\x12\'\n\tplan_type\x18\x01 \x01(\x0b\x32\x14.datafusion.PlanType\x12\x0c\n\x04plan\x18\x02 \x01(\t\"#\n\x12\x42\x61reTableReference\x12\r\n\x05table\x18\x01 \x01(\t\"6\n\x15PartialTableReference\x12\x0e\n\x06schema\x18\x01 \x01(\t\x12\r\n\x05table\x18\x02 \x01(\t\"D\n\x12\x46ullTableReference\x12\x0f\n\x07\x63\x61talog\x18\x01 \x01(\t\x12\x0e\n\x06schema\x18\x02 \x01(\t\x12\r\n\x05table\x18\x03 \x01(\t\"\xc3\x01\n\x13OwnedTableReference\x12.\n\x04\x62\x61re\x18\x01 \x01(\x0b\x32\x1e.datafusion.BareTableReferenceH\x00\x12\x34\n\x07partial\x18\x02 \x01(\x0b\x32!.datafusion.PartialTableReferenceH\x00\x12.\n\x04\x66ull\x18\x03 \x01(\x0b\x32\x1e.datafusion.FullTableReferenceH\x00\x42\x16\n\x14table_reference_enum\"\x8a\t\n\x10PhysicalPlanNode\x12\x37\n\x0cparquet_scan\x18\x01 \x01(\x0b\x32\x1f.datafusion.ParquetScanExecNodeH\x00\x12/\n\x08\x63sv_scan\x18\x02 \x01(\x0b\x32\x1b.datafusion.CsvScanExecNodeH\x00\x12*\n\x05\x65mpty\x18\x03 \x01(\x0b\x32\x19.datafusion.EmptyExecNodeH\x00\x12\x34\n\nprojection\x18\x04 \x01(\x0b\x32\x1e.datafusion.ProjectionExecNodeH\x00\x12\x37\n\x0cglobal_limit\x18\x06 \x01(\x0b\x32\x1f.datafusion.GlobalLimitExecNodeH\x00\x12\x35\n\x0blocal_limit\x18\x07 \x01(\x0b\x32\x1e.datafusion.LocalLimitExecNodeH\x00\x12\x32\n\taggregate\x18\x08 \x01(\x0b\x32\x1d.datafusion.AggregateExecNodeH\x00\x12\x31\n\thash_join\x18\t \x01(\x0b\x32\x1c.datafusion.HashJoinExecNodeH\x00\x12(\n\x04sort\x18\n \x01(\x0b\x32\x18.datafusion.SortExecNodeH\x00\x12?\n\x10\x63oalesce_batches\x18\x0b \x01(\x0b\x32#.datafusion.CoalesceBatchesExecNodeH\x00\x12,\n\x06\x66ilter\x18\x0c \x01(\x0b\x32\x1a.datafusion.FilterExecNodeH\x00\x12\x37\n\x05merge\x18\r \x01(\x0b\x32&.datafusion.CoalescePartitionsExecNodeH\x00\x12\x36\n\x0brepartition\x18\x0e \x01(\x0b\x32\x1f.datafusion.RepartitionExecNodeH\x00\x12/\n\x06window\x18\x0f \x01(\x0b\x32\x1d.datafusion.WindowAggExecNodeH\x00\x12\x33\n\ncross_join\x18\x10 \x01(\x0b\x32\x1d.datafusion.CrossJoinExecNodeH\x00\x12\x31\n\tavro_scan\x18\x11 \x01(\x0b\x32\x1c.datafusion.AvroScanExecNodeH\x00\x12\x36\n\textension\x18\x12 \x01(\x0b\x32!.datafusion.PhysicalExtensionNodeH\x00\x12*\n\x05union\x18\x13 \x01(\x0b\x32\x19.datafusion.UnionExecNodeH\x00\x12.\n\x07\x65xplain\x18\x14 \x01(\x0b\x32\x1b.datafusion.ExplainExecNodeH\x00\x12H\n\x15sort_preserving_merge\x18\x15 \x01(\x0b\x32\'.datafusion.SortPreservingMergeExecNodeH\x00\x12>\n\x10nested_loop_join\x18\x16 \x01(\x0b\x32\".datafusion.NestedLoopJoinExecNodeH\x00\x42\x12\n\x10PhysicalPlanType\"S\n\x15PhysicalExtensionNode\x12\x0c\n\x04node\x18\x01 \x01(\x0c\x12,\n\x06inputs\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"\xcb\x08\n\x10PhysicalExprNode\x12,\n\x06\x63olumn\x18\x01 \x01(\x0b\x32\x1a.datafusion.PhysicalColumnH\x00\x12*\n\x07literal\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValueH\x00\x12\x39\n\x0b\x62inary_expr\x18\x03 \x01(\x0b\x32\".datafusion.PhysicalBinaryExprNodeH\x00\x12?\n\x0e\x61ggregate_expr\x18\x04 \x01(\x0b\x32%.datafusion.PhysicalAggregateExprNodeH\x00\x12\x32\n\x0cis_null_expr\x18\x05 \x01(\x0b\x32\x1a.datafusion.PhysicalIsNullH\x00\x12\x39\n\x10is_not_null_expr\x18\x06 \x01(\x0b\x32\x1d.datafusion.PhysicalIsNotNullH\x00\x12+\n\x08not_expr\x18\x07 \x01(\x0b\x32\x17.datafusion.PhysicalNotH\x00\x12-\n\x05\x63\x61se_\x18\x08 \x01(\x0b\x32\x1c.datafusion.PhysicalCaseNodeH\x00\x12,\n\x04\x63\x61st\x18\t \x01(\x0b\x32\x1c.datafusion.PhysicalCastNodeH\x00\x12\x30\n\x04sort\x18\n \x01(\x0b\x32 .datafusion.PhysicalSortExprNodeH\x00\x12\x34\n\x08negative\x18\x0b \x01(\x0b\x32 .datafusion.PhysicalNegativeNodeH\x00\x12\x31\n\x07in_list\x18\x0c \x01(\x0b\x32\x1e.datafusion.PhysicalInListNodeH\x00\x12\x41\n\x0fscalar_function\x18\r \x01(\x0b\x32&.datafusion.PhysicalScalarFunctionNodeH\x00\x12\x33\n\x08try_cast\x18\x0e \x01(\x0b\x32\x1f.datafusion.PhysicalTryCastNodeH\x00\x12\x39\n\x0bwindow_expr\x18\x0f \x01(\x0b\x32\".datafusion.PhysicalWindowExprNodeH\x00\x12\x37\n\nscalar_udf\x18\x10 \x01(\x0b\x32!.datafusion.PhysicalScalarUdfNodeH\x00\x12O\n\x17\x64\x61te_time_interval_expr\x18\x11 \x01(\x0b\x32,.datafusion.PhysicalDateTimeIntervalExprNodeH\x00\x12\x35\n\tlike_expr\x18\x12 \x01(\x0b\x32 .datafusion.PhysicalLikeExprNodeH\x00\x12M\n\x16get_indexed_field_expr\x18\x13 \x01(\x0b\x32+.datafusion.PhysicalGetIndexedFieldExprNodeH\x00\x42\n\n\x08\x45xprType\"}\n\x15PhysicalScalarUdfNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12*\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x0breturn_type\x18\x04 \x01(\x0b\x32\x15.datafusion.ArrowType\"\x84\x02\n\x19PhysicalAggregateExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12$\n\x1auser_defined_aggr_function\x18\x04 \x01(\tH\x00\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x36\n\x0cordering_req\x18\x05 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\x12\x10\n\x08\x64istinct\x18\x03 \x01(\x08\x42\x13\n\x11\x41ggregateFunction\"\xcf\x01\n\x16PhysicalWindowExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12>\n\x11\x62uilt_in_function\x18\x02 \x01(\x0e\x32!.datafusion.BuiltInWindowFunctionH\x00\x12*\n\x04\x65xpr\x18\x04 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNodeB\x11\n\x0fwindow_function\"<\n\x0ePhysicalIsNull\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"?\n\x11PhysicalIsNotNull\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"9\n\x0bPhysicalNot\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"N\n\x11PhysicalAliasNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x61lias\x18\x02 \x01(\t\"v\n\x16PhysicalBinaryExprNode\x12\'\n\x01l\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x01r\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"\x80\x01\n PhysicalDateTimeIntervalExprNode\x12\'\n\x01l\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x01r\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"\x9c\x01\n\x14PhysicalLikeExprNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12\x18\n\x10\x63\x61se_insensitive\x18\x02 \x01(\x08\x12*\n\x04\x65xpr\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12-\n\x07pattern\x18\x04 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"d\n\x14PhysicalSortExprNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0b\n\x03\x61sc\x18\x02 \x01(\x08\x12\x13\n\x0bnulls_first\x18\x03 \x01(\x08\"t\n\x10PhysicalWhenThen\x12/\n\twhen_expr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\tthen_expr\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"}\n\x12PhysicalInListNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x04list\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0f\n\x07negated\x18\x03 \x01(\x08\"\xa5\x01\n\x10PhysicalCaseNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x34\n\x0ewhen_then_expr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalWhenThen\x12/\n\telse_expr\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"\xab\x01\n\x1aPhysicalScalarFunctionNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\'\n\x03\x66un\x18\x02 \x01(\x0e\x32\x1a.datafusion.ScalarFunction\x12*\n\x04\x61rgs\x18\x03 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x0breturn_type\x18\x04 \x01(\x0b\x32\x15.datafusion.ArrowType\"l\n\x13PhysicalTryCastNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"i\n\x10PhysicalCastNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"B\n\x14PhysicalNegativeNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"i\n\x0e\x46ilterExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"7\n\tFileGroup\x12*\n\x05\x66iles\x18\x01 \x03(\x0b\x32\x1b.datafusion.PartitionedFile\"\x1a\n\tScanLimit\x12\r\n\x05limit\x18\x01 \x01(\r\"d\n\x1ePhysicalSortExprNodeCollection\x12\x42\n\x18physical_sort_expr_nodes\x18\x01 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\"\xcb\x02\n\x10\x46ileScanExecConf\x12*\n\x0b\x66ile_groups\x18\x01 \x03(\x0b\x32\x15.datafusion.FileGroup\x12\"\n\x06schema\x18\x02 \x01(\x0b\x32\x12.datafusion.Schema\x12\x12\n\nprojection\x18\x04 \x03(\r\x12$\n\x05limit\x18\x05 \x01(\x0b\x32\x15.datafusion.ScanLimit\x12*\n\nstatistics\x18\x06 \x01(\x0b\x32\x16.datafusion.Statistics\x12\x1c\n\x14table_partition_cols\x18\x07 \x03(\t\x12\x18\n\x10object_store_url\x18\x08 \x01(\t\x12\x43\n\x0foutput_ordering\x18\t \x03(\x0b\x32*.datafusion.PhysicalSortExprNodeCollectionJ\x04\x08\n\x10\x0b\"}\n\x13ParquetScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\x12/\n\tpredicate\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNodeJ\x04\x08\x02\x10\x03\"i\n\x0f\x43svScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\x12\x12\n\nhas_header\x18\x02 \x01(\x08\x12\x11\n\tdelimiter\x18\x03 \x01(\t\"C\n\x10\x41vroScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\"\xa9\x02\n\x10HashJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x1e\n\x02on\x18\x03 \x03(\x0b\x32\x12.datafusion.JoinOn\x12\'\n\tjoin_type\x18\x04 \x01(\x0e\x32\x14.datafusion.JoinType\x12\x31\n\x0epartition_mode\x18\x06 \x01(\x0e\x32\x19.datafusion.PartitionMode\x12\x18\n\x10null_equals_null\x18\x07 \x01(\x08\x12&\n\x06\x66ilter\x18\x08 \x01(\x0b\x32\x16.datafusion.JoinFilter\"=\n\rUnionExecNode\x12,\n\x06inputs\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"~\n\x0f\x45xplainExecNode\x12\"\n\x06schema\x18\x01 \x01(\x0b\x32\x12.datafusion.Schema\x12\x36\n\x11stringified_plans\x18\x02 \x03(\x0b\x32\x1b.datafusion.StringifiedPlan\x12\x0f\n\x07verbose\x18\x03 \x01(\x08\"l\n\x11\x43rossJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"-\n\x0ePhysicalColumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\r\n\x05index\x18\x02 \x01(\r\"]\n\x06JoinOn\x12(\n\x04left\x18\x01 \x01(\x0b\x32\x1a.datafusion.PhysicalColumn\x12)\n\x05right\x18\x02 \x01(\x0b\x32\x1a.datafusion.PhysicalColumn\"L\n\rEmptyExecNode\x12\x17\n\x0fproduce_one_row\x18\x01 \x01(\x08\x12\"\n\x06schema\x18\x02 \x01(\x0b\x32\x12.datafusion.Schema\"\x80\x01\n\x12ProjectionExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x11\n\texpr_name\x18\x03 \x03(\t\"\xb7\x01\n\x11WindowAggExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x31\n\x0bwindow_expr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x18\n\x10window_expr_name\x18\x03 \x03(\t\x12(\n\x0cinput_schema\x18\x04 \x01(\x0b\x32\x12.datafusion.Schema\"9\n\x0bMaybeFilter\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"M\n\x16MaybePhysicalSortExprs\x12\x33\n\tsort_expr\x18\x01 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\"\xd1\x03\n\x11\x41ggregateExecNode\x12\x30\n\ngroup_expr\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\taggr_expr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x04mode\x18\x03 \x01(\x0e\x32\x19.datafusion.AggregateMode\x12+\n\x05input\x18\x04 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x17\n\x0fgroup_expr_name\x18\x05 \x03(\t\x12\x16\n\x0e\x61ggr_expr_name\x18\x06 \x03(\t\x12(\n\x0cinput_schema\x18\x07 \x01(\x0b\x32\x12.datafusion.Schema\x12/\n\tnull_expr\x18\x08 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0e\n\x06groups\x18\t \x03(\x08\x12,\n\x0b\x66ilter_expr\x18\n \x03(\x0b\x32\x17.datafusion.MaybeFilter\x12\x39\n\rorder_by_expr\x18\x0b \x03(\x0b\x32\".datafusion.MaybePhysicalSortExprs\"_\n\x13GlobalLimitExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x0c\n\x04skip\x18\x02 \x01(\r\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"P\n\x12LocalLimitExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\r\n\x05\x66\x65tch\x18\x02 \x01(\r\"\x95\x01\n\x0cSortExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\x12\x1d\n\x15preserve_partitioning\x18\x04 \x01(\x08\"\x85\x01\n\x1bSortPreservingMergeExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"\xc2\x01\n\x16NestedLoopJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\'\n\tjoin_type\x18\x03 \x01(\x0e\x32\x14.datafusion.JoinType\x12&\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x16.datafusion.JoinFilter\"a\n\x17\x43oalesceBatchesExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x19\n\x11target_batch_size\x18\x02 \x01(\r\"I\n\x1a\x43oalescePartitionsExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"c\n\x17PhysicalHashRepartition\x12/\n\thash_expr\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x17\n\x0fpartition_count\x18\x02 \x01(\x04\"\xb5\x01\n\x13RepartitionExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x15\n\x0bround_robin\x18\x02 \x01(\x04H\x00\x12\x33\n\x04hash\x18\x03 \x01(\x0b\x32#.datafusion.PhysicalHashRepartitionH\x00\x12\x11\n\x07unknown\x18\x04 \x01(\x04H\x00\x42\x12\n\x10partition_method\"\x93\x01\n\nJoinFilter\x12\x30\n\nexpression\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\x0e\x63olumn_indices\x18\x02 \x03(\x0b\x32\x17.datafusion.ColumnIndex\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\"@\n\x0b\x43olumnIndex\x12\r\n\x05index\x18\x01 \x01(\r\x12\"\n\x04side\x18\x02 \x01(\x0e\x32\x14.datafusion.JoinSide\"\xa0\x01\n\x0fPartitionedFile\x12\x0c\n\x04path\x18\x01 \x01(\t\x12\x0c\n\x04size\x18\x02 \x01(\x04\x12\x18\n\x10last_modified_ns\x18\x03 \x01(\x04\x12\x31\n\x10partition_values\x18\x04 \x03(\x0b\x32\x17.datafusion.ScalarValue\x12$\n\x05range\x18\x05 \x01(\x0b\x32\x15.datafusion.FileRange\"\'\n\tFileRange\x12\r\n\x05start\x18\x01 \x01(\x03\x12\x0b\n\x03\x65nd\x18\x02 \x01(\x03\"y\n\x0ePartitionStats\x12\x10\n\x08num_rows\x18\x01 \x01(\x03\x12\x13\n\x0bnum_batches\x18\x02 \x01(\x03\x12\x11\n\tnum_bytes\x18\x03 \x01(\x03\x12-\n\x0c\x63olumn_stats\x18\x04 \x03(\x0b\x32\x17.datafusion.ColumnStats\"x\n\nStatistics\x12\x10\n\x08num_rows\x18\x01 \x01(\x03\x12\x17\n\x0ftotal_byte_size\x18\x02 \x01(\x03\x12-\n\x0c\x63olumn_stats\x18\x03 \x03(\x0b\x32\x17.datafusion.ColumnStats\x12\x10\n\x08is_exact\x18\x04 \x01(\x08\"\x91\x01\n\x0b\x43olumnStats\x12*\n\tmin_value\x18\x01 \x01(\x0b\x32\x17.datafusion.ScalarValue\x12*\n\tmax_value\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\x12\x12\n\nnull_count\x18\x03 \x01(\r\x12\x16\n\x0e\x64istinct_count\x18\x04 \x01(\r\"r\n\x1fPhysicalGetIndexedFieldExprNode\x12)\n\x03\x61rg\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12$\n\x03key\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue*n\n\x08JoinType\x12\t\n\x05INNER\x10\x00\x12\x08\n\x04LEFT\x10\x01\x12\t\n\x05RIGHT\x10\x02\x12\x08\n\x04\x46ULL\x10\x03\x12\x0c\n\x08LEFTSEMI\x10\x04\x12\x0c\n\x08LEFTANTI\x10\x05\x12\r\n\tRIGHTSEMI\x10\x06\x12\r\n\tRIGHTANTI\x10\x07*#\n\x0eJoinConstraint\x12\x06\n\x02ON\x10\x00\x12\t\n\x05USING\x10\x01*\xe6\n\n\x0eScalarFunction\x12\x07\n\x03\x41\x62s\x10\x00\x12\x08\n\x04\x41\x63os\x10\x01\x12\x08\n\x04\x41sin\x10\x02\x12\x08\n\x04\x41tan\x10\x03\x12\t\n\x05\x41scii\x10\x04\x12\x08\n\x04\x43\x65il\x10\x05\x12\x07\n\x03\x43os\x10\x06\x12\n\n\x06\x44igest\x10\x07\x12\x07\n\x03\x45xp\x10\x08\x12\t\n\x05\x46loor\x10\t\x12\x06\n\x02Ln\x10\n\x12\x07\n\x03Log\x10\x0b\x12\t\n\x05Log10\x10\x0c\x12\x08\n\x04Log2\x10\r\x12\t\n\x05Round\x10\x0e\x12\n\n\x06Signum\x10\x0f\x12\x07\n\x03Sin\x10\x10\x12\x08\n\x04Sqrt\x10\x11\x12\x07\n\x03Tan\x10\x12\x12\t\n\x05Trunc\x10\x13\x12\t\n\x05\x41rray\x10\x14\x12\x0f\n\x0bRegexpMatch\x10\x15\x12\r\n\tBitLength\x10\x16\x12\t\n\x05\x42trim\x10\x17\x12\x13\n\x0f\x43haracterLength\x10\x18\x12\x07\n\x03\x43hr\x10\x19\x12\n\n\x06\x43oncat\x10\x1a\x12\x17\n\x13\x43oncatWithSeparator\x10\x1b\x12\x0c\n\x08\x44\x61tePart\x10\x1c\x12\r\n\tDateTrunc\x10\x1d\x12\x0b\n\x07InitCap\x10\x1e\x12\x08\n\x04Left\x10\x1f\x12\x08\n\x04Lpad\x10 \x12\t\n\x05Lower\x10!\x12\t\n\x05Ltrim\x10\"\x12\x07\n\x03MD5\x10#\x12\n\n\x06NullIf\x10$\x12\x0f\n\x0bOctetLength\x10%\x12\n\n\x06Random\x10&\x12\x11\n\rRegexpReplace\x10\'\x12\n\n\x06Repeat\x10(\x12\x0b\n\x07Replace\x10)\x12\x0b\n\x07Reverse\x10*\x12\t\n\x05Right\x10+\x12\x08\n\x04Rpad\x10,\x12\t\n\x05Rtrim\x10-\x12\n\n\x06SHA224\x10.\x12\n\n\x06SHA256\x10/\x12\n\n\x06SHA384\x10\x30\x12\n\n\x06SHA512\x10\x31\x12\r\n\tSplitPart\x10\x32\x12\x0e\n\nStartsWith\x10\x33\x12\n\n\x06Strpos\x10\x34\x12\n\n\x06Substr\x10\x35\x12\t\n\x05ToHex\x10\x36\x12\x0f\n\x0bToTimestamp\x10\x37\x12\x15\n\x11ToTimestampMillis\x10\x38\x12\x15\n\x11ToTimestampMicros\x10\x39\x12\x16\n\x12ToTimestampSeconds\x10:\x12\x07\n\x03Now\x10;\x12\r\n\tTranslate\x10<\x12\x08\n\x04Trim\x10=\x12\t\n\x05Upper\x10>\x12\x0c\n\x08\x43oalesce\x10?\x12\t\n\x05Power\x10@\x12\r\n\tStructFun\x10\x41\x12\x10\n\x0c\x46romUnixtime\x10\x42\x12\t\n\x05\x41tan2\x10\x43\x12\x0b\n\x07\x44\x61teBin\x10\x44\x12\x0f\n\x0b\x41rrowTypeof\x10\x45\x12\x0f\n\x0b\x43urrentDate\x10\x46\x12\x0f\n\x0b\x43urrentTime\x10G\x12\x08\n\x04Uuid\x10H\x12\x08\n\x04\x43\x62rt\x10I\x12\t\n\x05\x41\x63osh\x10J\x12\t\n\x05\x41sinh\x10K\x12\t\n\x05\x41tanh\x10L\x12\x08\n\x04Sinh\x10M\x12\x08\n\x04\x43osh\x10N\x12\x08\n\x04Tanh\x10O\x12\x06\n\x02Pi\x10P\x12\x0b\n\x07\x44\x65grees\x10Q\x12\x0b\n\x07Radians\x10R\x12\r\n\tFactorial\x10S\x12\x07\n\x03Lcm\x10T\x12\x07\n\x03Gcd\x10U\x12\x0f\n\x0b\x41rrayAppend\x10V\x12\x0f\n\x0b\x41rrayConcat\x10W\x12\r\n\tArrayDims\x10X\x12\r\n\tArrayFill\x10Y\x12\x0f\n\x0b\x41rrayLength\x10Z\x12\x0e\n\nArrayNdims\x10[\x12\x11\n\rArrayPosition\x10\\\x12\x12\n\x0e\x41rrayPositions\x10]\x12\x10\n\x0c\x41rrayPrepend\x10^\x12\x0f\n\x0b\x41rrayRemove\x10_\x12\x10\n\x0c\x41rrayReplace\x10`\x12\x11\n\rArrayToString\x10\x61\x12\x0f\n\x0b\x43\x61rdinality\x10\x62\x12\r\n\tTrimArray\x10\x63\x12\x11\n\rArrayContains\x10\x64\x12\n\n\x06\x45ncode\x10\x65\x12\n\n\x06\x44\x65\x63ode\x10\x66\x12\x07\n\x03\x43ot\x10g*\xb2\x03\n\x11\x41ggregateFunction\x12\x07\n\x03MIN\x10\x00\x12\x07\n\x03MAX\x10\x01\x12\x07\n\x03SUM\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\x13\n\x0f\x41PPROX_DISTINCT\x10\x05\x12\r\n\tARRAY_AGG\x10\x06\x12\x0c\n\x08VARIANCE\x10\x07\x12\x10\n\x0cVARIANCE_POP\x10\x08\x12\x0e\n\nCOVARIANCE\x10\t\x12\x12\n\x0e\x43OVARIANCE_POP\x10\n\x12\n\n\x06STDDEV\x10\x0b\x12\x0e\n\nSTDDEV_POP\x10\x0c\x12\x0f\n\x0b\x43ORRELATION\x10\r\x12\x1a\n\x16\x41PPROX_PERCENTILE_CONT\x10\x0e\x12\x11\n\rAPPROX_MEDIAN\x10\x0f\x12&\n\"APPROX_PERCENTILE_CONT_WITH_WEIGHT\x10\x10\x12\x0c\n\x08GROUPING\x10\x11\x12\n\n\x06MEDIAN\x10\x12\x12\x0b\n\x07\x42IT_AND\x10\x13\x12\n\n\x06\x42IT_OR\x10\x14\x12\x0b\n\x07\x42IT_XOR\x10\x15\x12\x0c\n\x08\x42OOL_AND\x10\x16\x12\x0b\n\x07\x42OOL_OR\x10\x17\x12\x13\n\x0f\x46IRST_VALUE_AGG\x10\x18\x12\x12\n\x0eLAST_VALUE_AGG\x10\x19*\xb0\x01\n\x15\x42uiltInWindowFunction\x12\x0e\n\nROW_NUMBER\x10\x00\x12\x08\n\x04RANK\x10\x01\x12\x0e\n\nDENSE_RANK\x10\x02\x12\x10\n\x0cPERCENT_RANK\x10\x03\x12\r\n\tCUME_DIST\x10\x04\x12\t\n\x05NTILE\x10\x05\x12\x07\n\x03LAG\x10\x06\x12\x08\n\x04LEAD\x10\x07\x12\x0f\n\x0b\x46IRST_VALUE\x10\x08\x12\x0e\n\nLAST_VALUE\x10\t\x12\r\n\tNTH_VALUE\x10\n*3\n\x10WindowFrameUnits\x12\x08\n\x04ROWS\x10\x00\x12\t\n\x05RANGE\x10\x01\x12\n\n\x06GROUPS\x10\x02*E\n\x14WindowFrameBoundType\x12\x0f\n\x0b\x43URRENT_ROW\x10\x00\x12\r\n\tPRECEDING\x10\x01\x12\r\n\tFOLLOWING\x10\x02*(\n\x08\x44\x61teUnit\x12\x07\n\x03\x44\x61y\x10\x00\x12\x13\n\x0f\x44\x61teMillisecond\x10\x01*H\n\x08TimeUnit\x12\n\n\x06Second\x10\x00\x12\x0f\n\x0bMillisecond\x10\x01\x12\x0f\n\x0bMicrosecond\x10\x02\x12\x0e\n\nNanosecond\x10\x03*<\n\x0cIntervalUnit\x12\r\n\tYearMonth\x10\x00\x12\x0b\n\x07\x44\x61yTime\x10\x01\x12\x10\n\x0cMonthDayNano\x10\x02*\"\n\tUnionMode\x12\n\n\x06sparse\x10\x00\x12\t\n\x05\x64\x65nse\x10\x01*<\n\rPartitionMode\x12\x10\n\x0c\x43OLLECT_LEFT\x10\x00\x12\x0f\n\x0bPARTITIONED\x10\x01\x12\x08\n\x04\x41UTO\x10\x02*b\n\rAggregateMode\x12\x0b\n\x07PARTIAL\x10\x00\x12\t\n\x05\x46INAL\x10\x01\x12\x15\n\x11\x46INAL_PARTITIONED\x10\x02\x12\n\n\x06SINGLE\x10\x03\x12\x16\n\x12SINGLE_PARTITIONED\x10\x04*)\n\x08JoinSide\x12\r\n\tLEFT_SIDE\x10\x00\x12\x0e\n\nRIGHT_SIDE\x10\x01\x42\x39\n$org.apache.arrow.datafusion.protobufB\x0f\x44\x61tafusionProtoP\x01\x62\x06proto3') +DESCRIPTOR = _descriptor_pool.Default().AddSerializedFile(b'\n\x16proto/datafusion.proto\x12\ndatafusion\"\"\n\x0e\x43olumnRelation\x12\x10\n\x08relation\x18\x01 \x01(\t\"D\n\x06\x43olumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12,\n\x08relation\x18\x02 \x01(\x0b\x32\x1a.datafusion.ColumnRelation\"Z\n\x07\x44\x66\x46ield\x12 \n\x05\x66ield\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12-\n\tqualifier\x18\x02 \x01(\x0b\x32\x1a.datafusion.ColumnRelation\"\x97\x01\n\x08\x44\x66Schema\x12$\n\x07\x63olumns\x18\x01 \x03(\x0b\x32\x13.datafusion.DfField\x12\x34\n\x08metadata\x18\x02 \x03(\x0b\x32\".datafusion.DfSchema.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x91\x0b\n\x0fLogicalPlanNode\x12\x38\n\x0clisting_scan\x18\x01 \x01(\x0b\x32 .datafusion.ListingTableScanNodeH\x00\x12\x30\n\nprojection\x18\x03 \x01(\x0b\x32\x1a.datafusion.ProjectionNodeH\x00\x12.\n\tselection\x18\x04 \x01(\x0b\x32\x19.datafusion.SelectionNodeH\x00\x12&\n\x05limit\x18\x05 \x01(\x0b\x32\x15.datafusion.LimitNodeH\x00\x12.\n\taggregate\x18\x06 \x01(\x0b\x32\x19.datafusion.AggregateNodeH\x00\x12$\n\x04join\x18\x07 \x01(\x0b\x32\x14.datafusion.JoinNodeH\x00\x12$\n\x04sort\x18\x08 \x01(\x0b\x32\x14.datafusion.SortNodeH\x00\x12\x32\n\x0brepartition\x18\t \x01(\x0b\x32\x1b.datafusion.RepartitionNodeH\x00\x12\x37\n\x0e\x65mpty_relation\x18\n \x01(\x0b\x32\x1d.datafusion.EmptyRelationNodeH\x00\x12\x44\n\x15\x63reate_external_table\x18\x0b \x01(\x0b\x32#.datafusion.CreateExternalTableNodeH\x00\x12*\n\x07\x65xplain\x18\x0c \x01(\x0b\x32\x17.datafusion.ExplainNodeH\x00\x12(\n\x06window\x18\r \x01(\x0b\x32\x16.datafusion.WindowNodeH\x00\x12*\n\x07\x61nalyze\x18\x0e \x01(\x0b\x32\x17.datafusion.AnalyzeNodeH\x00\x12/\n\ncross_join\x18\x0f \x01(\x0b\x32\x19.datafusion.CrossJoinNodeH\x00\x12(\n\x06values\x18\x10 \x01(\x0b\x32\x16.datafusion.ValuesNodeH\x00\x12\x35\n\textension\x18\x11 \x01(\x0b\x32 .datafusion.LogicalExtensionNodeH\x00\x12\x44\n\x15\x63reate_catalog_schema\x18\x12 \x01(\x0b\x32#.datafusion.CreateCatalogSchemaNodeH\x00\x12&\n\x05union\x18\x13 \x01(\x0b\x32\x15.datafusion.UnionNodeH\x00\x12\x37\n\x0e\x63reate_catalog\x18\x14 \x01(\x0b\x32\x1d.datafusion.CreateCatalogNodeH\x00\x12\x37\n\x0esubquery_alias\x18\x15 \x01(\x0b\x32\x1d.datafusion.SubqueryAliasNodeH\x00\x12\x31\n\x0b\x63reate_view\x18\x16 \x01(\x0b\x32\x1a.datafusion.CreateViewNodeH\x00\x12,\n\x08\x64istinct\x18\x17 \x01(\x0b\x32\x18.datafusion.DistinctNodeH\x00\x12\x32\n\tview_scan\x18\x18 \x01(\x0b\x32\x1d.datafusion.ViewTableScanNodeH\x00\x12\x36\n\x0b\x63ustom_scan\x18\x19 \x01(\x0b\x32\x1f.datafusion.CustomTableScanNodeH\x00\x12*\n\x07prepare\x18\x1a \x01(\x0b\x32\x17.datafusion.PrepareNodeH\x00\x12-\n\tdrop_view\x18\x1b \x01(\x0b\x32\x18.datafusion.DropViewNodeH\x00\x12\x31\n\x0b\x64istinct_on\x18\x1c \x01(\x0b\x32\x1a.datafusion.DistinctOnNodeH\x00\x12)\n\x07\x63opy_to\x18\x1d \x01(\x0b\x32\x16.datafusion.CopyToNodeH\x00\x42\x11\n\x0fLogicalPlanType\"Q\n\x14LogicalExtensionNode\x12\x0c\n\x04node\x18\x01 \x01(\x0c\x12+\n\x06inputs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalPlanNode\"$\n\x11ProjectionColumns\x12\x0f\n\x07\x63olumns\x18\x01 \x03(\t\"f\n\tCsvFormat\x12\x12\n\nhas_header\x18\x01 \x01(\x08\x12\x11\n\tdelimiter\x18\x02 \x01(\t\x12\r\n\x05quote\x18\x03 \x01(\t\x12\x10\n\x06\x65scape\x18\x04 \x01(\tH\x00\x42\x11\n\x0foptional_escape\"\x15\n\rParquetFormatJ\x04\x08\x01\x10\x02\"\x0c\n\nAvroFormat\"T\n\x19LogicalExprNodeCollection\x12\x37\n\x12logical_expr_nodes\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\x9a\x04\n\x14ListingTableScanNode\x12\x33\n\ntable_name\x18\x0e \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\r\n\x05paths\x18\x02 \x03(\t\x12\x16\n\x0e\x66ile_extension\x18\x03 \x01(\t\x12\x31\n\nprojection\x18\x04 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\"\n\x06schema\x18\x05 \x01(\x0b\x32\x12.datafusion.Schema\x12,\n\x07\x66ilters\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x1c\n\x14table_partition_cols\x18\x07 \x03(\t\x12\x14\n\x0c\x63ollect_stat\x18\x08 \x01(\x08\x12\x19\n\x11target_partitions\x18\t \x01(\r\x12$\n\x03\x63sv\x18\n \x01(\x0b\x32\x15.datafusion.CsvFormatH\x00\x12,\n\x07parquet\x18\x0b \x01(\x0b\x32\x19.datafusion.ParquetFormatH\x00\x12&\n\x04\x61vro\x18\x0c \x01(\x0b\x32\x16.datafusion.AvroFormatH\x00\x12>\n\x0f\x66ile_sort_order\x18\r \x03(\x0b\x32%.datafusion.LogicalExprNodeCollectionB\x10\n\x0e\x46ileFormatTypeJ\x04\x08\x01\x10\x02\"\xe5\x01\n\x11ViewTableScanNode\x12\x33\n\ntable_name\x18\x06 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12*\n\x05input\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12\x31\n\nprojection\x18\x04 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\x12\n\ndefinition\x18\x05 \x01(\tJ\x04\x08\x01\x10\x02\"\xf0\x01\n\x13\x43ustomTableScanNode\x12\x33\n\ntable_name\x18\x06 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x31\n\nprojection\x18\x02 \x01(\x0b\x32\x1d.datafusion.ProjectionColumns\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12,\n\x07\x66ilters\x18\x04 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x19\n\x11\x63ustom_table_data\x18\x05 \x01(\x0cJ\x04\x08\x01\x10\x02\"\x8a\x01\n\x0eProjectionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x05\x61lias\x18\x03 \x01(\tH\x00\x42\x10\n\x0eoptional_alias\"f\n\rSelectionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"p\n\x08SortNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"\x95\x01\n\x0fRepartitionNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x15\n\x0bround_robin\x18\x02 \x01(\x04H\x00\x12+\n\x04hash\x18\x03 \x01(\x0b\x32\x1b.datafusion.HashRepartitionH\x00\x42\x12\n\x10partition_method\"Z\n\x0fHashRepartition\x12.\n\thash_expr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x17\n\x0fpartition_count\x18\x02 \x01(\x04\",\n\x11\x45mptyRelationNode\x12\x17\n\x0fproduce_one_row\x18\x01 \x01(\x08\"\'\n\x14PrimaryKeyConstraint\x12\x0f\n\x07indices\x18\x01 \x03(\x04\"#\n\x10UniqueConstraint\x12\x0f\n\x07indices\x18\x01 \x03(\x04\"\x88\x01\n\nConstraint\x12\x37\n\x0bprimary_key\x18\x01 \x01(\x0b\x32 .datafusion.PrimaryKeyConstraintH\x00\x12.\n\x06unique\x18\x02 \x01(\x0b\x32\x1c.datafusion.UniqueConstraintH\x00\x42\x11\n\x0f\x63onstraint_mode\":\n\x0b\x43onstraints\x12+\n\x0b\x63onstraints\x18\x01 \x03(\x0b\x32\x16.datafusion.Constraint\"\xe8\x05\n\x17\x43reateExternalTableNode\x12-\n\x04name\x18\x0c \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x10\n\x08location\x18\x02 \x01(\t\x12\x11\n\tfile_type\x18\x03 \x01(\t\x12\x12\n\nhas_header\x18\x04 \x01(\x08\x12$\n\x06schema\x18\x05 \x01(\x0b\x32\x14.datafusion.DfSchema\x12\x1c\n\x14table_partition_cols\x18\x06 \x03(\t\x12\x15\n\rif_not_exists\x18\x07 \x01(\x08\x12\x11\n\tdelimiter\x18\x08 \x01(\t\x12\x12\n\ndefinition\x18\t \x01(\t\x12\x41\n\x15\x66ile_compression_type\x18\x11 \x01(\x0e\x32\".datafusion.CompressionTypeVariant\x12:\n\x0border_exprs\x18\r \x03(\x0b\x32%.datafusion.LogicalExprNodeCollection\x12\x11\n\tunbounded\x18\x0e \x01(\x08\x12\x41\n\x07options\x18\x0b \x03(\x0b\x32\x30.datafusion.CreateExternalTableNode.OptionsEntry\x12,\n\x0b\x63onstraints\x18\x0f \x01(\x0b\x32\x17.datafusion.Constraints\x12P\n\x0f\x63olumn_defaults\x18\x10 \x03(\x0b\x32\x37.datafusion.CreateExternalTableNode.ColumnDefaultsEntry\x1a.\n\x0cOptionsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\x1aR\n\x13\x43olumnDefaultsEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12*\n\x05value\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode:\x02\x38\x01J\x04\x08\x01\x10\x02J\x04\x08\n\x10\x0b\"r\n\x0bPrepareNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12)\n\ndata_types\x18\x02 \x03(\x0b\x32\x15.datafusion.ArrowType\x12*\n\x05input\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"k\n\x17\x43reateCatalogSchemaNode\x12\x13\n\x0bschema_name\x18\x01 \x01(\t\x12\x15\n\rif_not_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"f\n\x11\x43reateCatalogNode\x12\x14\n\x0c\x63\x61talog_name\x18\x01 \x01(\t\x12\x15\n\rif_not_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"v\n\x0c\x44ropViewNode\x12-\n\x04name\x18\x01 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12\x11\n\tif_exists\x18\x02 \x01(\x08\x12$\n\x06schema\x18\x03 \x01(\x0b\x32\x14.datafusion.DfSchema\"\x99\x01\n\x0e\x43reateViewNode\x12-\n\x04name\x18\x05 \x01(\x0b\x32\x1f.datafusion.OwnedTableReference\x12*\n\x05input\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x12\n\nor_replace\x18\x03 \x01(\x08\x12\x12\n\ndefinition\x18\x04 \x01(\tJ\x04\x08\x01\x10\x02\"N\n\nValuesNode\x12\x0e\n\x06n_cols\x18\x01 \x01(\x04\x12\x30\n\x0bvalues_list\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"J\n\x0b\x41nalyzeNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0f\n\x07verbose\x18\x02 \x01(\x08\"J\n\x0b\x45xplainNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0f\n\x07verbose\x18\x02 \x01(\x08\"\x9c\x01\n\rAggregateNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12/\n\ngroup_expr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12.\n\taggr_expr\x18\x03 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"j\n\nWindowNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x30\n\x0bwindow_expr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xef\x02\n\x08JoinNode\x12)\n\x04left\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12*\n\x05right\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\'\n\tjoin_type\x18\x03 \x01(\x0e\x32\x14.datafusion.JoinType\x12\x33\n\x0fjoin_constraint\x18\x04 \x01(\x0e\x32\x1a.datafusion.JoinConstraint\x12\x32\n\rleft_join_key\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x33\n\x0eright_join_key\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x18\n\x10null_equals_null\x18\x07 \x01(\x08\x12+\n\x06\x66ilter\x18\x08 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\":\n\x0c\x44istinctNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"\xcc\x01\n\x0e\x44istinctOnNode\x12,\n\x07on_expr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x30\n\x0bselect_expr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12.\n\tsort_expr\x18\x03 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12*\n\x05input\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"\xda\x01\n\nCopyToNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x12\n\noutput_url\x18\x02 \x01(\t\x12-\n\x0bsql_options\x18\x04 \x01(\x0b\x32\x16.datafusion.SQLOptionsH\x00\x12;\n\x0ewriter_options\x18\x05 \x01(\x0b\x32!.datafusion.FileTypeWriterOptionsH\x00\x12\x11\n\tfile_type\x18\x06 \x01(\tB\r\n\x0b\x43opyOptions\"3\n\nSQLOptions\x12%\n\x06option\x18\x01 \x03(\x0b\x32\x15.datafusion.SQLOption\"\'\n\tSQLOption\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t\"8\n\tUnionNode\x12+\n\x06inputs\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalPlanNode\"f\n\rCrossJoinNode\x12)\n\x04left\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12*\n\x05right\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\"T\n\tLimitNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12\x0c\n\x04skip\x18\x02 \x01(\x03\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\">\n\x11SelectionExecNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"u\n\x11SubqueryAliasNode\x12*\n\x05input\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalPlanNode\x12.\n\x05\x61lias\x18\x03 \x01(\x0b\x32\x1f.datafusion.OwnedTableReferenceJ\x04\x08\x02\x10\x03\"\xe9\x0c\n\x0fLogicalExprNode\x12$\n\x06\x63olumn\x18\x01 \x01(\x0b\x32\x12.datafusion.ColumnH\x00\x12&\n\x05\x61lias\x18\x02 \x01(\x0b\x32\x15.datafusion.AliasNodeH\x00\x12*\n\x07literal\x18\x03 \x01(\x0b\x32\x17.datafusion.ScalarValueH\x00\x12\x31\n\x0b\x62inary_expr\x18\x04 \x01(\x0b\x32\x1a.datafusion.BinaryExprNodeH\x00\x12\x37\n\x0e\x61ggregate_expr\x18\x05 \x01(\x0b\x32\x1d.datafusion.AggregateExprNodeH\x00\x12*\n\x0cis_null_expr\x18\x06 \x01(\x0b\x32\x12.datafusion.IsNullH\x00\x12\x31\n\x10is_not_null_expr\x18\x07 \x01(\x0b\x32\x15.datafusion.IsNotNullH\x00\x12#\n\x08not_expr\x18\x08 \x01(\x0b\x32\x0f.datafusion.NotH\x00\x12*\n\x07\x62\x65tween\x18\t \x01(\x0b\x32\x17.datafusion.BetweenNodeH\x00\x12%\n\x05\x63\x61se_\x18\n \x01(\x0b\x32\x14.datafusion.CaseNodeH\x00\x12$\n\x04\x63\x61st\x18\x0b \x01(\x0b\x32\x14.datafusion.CastNodeH\x00\x12(\n\x04sort\x18\x0c \x01(\x0b\x32\x18.datafusion.SortExprNodeH\x00\x12,\n\x08negative\x18\r \x01(\x0b\x32\x18.datafusion.NegativeNodeH\x00\x12)\n\x07in_list\x18\x0e \x01(\x0b\x32\x16.datafusion.InListNodeH\x00\x12(\n\x08wildcard\x18\x0f \x01(\x0b\x32\x14.datafusion.WildcardH\x00\x12\x39\n\x0fscalar_function\x18\x10 \x01(\x0b\x32\x1e.datafusion.ScalarFunctionNodeH\x00\x12+\n\x08try_cast\x18\x11 \x01(\x0b\x32\x17.datafusion.TryCastNodeH\x00\x12\x31\n\x0bwindow_expr\x18\x12 \x01(\x0b\x32\x1a.datafusion.WindowExprNodeH\x00\x12>\n\x12\x61ggregate_udf_expr\x18\x13 \x01(\x0b\x32 .datafusion.AggregateUDFExprNodeH\x00\x12\x38\n\x0fscalar_udf_expr\x18\x14 \x01(\x0b\x32\x1d.datafusion.ScalarUDFExprNodeH\x00\x12\x38\n\x11get_indexed_field\x18\x15 \x01(\x0b\x32\x1b.datafusion.GetIndexedFieldH\x00\x12\x33\n\x0cgrouping_set\x18\x16 \x01(\x0b\x32\x1b.datafusion.GroupingSetNodeH\x00\x12$\n\x04\x63ube\x18\x17 \x01(\x0b\x32\x14.datafusion.CubeNodeH\x00\x12(\n\x06rollup\x18\x18 \x01(\x0b\x32\x16.datafusion.RollupNodeH\x00\x12%\n\x07is_true\x18\x19 \x01(\x0b\x32\x12.datafusion.IsTrueH\x00\x12\'\n\x08is_false\x18\x1a \x01(\x0b\x32\x13.datafusion.IsFalseH\x00\x12+\n\nis_unknown\x18\x1b \x01(\x0b\x32\x15.datafusion.IsUnknownH\x00\x12,\n\x0bis_not_true\x18\x1c \x01(\x0b\x32\x15.datafusion.IsNotTrueH\x00\x12.\n\x0cis_not_false\x18\x1d \x01(\x0b\x32\x16.datafusion.IsNotFalseH\x00\x12\x32\n\x0eis_not_unknown\x18\x1e \x01(\x0b\x32\x18.datafusion.IsNotUnknownH\x00\x12$\n\x04like\x18\x1f \x01(\x0b\x32\x14.datafusion.LikeNodeH\x00\x12&\n\x05ilike\x18 \x01(\x0b\x32\x15.datafusion.ILikeNodeH\x00\x12/\n\nsimilar_to\x18! \x01(\x0b\x32\x19.datafusion.SimilarToNodeH\x00\x12\x32\n\x0bplaceholder\x18\" \x01(\x0b\x32\x1b.datafusion.PlaceholderNodeH\x00\x12$\n\x06unnest\x18# \x01(\x0b\x32\x12.datafusion.UnnestH\x00\x42\n\n\x08\x45xprType\"\x1d\n\x08Wildcard\x12\x11\n\tqualifier\x18\x01 \x01(\t\"G\n\x0fPlaceholderNode\x12\n\n\x02id\x18\x01 \x01(\t\x12(\n\tdata_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"<\n\x0fLogicalExprList\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"<\n\x0fGroupingSetNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprList\"5\n\x08\x43ubeNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"7\n\nRollupNode\x12)\n\x04\x65xpr\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"9\n\x10NamedStructField\x12%\n\x04name\x18\x01 \x01(\x0b\x32\x17.datafusion.ScalarValue\"5\n\tListIndex\x12(\n\x03key\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"\x8f\x01\n\tListRange\x12*\n\x05start\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\x04stop\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12+\n\x06stride\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xdb\x01\n\x0fGetIndexedField\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12:\n\x12named_struct_field\x18\x02 \x01(\x0b\x32\x1c.datafusion.NamedStructFieldH\x00\x12+\n\nlist_index\x18\x03 \x01(\x0b\x32\x15.datafusion.ListIndexH\x00\x12+\n\nlist_range\x18\x04 \x01(\x0b\x32\x15.datafusion.ListRangeH\x00\x42\x07\n\x05\x66ield\"3\n\x06IsNull\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsNotNull\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"3\n\x06IsTrue\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"4\n\x07IsFalse\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsUnknown\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"6\n\tIsNotTrue\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"7\n\nIsNotFalse\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"9\n\x0cIsNotUnknown\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"0\n\x03Not\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"x\n\tAliasNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\r\n\x05\x61lias\x18\x02 \x01(\t\x12\x31\n\x08relation\x18\x03 \x03(\x0b\x32\x1f.datafusion.OwnedTableReference\"K\n\x0e\x42inaryExprNode\x12-\n\x08operands\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"9\n\x0cNegativeNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"4\n\x06Unnest\x12*\n\x05\x65xprs\x18\x01 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"s\n\nInListNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\x04list\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x07negated\x18\x03 \x01(\x08\"h\n\x12ScalarFunctionNode\x12\'\n\x03\x66un\x18\x01 \x01(\x0e\x32\x1a.datafusion.ScalarFunction\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xe2\x01\n\x11\x41ggregateExprNode\x12\x34\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunction\x12)\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x10\n\x08\x64istinct\x18\x03 \x01(\x08\x12+\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xaf\x01\n\x14\x41ggregateUDFExprNode\x12\x10\n\x08\x66un_name\x18\x01 \x01(\t\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12+\n\x06\x66ilter\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x04 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"P\n\x11ScalarUDFExprNode\x12\x10\n\x08\x66un_name\x18\x01 \x01(\t\x12)\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\"\xf7\x02\n\x0eWindowExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12>\n\x11\x62uilt_in_function\x18\x02 \x01(\x0e\x32!.datafusion.BuiltInWindowFunctionH\x00\x12\x0e\n\x04udaf\x18\x03 \x01(\tH\x00\x12\x0e\n\x04udwf\x18\t \x01(\tH\x00\x12)\n\x04\x65xpr\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x31\n\x0cpartition_by\x18\x05 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x08order_by\x18\x06 \x03(\x0b\x32\x1b.datafusion.LogicalExprNode\x12-\n\x0cwindow_frame\x18\x08 \x01(\x0b\x32\x17.datafusion.WindowFrameB\x11\n\x0fwindow_function\"\x9e\x01\n\x0b\x42\x65tweenNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0f\n\x07negated\x18\x02 \x01(\x08\x12(\n\x03low\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\x04high\x18\x04 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"\x89\x01\n\x08LikeNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x8a\x01\n\tILikeNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x8e\x01\n\rSimilarToNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12)\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x07pattern\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x13\n\x0b\x65scape_char\x18\x04 \x01(\t\"\x93\x01\n\x08\x43\x61seNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12,\n\x0ewhen_then_expr\x18\x02 \x03(\x0b\x32\x14.datafusion.WhenThen\x12.\n\telse_expr\x18\x03 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"j\n\x08WhenThen\x12.\n\twhen_expr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12.\n\tthen_expr\x18\x02 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\"`\n\x08\x43\x61stNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"c\n\x0bTryCastNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"[\n\x0cSortExprNode\x12)\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1b.datafusion.LogicalExprNode\x12\x0b\n\x03\x61sc\x18\x02 \x01(\x08\x12\x13\n\x0bnulls_first\x18\x03 \x01(\x08\"\xb6\x01\n\x0bWindowFrame\x12\x38\n\x12window_frame_units\x18\x01 \x01(\x0e\x32\x1c.datafusion.WindowFrameUnits\x12\x31\n\x0bstart_bound\x18\x02 \x01(\x0b\x32\x1c.datafusion.WindowFrameBound\x12-\n\x05\x62ound\x18\x03 \x01(\x0b\x32\x1c.datafusion.WindowFrameBoundH\x00\x42\x0b\n\tend_bound\"\x83\x01\n\x10WindowFrameBound\x12\x41\n\x17window_frame_bound_type\x18\x01 \x01(\x0e\x32 .datafusion.WindowFrameBoundType\x12,\n\x0b\x62ound_value\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"\x91\x01\n\x06Schema\x12\"\n\x07\x63olumns\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\x12\x32\n\x08metadata\x18\x02 \x03(\x0b\x32 .datafusion.Schema.MetadataEntry\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"\x82\x02\n\x05\x46ield\x12\x0c\n\x04name\x18\x01 \x01(\t\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\x12\x10\n\x08nullable\x18\x03 \x01(\x08\x12#\n\x08\x63hildren\x18\x04 \x03(\x0b\x32\x11.datafusion.Field\x12\x31\n\x08metadata\x18\x05 \x03(\x0b\x32\x1f.datafusion.Field.MetadataEntry\x12\x0f\n\x07\x64ict_id\x18\x06 \x01(\x03\x12\x14\n\x0c\x64ict_ordered\x18\x07 \x01(\x08\x1a/\n\rMetadataEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12\r\n\x05value\x18\x02 \x01(\t:\x02\x38\x01\"!\n\x0f\x46ixedSizeBinary\x12\x0e\n\x06length\x18\x01 \x01(\x05\"F\n\tTimestamp\x12\'\n\ttime_unit\x18\x01 \x01(\x0e\x32\x14.datafusion.TimeUnit\x12\x10\n\x08timezone\x18\x02 \x01(\t\"7\n\x07\x44\x65\x63imal\x12\x11\n\tprecision\x18\x03 \x01(\r\x12\r\n\x05scale\x18\x04 \x01(\x05J\x04\x08\x01\x10\x02J\x04\x08\x02\x10\x03\"-\n\x04List\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\"I\n\rFixedSizeList\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12\x11\n\tlist_size\x18\x02 \x01(\x05\"V\n\nDictionary\x12\"\n\x03key\x18\x01 \x01(\x0b\x32\x15.datafusion.ArrowType\x12$\n\x05value\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"4\n\x06Struct\x12*\n\x0fsub_field_types\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\"A\n\x03Map\x12%\n\nfield_type\x18\x01 \x01(\x0b\x32\x11.datafusion.Field\x12\x13\n\x0bkeys_sorted\x18\x02 \x01(\x08\"l\n\x05Union\x12&\n\x0bunion_types\x18\x01 \x03(\x0b\x32\x11.datafusion.Field\x12)\n\nunion_mode\x18\x02 \x01(\x0e\x32\x15.datafusion.UnionMode\x12\x10\n\x08type_ids\x18\x03 \x03(\x05\"`\n\x11ScalarNestedValue\x12\x13\n\x0bipc_message\x18\x01 \x01(\x0c\x12\x12\n\narrow_data\x18\x02 \x01(\x0c\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\"_\n\x11ScalarTime32Value\x12\x1d\n\x13time32_second_value\x18\x01 \x01(\x05H\x00\x12\"\n\x18time32_millisecond_value\x18\x02 \x01(\x05H\x00\x42\x07\n\x05value\"c\n\x11ScalarTime64Value\x12\"\n\x18time64_microsecond_value\x18\x01 \x01(\x03H\x00\x12!\n\x17time64_nanosecond_value\x18\x02 \x01(\x03H\x00\x42\x07\n\x05value\"\xb3\x01\n\x14ScalarTimestampValue\x12 \n\x16time_microsecond_value\x18\x01 \x01(\x03H\x00\x12\x1f\n\x15time_nanosecond_value\x18\x02 \x01(\x03H\x00\x12\x1b\n\x11time_second_value\x18\x03 \x01(\x03H\x00\x12 \n\x16time_millisecond_value\x18\x04 \x01(\x03H\x00\x12\x10\n\x08timezone\x18\x05 \x01(\tB\x07\n\x05value\"j\n\x15ScalarDictionaryValue\x12)\n\nindex_type\x18\x01 \x01(\x0b\x32\x15.datafusion.ArrowType\x12&\n\x05value\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"H\n\x19IntervalMonthDayNanoValue\x12\x0e\n\x06months\x18\x01 \x01(\x05\x12\x0c\n\x04\x64\x61ys\x18\x02 \x01(\x05\x12\r\n\x05nanos\x18\x03 \x01(\x03\"7\n\x15ScalarFixedSizeBinary\x12\x0e\n\x06values\x18\x01 \x01(\x0c\x12\x0e\n\x06length\x18\x02 \x01(\x05\"\x86\x0b\n\x0bScalarValue\x12+\n\nnull_value\x18! \x01(\x0b\x32\x15.datafusion.ArrowTypeH\x00\x12\x14\n\nbool_value\x18\x01 \x01(\x08H\x00\x12\x14\n\nutf8_value\x18\x02 \x01(\tH\x00\x12\x1a\n\x10large_utf8_value\x18\x03 \x01(\tH\x00\x12\x14\n\nint8_value\x18\x04 \x01(\x05H\x00\x12\x15\n\x0bint16_value\x18\x05 \x01(\x05H\x00\x12\x15\n\x0bint32_value\x18\x06 \x01(\x05H\x00\x12\x15\n\x0bint64_value\x18\x07 \x01(\x03H\x00\x12\x15\n\x0buint8_value\x18\x08 \x01(\rH\x00\x12\x16\n\x0cuint16_value\x18\t \x01(\rH\x00\x12\x16\n\x0cuint32_value\x18\n \x01(\rH\x00\x12\x16\n\x0cuint64_value\x18\x0b \x01(\x04H\x00\x12\x17\n\rfloat32_value\x18\x0c \x01(\x02H\x00\x12\x17\n\rfloat64_value\x18\r \x01(\x01H\x00\x12\x17\n\rdate_32_value\x18\x0e \x01(\x05H\x00\x12\x35\n\x0ctime32_value\x18\x0f \x01(\x0b\x32\x1d.datafusion.ScalarTime32ValueH\x00\x12\x39\n\x10large_list_value\x18\x10 \x01(\x0b\x32\x1d.datafusion.ScalarNestedValueH\x00\x12\x33\n\nlist_value\x18\x11 \x01(\x0b\x32\x1d.datafusion.ScalarNestedValueH\x00\x12>\n\x15\x66ixed_size_list_value\x18\x12 \x01(\x0b\x32\x1d.datafusion.ScalarNestedValueH\x00\x12\x35\n\x0cstruct_value\x18 \x01(\x0b\x32\x1d.datafusion.ScalarNestedValueH\x00\x12\x32\n\x10\x64\x65\x63imal128_value\x18\x14 \x01(\x0b\x32\x16.datafusion.Decimal128H\x00\x12\x32\n\x10\x64\x65\x63imal256_value\x18\' \x01(\x0b\x32\x16.datafusion.Decimal256H\x00\x12\x17\n\rdate_64_value\x18\x15 \x01(\x03H\x00\x12\"\n\x18interval_yearmonth_value\x18\x18 \x01(\x05H\x00\x12 \n\x16interval_daytime_value\x18\x19 \x01(\x03H\x00\x12\x1f\n\x15\x64uration_second_value\x18# \x01(\x03H\x00\x12$\n\x1a\x64uration_millisecond_value\x18$ \x01(\x03H\x00\x12$\n\x1a\x64uration_microsecond_value\x18% \x01(\x03H\x00\x12#\n\x19\x64uration_nanosecond_value\x18& \x01(\x03H\x00\x12;\n\x0ftimestamp_value\x18\x1a \x01(\x0b\x32 .datafusion.ScalarTimestampValueH\x00\x12=\n\x10\x64ictionary_value\x18\x1b \x01(\x0b\x32!.datafusion.ScalarDictionaryValueH\x00\x12\x16\n\x0c\x62inary_value\x18\x1c \x01(\x0cH\x00\x12\x1c\n\x12large_binary_value\x18\x1d \x01(\x0cH\x00\x12\x35\n\x0ctime64_value\x18\x1e \x01(\x0b\x32\x1d.datafusion.ScalarTime64ValueH\x00\x12H\n\x17interval_month_day_nano\x18\x1f \x01(\x0b\x32%.datafusion.IntervalMonthDayNanoValueH\x00\x12\x44\n\x17\x66ixed_size_binary_value\x18\" \x01(\x0b\x32!.datafusion.ScalarFixedSizeBinaryH\x00\x42\x07\n\x05valueJ\x04\x08\x13\x10\x14\"1\n\nDecimal128\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\t\n\x01p\x18\x02 \x01(\x03\x12\t\n\x01s\x18\x03 \x01(\x03\"1\n\nDecimal256\x12\r\n\x05value\x18\x01 \x01(\x0c\x12\t\n\x01p\x18\x02 \x01(\x03\x12\t\n\x01s\x18\x03 \x01(\x03\"\x94\x0b\n\tArrowType\x12(\n\x04NONE\x18\x01 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04\x42OOL\x18\x02 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05UINT8\x18\x03 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04INT8\x18\x04 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT16\x18\x05 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT16\x18\x06 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT32\x18\x07 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT32\x18\x08 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06UINT64\x18\t \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12)\n\x05INT64\x18\n \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT16\x18\x0b \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT32\x18\x0c \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12+\n\x07\x46LOAT64\x18\r \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x04UTF8\x18\x0e \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12.\n\nLARGE_UTF8\x18 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x42INARY\x18\x0f \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x1b\n\x11\x46IXED_SIZE_BINARY\x18\x10 \x01(\x05H\x00\x12\x30\n\x0cLARGE_BINARY\x18\x1f \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x44\x41TE32\x18\x11 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12*\n\x06\x44\x41TE64\x18\x12 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12(\n\x08\x44URATION\x18\x13 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12*\n\tTIMESTAMP\x18\x14 \x01(\x0b\x32\x15.datafusion.TimestampH\x00\x12&\n\x06TIME32\x18\x15 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12&\n\x06TIME64\x18\x16 \x01(\x0e\x32\x14.datafusion.TimeUnitH\x00\x12,\n\x08INTERVAL\x18\x17 \x01(\x0e\x32\x18.datafusion.IntervalUnitH\x00\x12&\n\x07\x44\x45\x43IMAL\x18\x18 \x01(\x0b\x32\x13.datafusion.DecimalH\x00\x12 \n\x04LIST\x18\x19 \x01(\x0b\x32\x10.datafusion.ListH\x00\x12&\n\nLARGE_LIST\x18\x1a \x01(\x0b\x32\x10.datafusion.ListH\x00\x12\x34\n\x0f\x46IXED_SIZE_LIST\x18\x1b \x01(\x0b\x32\x19.datafusion.FixedSizeListH\x00\x12$\n\x06STRUCT\x18\x1c \x01(\x0b\x32\x12.datafusion.StructH\x00\x12\"\n\x05UNION\x18\x1d \x01(\x0b\x32\x11.datafusion.UnionH\x00\x12,\n\nDICTIONARY\x18\x1e \x01(\x0b\x32\x16.datafusion.DictionaryH\x00\x12\x1e\n\x03MAP\x18! \x01(\x0b\x32\x0f.datafusion.MapH\x00\x42\x11\n\x0f\x61rrow_type_enum\"\x0e\n\x0c\x45mptyMessage\"0\n\x17\x41nalyzedLogicalPlanType\x12\x15\n\ranalyzer_name\x18\x01 \x01(\t\"2\n\x18OptimizedLogicalPlanType\x12\x16\n\x0eoptimizer_name\x18\x01 \x01(\t\"3\n\x19OptimizedPhysicalPlanType\x12\x16\n\x0eoptimizer_name\x18\x01 \x01(\t\"\x8c\x05\n\x08PlanType\x12\x36\n\x12InitialLogicalPlan\x18\x01 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x42\n\x13\x41nalyzedLogicalPlan\x18\x07 \x01(\x0b\x32#.datafusion.AnalyzedLogicalPlanTypeH\x00\x12<\n\x18\x46inalAnalyzedLogicalPlan\x18\x08 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x44\n\x14OptimizedLogicalPlan\x18\x02 \x01(\x0b\x32$.datafusion.OptimizedLogicalPlanTypeH\x00\x12\x34\n\x10\x46inalLogicalPlan\x18\x03 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x37\n\x13InitialPhysicalPlan\x18\x04 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12@\n\x1cInitialPhysicalPlanWithStats\x18\t \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x46\n\x15OptimizedPhysicalPlan\x18\x05 \x01(\x0b\x32%.datafusion.OptimizedPhysicalPlanTypeH\x00\x12\x35\n\x11\x46inalPhysicalPlan\x18\x06 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12>\n\x1a\x46inalPhysicalPlanWithStats\x18\n \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x42\x10\n\x0eplan_type_enum\"H\n\x0fStringifiedPlan\x12\'\n\tplan_type\x18\x01 \x01(\x0b\x32\x14.datafusion.PlanType\x12\x0c\n\x04plan\x18\x02 \x01(\t\"#\n\x12\x42\x61reTableReference\x12\r\n\x05table\x18\x01 \x01(\t\"6\n\x15PartialTableReference\x12\x0e\n\x06schema\x18\x01 \x01(\t\x12\r\n\x05table\x18\x02 \x01(\t\"D\n\x12\x46ullTableReference\x12\x0f\n\x07\x63\x61talog\x18\x01 \x01(\t\x12\x0e\n\x06schema\x18\x02 \x01(\t\x12\r\n\x05table\x18\x03 \x01(\t\"\xc3\x01\n\x13OwnedTableReference\x12.\n\x04\x62\x61re\x18\x01 \x01(\x0b\x32\x1e.datafusion.BareTableReferenceH\x00\x12\x34\n\x07partial\x18\x02 \x01(\x0b\x32!.datafusion.PartialTableReferenceH\x00\x12.\n\x04\x66ull\x18\x03 \x01(\x0b\x32\x1e.datafusion.FullTableReferenceH\x00\x42\x16\n\x14table_reference_enum\"\x92\x0c\n\x10PhysicalPlanNode\x12\x37\n\x0cparquet_scan\x18\x01 \x01(\x0b\x32\x1f.datafusion.ParquetScanExecNodeH\x00\x12/\n\x08\x63sv_scan\x18\x02 \x01(\x0b\x32\x1b.datafusion.CsvScanExecNodeH\x00\x12*\n\x05\x65mpty\x18\x03 \x01(\x0b\x32\x19.datafusion.EmptyExecNodeH\x00\x12\x34\n\nprojection\x18\x04 \x01(\x0b\x32\x1e.datafusion.ProjectionExecNodeH\x00\x12\x37\n\x0cglobal_limit\x18\x06 \x01(\x0b\x32\x1f.datafusion.GlobalLimitExecNodeH\x00\x12\x35\n\x0blocal_limit\x18\x07 \x01(\x0b\x32\x1e.datafusion.LocalLimitExecNodeH\x00\x12\x32\n\taggregate\x18\x08 \x01(\x0b\x32\x1d.datafusion.AggregateExecNodeH\x00\x12\x31\n\thash_join\x18\t \x01(\x0b\x32\x1c.datafusion.HashJoinExecNodeH\x00\x12(\n\x04sort\x18\n \x01(\x0b\x32\x18.datafusion.SortExecNodeH\x00\x12?\n\x10\x63oalesce_batches\x18\x0b \x01(\x0b\x32#.datafusion.CoalesceBatchesExecNodeH\x00\x12,\n\x06\x66ilter\x18\x0c \x01(\x0b\x32\x1a.datafusion.FilterExecNodeH\x00\x12\x37\n\x05merge\x18\r \x01(\x0b\x32&.datafusion.CoalescePartitionsExecNodeH\x00\x12\x36\n\x0brepartition\x18\x0e \x01(\x0b\x32\x1f.datafusion.RepartitionExecNodeH\x00\x12/\n\x06window\x18\x0f \x01(\x0b\x32\x1d.datafusion.WindowAggExecNodeH\x00\x12\x33\n\ncross_join\x18\x10 \x01(\x0b\x32\x1d.datafusion.CrossJoinExecNodeH\x00\x12\x31\n\tavro_scan\x18\x11 \x01(\x0b\x32\x1c.datafusion.AvroScanExecNodeH\x00\x12\x36\n\textension\x18\x12 \x01(\x0b\x32!.datafusion.PhysicalExtensionNodeH\x00\x12*\n\x05union\x18\x13 \x01(\x0b\x32\x19.datafusion.UnionExecNodeH\x00\x12.\n\x07\x65xplain\x18\x14 \x01(\x0b\x32\x1b.datafusion.ExplainExecNodeH\x00\x12H\n\x15sort_preserving_merge\x18\x15 \x01(\x0b\x32\'.datafusion.SortPreservingMergeExecNodeH\x00\x12>\n\x10nested_loop_join\x18\x16 \x01(\x0b\x32\".datafusion.NestedLoopJoinExecNodeH\x00\x12.\n\x07\x61nalyze\x18\x17 \x01(\x0b\x32\x1b.datafusion.AnalyzeExecNodeH\x00\x12\x31\n\tjson_sink\x18\x18 \x01(\x0b\x32\x1c.datafusion.JsonSinkExecNodeH\x00\x12\x44\n\x13symmetric_hash_join\x18\x19 \x01(\x0b\x32%.datafusion.SymmetricHashJoinExecNodeH\x00\x12\x34\n\ninterleave\x18\x1a \x01(\x0b\x32\x1e.datafusion.InterleaveExecNodeH\x00\x12=\n\x0fplaceholder_row\x18\x1b \x01(\x0b\x32\".datafusion.PlaceholderRowExecNodeH\x00\x12/\n\x08\x63sv_sink\x18\x1c \x01(\x0b\x32\x1b.datafusion.CsvSinkExecNodeH\x00\x12\x37\n\x0cparquet_sink\x18\x1d \x01(\x0b\x32\x1f.datafusion.ParquetSinkExecNodeH\x00\x42\x12\n\x10PhysicalPlanType\"J\n\x0fPartitionColumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"\x85\x02\n\x15\x46ileTypeWriterOptions\x12\x35\n\x0cjson_options\x18\x01 \x01(\x0b\x32\x1d.datafusion.JsonWriterOptionsH\x00\x12;\n\x0fparquet_options\x18\x02 \x01(\x0b\x32 .datafusion.ParquetWriterOptionsH\x00\x12\x33\n\x0b\x63sv_options\x18\x03 \x01(\x0b\x32\x1c.datafusion.CsvWriterOptionsH\x00\x12\x37\n\rarrow_options\x18\x04 \x01(\x0b\x32\x1e.datafusion.ArrowWriterOptionsH\x00\x42\n\n\x08\x46ileType\"L\n\x11JsonWriterOptions\x12\x37\n\x0b\x63ompression\x18\x01 \x01(\x0e\x32\".datafusion.CompressionTypeVariant\"O\n\x14ParquetWriterOptions\x12\x37\n\x11writer_properties\x18\x01 \x01(\x0b\x32\x1c.datafusion.WriterProperties\"\xe3\x01\n\x10\x43svWriterOptions\x12\x37\n\x0b\x63ompression\x18\x01 \x01(\x0e\x32\".datafusion.CompressionTypeVariant\x12\x11\n\tdelimiter\x18\x02 \x01(\t\x12\x12\n\nhas_header\x18\x03 \x01(\x08\x12\x13\n\x0b\x64\x61te_format\x18\x04 \x01(\t\x12\x17\n\x0f\x64\x61tetime_format\x18\x05 \x01(\t\x12\x18\n\x10timestamp_format\x18\x06 \x01(\t\x12\x13\n\x0btime_format\x18\x07 \x01(\t\x12\x12\n\nnull_value\x18\x08 \x01(\t\"\x14\n\x12\x41rrowWriterOptions\"\xd9\x01\n\x10WriterProperties\x12\x1c\n\x14\x64\x61ta_page_size_limit\x18\x01 \x01(\x04\x12\"\n\x1a\x64ictionary_page_size_limit\x18\x02 \x01(\x04\x12!\n\x19\x64\x61ta_page_row_count_limit\x18\x03 \x01(\x04\x12\x18\n\x10write_batch_size\x18\x04 \x01(\x04\x12\x1a\n\x12max_row_group_size\x18\x05 \x01(\x04\x12\x16\n\x0ewriter_version\x18\x06 \x01(\t\x12\x12\n\ncreated_by\x18\x07 \x01(\t\"\xb5\x02\n\x0e\x46ileSinkConfig\x12\x18\n\x10object_store_url\x18\x01 \x01(\t\x12\x30\n\x0b\x66ile_groups\x18\x02 \x03(\x0b\x32\x1b.datafusion.PartitionedFile\x12\x13\n\x0btable_paths\x18\x03 \x03(\t\x12)\n\routput_schema\x18\x04 \x01(\x0b\x32\x12.datafusion.Schema\x12\x39\n\x14table_partition_cols\x18\x05 \x03(\x0b\x32\x1b.datafusion.PartitionColumn\x12\x11\n\toverwrite\x18\x08 \x01(\x08\x12\x43\n\x18\x66ile_type_writer_options\x18\t \x01(\x0b\x32!.datafusion.FileTypeWriterOptionsJ\x04\x08\x06\x10\x07\"6\n\x08JsonSink\x12*\n\x06\x63onfig\x18\x01 \x01(\x0b\x32\x1a.datafusion.FileSinkConfig\"\xcc\x01\n\x10JsonSinkExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\"\n\x04sink\x18\x02 \x01(\x0b\x32\x14.datafusion.JsonSink\x12\'\n\x0bsink_schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12>\n\nsort_order\x18\x04 \x01(\x0b\x32*.datafusion.PhysicalSortExprNodeCollection\"5\n\x07\x43svSink\x12*\n\x06\x63onfig\x18\x01 \x01(\x0b\x32\x1a.datafusion.FileSinkConfig\"\xca\x01\n\x0f\x43svSinkExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12!\n\x04sink\x18\x02 \x01(\x0b\x32\x13.datafusion.CsvSink\x12\'\n\x0bsink_schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12>\n\nsort_order\x18\x04 \x01(\x0b\x32*.datafusion.PhysicalSortExprNodeCollection\"9\n\x0bParquetSink\x12*\n\x06\x63onfig\x18\x01 \x01(\x0b\x32\x1a.datafusion.FileSinkConfig\"\xd2\x01\n\x13ParquetSinkExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12%\n\x04sink\x18\x02 \x01(\x0b\x32\x17.datafusion.ParquetSink\x12\'\n\x0bsink_schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\x12>\n\nsort_order\x18\x04 \x01(\x0b\x32*.datafusion.PhysicalSortExprNodeCollection\"S\n\x15PhysicalExtensionNode\x12\x0c\n\x04node\x18\x01 \x01(\x0c\x12,\n\x06inputs\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"\x80\x08\n\x10PhysicalExprNode\x12,\n\x06\x63olumn\x18\x01 \x01(\x0b\x32\x1a.datafusion.PhysicalColumnH\x00\x12*\n\x07literal\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValueH\x00\x12\x39\n\x0b\x62inary_expr\x18\x03 \x01(\x0b\x32\".datafusion.PhysicalBinaryExprNodeH\x00\x12?\n\x0e\x61ggregate_expr\x18\x04 \x01(\x0b\x32%.datafusion.PhysicalAggregateExprNodeH\x00\x12\x32\n\x0cis_null_expr\x18\x05 \x01(\x0b\x32\x1a.datafusion.PhysicalIsNullH\x00\x12\x39\n\x10is_not_null_expr\x18\x06 \x01(\x0b\x32\x1d.datafusion.PhysicalIsNotNullH\x00\x12+\n\x08not_expr\x18\x07 \x01(\x0b\x32\x17.datafusion.PhysicalNotH\x00\x12-\n\x05\x63\x61se_\x18\x08 \x01(\x0b\x32\x1c.datafusion.PhysicalCaseNodeH\x00\x12,\n\x04\x63\x61st\x18\t \x01(\x0b\x32\x1c.datafusion.PhysicalCastNodeH\x00\x12\x30\n\x04sort\x18\n \x01(\x0b\x32 .datafusion.PhysicalSortExprNodeH\x00\x12\x34\n\x08negative\x18\x0b \x01(\x0b\x32 .datafusion.PhysicalNegativeNodeH\x00\x12\x31\n\x07in_list\x18\x0c \x01(\x0b\x32\x1e.datafusion.PhysicalInListNodeH\x00\x12\x41\n\x0fscalar_function\x18\r \x01(\x0b\x32&.datafusion.PhysicalScalarFunctionNodeH\x00\x12\x33\n\x08try_cast\x18\x0e \x01(\x0b\x32\x1f.datafusion.PhysicalTryCastNodeH\x00\x12\x39\n\x0bwindow_expr\x18\x0f \x01(\x0b\x32\".datafusion.PhysicalWindowExprNodeH\x00\x12\x37\n\nscalar_udf\x18\x10 \x01(\x0b\x32!.datafusion.PhysicalScalarUdfNodeH\x00\x12\x35\n\tlike_expr\x18\x12 \x01(\x0b\x32 .datafusion.PhysicalLikeExprNodeH\x00\x12M\n\x16get_indexed_field_expr\x18\x13 \x01(\x0b\x32+.datafusion.PhysicalGetIndexedFieldExprNodeH\x00\x42\n\n\x08\x45xprTypeJ\x04\x08\x11\x10\x12\"}\n\x15PhysicalScalarUdfNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12*\n\x04\x61rgs\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x0breturn_type\x18\x04 \x01(\x0b\x32\x15.datafusion.ArrowType\"\x84\x02\n\x19PhysicalAggregateExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12$\n\x1auser_defined_aggr_function\x18\x04 \x01(\tH\x00\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x36\n\x0cordering_req\x18\x05 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\x12\x10\n\x08\x64istinct\x18\x03 \x01(\x08\x42\x13\n\x11\x41ggregateFunction\"\xf4\x02\n\x16PhysicalWindowExprNode\x12\x36\n\raggr_function\x18\x01 \x01(\x0e\x32\x1d.datafusion.AggregateFunctionH\x00\x12>\n\x11\x62uilt_in_function\x18\x02 \x01(\x0e\x32!.datafusion.BuiltInWindowFunctionH\x00\x12*\n\x04\x61rgs\x18\x04 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x32\n\x0cpartition_by\x18\x05 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x32\n\x08order_by\x18\x06 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\x12-\n\x0cwindow_frame\x18\x07 \x01(\x0b\x32\x17.datafusion.WindowFrame\x12\x0c\n\x04name\x18\x08 \x01(\tB\x11\n\x0fwindow_function\"<\n\x0ePhysicalIsNull\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"?\n\x11PhysicalIsNotNull\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"9\n\x0bPhysicalNot\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"N\n\x11PhysicalAliasNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x61lias\x18\x02 \x01(\t\"v\n\x16PhysicalBinaryExprNode\x12\'\n\x01l\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x01r\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"\x80\x01\n PhysicalDateTimeIntervalExprNode\x12\'\n\x01l\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x01r\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\n\n\x02op\x18\x03 \x01(\t\"\x9c\x01\n\x14PhysicalLikeExprNode\x12\x0f\n\x07negated\x18\x01 \x01(\x08\x12\x18\n\x10\x63\x61se_insensitive\x18\x02 \x01(\x08\x12*\n\x04\x65xpr\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12-\n\x07pattern\x18\x04 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"d\n\x14PhysicalSortExprNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0b\n\x03\x61sc\x18\x02 \x01(\x08\x12\x13\n\x0bnulls_first\x18\x03 \x01(\x08\"t\n\x10PhysicalWhenThen\x12/\n\twhen_expr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\tthen_expr\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"}\n\x12PhysicalInListNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x04list\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0f\n\x07negated\x18\x03 \x01(\x08\"\xa5\x01\n\x10PhysicalCaseNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x34\n\x0ewhen_then_expr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalWhenThen\x12/\n\telse_expr\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"\xab\x01\n\x1aPhysicalScalarFunctionNode\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\'\n\x03\x66un\x18\x02 \x01(\x0e\x32\x1a.datafusion.ScalarFunction\x12*\n\x04\x61rgs\x18\x03 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x0breturn_type\x18\x04 \x01(\x0b\x32\x15.datafusion.ArrowType\"l\n\x13PhysicalTryCastNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"i\n\x10PhysicalCastNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12)\n\narrow_type\x18\x02 \x01(\x0b\x32\x15.datafusion.ArrowType\"B\n\x14PhysicalNegativeNode\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"\x8d\x01\n\x0e\x46ilterExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\"\n\x1a\x64\x65\x66\x61ult_filter_selectivity\x18\x03 \x01(\r\"7\n\tFileGroup\x12*\n\x05\x66iles\x18\x01 \x03(\x0b\x32\x1b.datafusion.PartitionedFile\"\x1a\n\tScanLimit\x12\r\n\x05limit\x18\x01 \x01(\r\"d\n\x1ePhysicalSortExprNodeCollection\x12\x42\n\x18physical_sort_expr_nodes\x18\x01 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\"\xcb\x02\n\x10\x46ileScanExecConf\x12*\n\x0b\x66ile_groups\x18\x01 \x03(\x0b\x32\x15.datafusion.FileGroup\x12\"\n\x06schema\x18\x02 \x01(\x0b\x32\x12.datafusion.Schema\x12\x12\n\nprojection\x18\x04 \x03(\r\x12$\n\x05limit\x18\x05 \x01(\x0b\x32\x15.datafusion.ScanLimit\x12*\n\nstatistics\x18\x06 \x01(\x0b\x32\x16.datafusion.Statistics\x12\x1c\n\x14table_partition_cols\x18\x07 \x03(\t\x12\x18\n\x10object_store_url\x18\x08 \x01(\t\x12\x43\n\x0foutput_ordering\x18\t \x03(\x0b\x32*.datafusion.PhysicalSortExprNodeCollectionJ\x04\x08\n\x10\x0b\"}\n\x13ParquetScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\x12/\n\tpredicate\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNodeJ\x04\x08\x02\x10\x03\"\x9d\x01\n\x0f\x43svScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\x12\x12\n\nhas_header\x18\x02 \x01(\x08\x12\x11\n\tdelimiter\x18\x03 \x01(\t\x12\r\n\x05quote\x18\x04 \x01(\t\x12\x10\n\x06\x65scape\x18\x05 \x01(\tH\x00\x42\x11\n\x0foptional_escape\"C\n\x10\x41vroScanExecNode\x12/\n\tbase_conf\x18\x01 \x01(\x0b\x32\x1c.datafusion.FileScanExecConf\"\xa9\x02\n\x10HashJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x1e\n\x02on\x18\x03 \x03(\x0b\x32\x12.datafusion.JoinOn\x12\'\n\tjoin_type\x18\x04 \x01(\x0e\x32\x14.datafusion.JoinType\x12\x31\n\x0epartition_mode\x18\x06 \x01(\x0e\x32\x19.datafusion.PartitionMode\x12\x18\n\x10null_equals_null\x18\x07 \x01(\x08\x12&\n\x06\x66ilter\x18\x08 \x01(\x0b\x32\x16.datafusion.JoinFilter\"\xaf\x03\n\x19SymmetricHashJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x1e\n\x02on\x18\x03 \x03(\x0b\x32\x12.datafusion.JoinOn\x12\'\n\tjoin_type\x18\x04 \x01(\x0e\x32\x14.datafusion.JoinType\x12\x37\n\x0epartition_mode\x18\x06 \x01(\x0e\x32\x1f.datafusion.StreamPartitionMode\x12\x18\n\x10null_equals_null\x18\x07 \x01(\x08\x12&\n\x06\x66ilter\x18\x08 \x01(\x0b\x32\x16.datafusion.JoinFilter\x12\x39\n\x0fleft_sort_exprs\x18\t \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\x12:\n\x10right_sort_exprs\x18\n \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\"B\n\x12InterleaveExecNode\x12,\n\x06inputs\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"=\n\rUnionExecNode\x12,\n\x06inputs\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"~\n\x0f\x45xplainExecNode\x12\"\n\x06schema\x18\x01 \x01(\x0b\x32\x12.datafusion.Schema\x12\x36\n\x11stringified_plans\x18\x02 \x03(\x0b\x32\x1b.datafusion.StringifiedPlan\x12\x0f\n\x07verbose\x18\x03 \x01(\x08\"\x8c\x01\n\x0f\x41nalyzeExecNode\x12\x0f\n\x07verbose\x18\x01 \x01(\x08\x12\x17\n\x0fshow_statistics\x18\x02 \x01(\x08\x12+\n\x05input\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\"\n\x06schema\x18\x04 \x01(\x0b\x32\x12.datafusion.Schema\"l\n\x11\x43rossJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"-\n\x0ePhysicalColumn\x12\x0c\n\x04name\x18\x01 \x01(\t\x12\r\n\x05index\x18\x02 \x01(\r\"a\n\x06JoinOn\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"3\n\rEmptyExecNode\x12\"\n\x06schema\x18\x01 \x01(\x0b\x32\x12.datafusion.Schema\"<\n\x16PlaceholderRowExecNode\x12\"\n\x06schema\x18\x01 \x01(\x0b\x32\x12.datafusion.Schema\"\x80\x01\n\x12ProjectionExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x11\n\texpr_name\x18\x03 \x03(\t\"0\n\x1dPartiallySortedInputOrderMode\x12\x0f\n\x07\x63olumns\x18\x06 \x03(\x04\"\xe2\x02\n\x11WindowAggExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x37\n\x0bwindow_expr\x18\x02 \x03(\x0b\x32\".datafusion.PhysicalWindowExprNode\x12\x34\n\x0epartition_keys\x18\x05 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x06linear\x18\x07 \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x12\x45\n\x10partially_sorted\x18\x08 \x01(\x0b\x32).datafusion.PartiallySortedInputOrderModeH\x00\x12*\n\x06sorted\x18\t \x01(\x0b\x32\x18.datafusion.EmptyMessageH\x00\x42\x12\n\x10input_order_mode\"9\n\x0bMaybeFilter\x12*\n\x04\x65xpr\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"M\n\x16MaybePhysicalSortExprs\x12\x33\n\tsort_expr\x18\x01 \x03(\x0b\x32 .datafusion.PhysicalSortExprNode\"\x96\x03\n\x11\x41ggregateExecNode\x12\x30\n\ngroup_expr\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\taggr_expr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\'\n\x04mode\x18\x03 \x01(\x0e\x32\x19.datafusion.AggregateMode\x12+\n\x05input\x18\x04 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x17\n\x0fgroup_expr_name\x18\x05 \x03(\t\x12\x16\n\x0e\x61ggr_expr_name\x18\x06 \x03(\t\x12(\n\x0cinput_schema\x18\x07 \x01(\x0b\x32\x12.datafusion.Schema\x12/\n\tnull_expr\x18\x08 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x0e\n\x06groups\x18\t \x03(\x08\x12,\n\x0b\x66ilter_expr\x18\n \x03(\x0b\x32\x17.datafusion.MaybeFilter\"_\n\x13GlobalLimitExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x0c\n\x04skip\x18\x02 \x01(\r\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"P\n\x12LocalLimitExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\r\n\x05\x66\x65tch\x18\x02 \x01(\r\"\x95\x01\n\x0cSortExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\x12\x1d\n\x15preserve_partitioning\x18\x04 \x01(\x08\"\x85\x01\n\x1bSortPreservingMergeExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12*\n\x04\x65xpr\x18\x02 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\r\n\x05\x66\x65tch\x18\x03 \x01(\x03\"\xc2\x01\n\x16NestedLoopJoinExecNode\x12*\n\x04left\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12+\n\x05right\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\'\n\tjoin_type\x18\x03 \x01(\x0e\x32\x14.datafusion.JoinType\x12&\n\x06\x66ilter\x18\x04 \x01(\x0b\x32\x16.datafusion.JoinFilter\"a\n\x17\x43oalesceBatchesExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x19\n\x11target_batch_size\x18\x02 \x01(\r\"I\n\x1a\x43oalescePartitionsExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\"c\n\x17PhysicalHashRepartition\x12/\n\thash_expr\x18\x01 \x03(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x17\n\x0fpartition_count\x18\x02 \x01(\x04\"\xb5\x01\n\x13RepartitionExecNode\x12+\n\x05input\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalPlanNode\x12\x15\n\x0bround_robin\x18\x02 \x01(\x04H\x00\x12\x33\n\x04hash\x18\x03 \x01(\x0b\x32#.datafusion.PhysicalHashRepartitionH\x00\x12\x11\n\x07unknown\x18\x04 \x01(\x04H\x00\x42\x12\n\x10partition_method\"\x93\x01\n\nJoinFilter\x12\x30\n\nexpression\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12/\n\x0e\x63olumn_indices\x18\x02 \x03(\x0b\x32\x17.datafusion.ColumnIndex\x12\"\n\x06schema\x18\x03 \x01(\x0b\x32\x12.datafusion.Schema\"@\n\x0b\x43olumnIndex\x12\r\n\x05index\x18\x01 \x01(\r\x12\"\n\x04side\x18\x02 \x01(\x0e\x32\x14.datafusion.JoinSide\"\xa0\x01\n\x0fPartitionedFile\x12\x0c\n\x04path\x18\x01 \x01(\t\x12\x0c\n\x04size\x18\x02 \x01(\x04\x12\x18\n\x10last_modified_ns\x18\x03 \x01(\x04\x12\x31\n\x10partition_values\x18\x04 \x03(\x0b\x32\x17.datafusion.ScalarValue\x12$\n\x05range\x18\x05 \x01(\x0b\x32\x15.datafusion.FileRange\"\'\n\tFileRange\x12\r\n\x05start\x18\x01 \x01(\x03\x12\x0b\n\x03\x65nd\x18\x02 \x01(\x03\"y\n\x0ePartitionStats\x12\x10\n\x08num_rows\x18\x01 \x01(\x03\x12\x13\n\x0bnum_batches\x18\x02 \x01(\x03\x12\x11\n\tnum_bytes\x18\x03 \x01(\x03\x12-\n\x0c\x63olumn_stats\x18\x04 \x03(\x0b\x32\x17.datafusion.ColumnStats\"d\n\tPrecision\x12\x31\n\x0eprecision_info\x18\x01 \x01(\x0e\x32\x19.datafusion.PrecisionInfo\x12$\n\x03val\x18\x02 \x01(\x0b\x32\x17.datafusion.ScalarValue\"\x94\x01\n\nStatistics\x12\'\n\x08num_rows\x18\x01 \x01(\x0b\x32\x15.datafusion.Precision\x12.\n\x0ftotal_byte_size\x18\x02 \x01(\x0b\x32\x15.datafusion.Precision\x12-\n\x0c\x63olumn_stats\x18\x03 \x03(\x0b\x32\x17.datafusion.ColumnStats\"\xbb\x01\n\x0b\x43olumnStats\x12(\n\tmin_value\x18\x01 \x01(\x0b\x32\x15.datafusion.Precision\x12(\n\tmax_value\x18\x02 \x01(\x0b\x32\x15.datafusion.Precision\x12)\n\nnull_count\x18\x03 \x01(\x0b\x32\x15.datafusion.Precision\x12-\n\x0e\x64istinct_count\x18\x04 \x01(\x0b\x32\x15.datafusion.Precision\"=\n\x14NamedStructFieldExpr\x12%\n\x04name\x18\x01 \x01(\x0b\x32\x17.datafusion.ScalarValue\":\n\rListIndexExpr\x12)\n\x03key\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"\x96\x01\n\rListRangeExpr\x12+\n\x05start\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12*\n\x04stop\x18\x02 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12,\n\x06stride\x18\x03 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\"\x86\x02\n\x1fPhysicalGetIndexedFieldExprNode\x12)\n\x03\x61rg\x18\x01 \x01(\x0b\x32\x1c.datafusion.PhysicalExprNode\x12\x43\n\x17named_struct_field_expr\x18\x02 \x01(\x0b\x32 .datafusion.NamedStructFieldExprH\x00\x12\x34\n\x0flist_index_expr\x18\x03 \x01(\x0b\x32\x19.datafusion.ListIndexExprH\x00\x12\x34\n\x0flist_range_expr\x18\x04 \x01(\x0b\x32\x19.datafusion.ListRangeExprH\x00\x42\x07\n\x05\x66ield*n\n\x08JoinType\x12\t\n\x05INNER\x10\x00\x12\x08\n\x04LEFT\x10\x01\x12\t\n\x05RIGHT\x10\x02\x12\x08\n\x04\x46ULL\x10\x03\x12\x0c\n\x08LEFTSEMI\x10\x04\x12\x0c\n\x08LEFTANTI\x10\x05\x12\r\n\tRIGHTSEMI\x10\x06\x12\r\n\tRIGHTANTI\x10\x07*#\n\x0eJoinConstraint\x12\x06\n\x02ON\x10\x00\x12\t\n\x05USING\x10\x01*\xc0\x0e\n\x0eScalarFunction\x12\x07\n\x03\x41\x62s\x10\x00\x12\x08\n\x04\x41\x63os\x10\x01\x12\x08\n\x04\x41sin\x10\x02\x12\x08\n\x04\x41tan\x10\x03\x12\t\n\x05\x41scii\x10\x04\x12\x08\n\x04\x43\x65il\x10\x05\x12\x07\n\x03\x43os\x10\x06\x12\n\n\x06\x44igest\x10\x07\x12\x07\n\x03\x45xp\x10\x08\x12\t\n\x05\x46loor\x10\t\x12\x06\n\x02Ln\x10\n\x12\x07\n\x03Log\x10\x0b\x12\t\n\x05Log10\x10\x0c\x12\x08\n\x04Log2\x10\r\x12\t\n\x05Round\x10\x0e\x12\n\n\x06Signum\x10\x0f\x12\x07\n\x03Sin\x10\x10\x12\x08\n\x04Sqrt\x10\x11\x12\x07\n\x03Tan\x10\x12\x12\t\n\x05Trunc\x10\x13\x12\t\n\x05\x41rray\x10\x14\x12\x0f\n\x0bRegexpMatch\x10\x15\x12\r\n\tBitLength\x10\x16\x12\t\n\x05\x42trim\x10\x17\x12\x13\n\x0f\x43haracterLength\x10\x18\x12\x07\n\x03\x43hr\x10\x19\x12\n\n\x06\x43oncat\x10\x1a\x12\x17\n\x13\x43oncatWithSeparator\x10\x1b\x12\x0c\n\x08\x44\x61tePart\x10\x1c\x12\r\n\tDateTrunc\x10\x1d\x12\x0b\n\x07InitCap\x10\x1e\x12\x08\n\x04Left\x10\x1f\x12\x08\n\x04Lpad\x10 \x12\t\n\x05Lower\x10!\x12\t\n\x05Ltrim\x10\"\x12\x07\n\x03MD5\x10#\x12\x0f\n\x0bOctetLength\x10%\x12\n\n\x06Random\x10&\x12\x11\n\rRegexpReplace\x10\'\x12\n\n\x06Repeat\x10(\x12\x0b\n\x07Replace\x10)\x12\x0b\n\x07Reverse\x10*\x12\t\n\x05Right\x10+\x12\x08\n\x04Rpad\x10,\x12\t\n\x05Rtrim\x10-\x12\n\n\x06SHA224\x10.\x12\n\n\x06SHA256\x10/\x12\n\n\x06SHA384\x10\x30\x12\n\n\x06SHA512\x10\x31\x12\r\n\tSplitPart\x10\x32\x12\x0e\n\nStartsWith\x10\x33\x12\n\n\x06Strpos\x10\x34\x12\n\n\x06Substr\x10\x35\x12\t\n\x05ToHex\x10\x36\x12\x0f\n\x0bToTimestamp\x10\x37\x12\x15\n\x11ToTimestampMillis\x10\x38\x12\x15\n\x11ToTimestampMicros\x10\x39\x12\x16\n\x12ToTimestampSeconds\x10:\x12\x07\n\x03Now\x10;\x12\r\n\tTranslate\x10<\x12\x08\n\x04Trim\x10=\x12\t\n\x05Upper\x10>\x12\x0c\n\x08\x43oalesce\x10?\x12\t\n\x05Power\x10@\x12\r\n\tStructFun\x10\x41\x12\x10\n\x0c\x46romUnixtime\x10\x42\x12\t\n\x05\x41tan2\x10\x43\x12\x0b\n\x07\x44\x61teBin\x10\x44\x12\x0f\n\x0b\x41rrowTypeof\x10\x45\x12\x0f\n\x0b\x43urrentDate\x10\x46\x12\x0f\n\x0b\x43urrentTime\x10G\x12\x08\n\x04Uuid\x10H\x12\x08\n\x04\x43\x62rt\x10I\x12\t\n\x05\x41\x63osh\x10J\x12\t\n\x05\x41sinh\x10K\x12\t\n\x05\x41tanh\x10L\x12\x08\n\x04Sinh\x10M\x12\x08\n\x04\x43osh\x10N\x12\x08\n\x04Tanh\x10O\x12\x06\n\x02Pi\x10P\x12\x0b\n\x07\x44\x65grees\x10Q\x12\x0b\n\x07Radians\x10R\x12\r\n\tFactorial\x10S\x12\x07\n\x03Lcm\x10T\x12\x07\n\x03Gcd\x10U\x12\x0f\n\x0b\x41rrayAppend\x10V\x12\x0f\n\x0b\x41rrayConcat\x10W\x12\r\n\tArrayDims\x10X\x12\x0f\n\x0b\x41rrayRepeat\x10Y\x12\x0f\n\x0b\x41rrayLength\x10Z\x12\x0e\n\nArrayNdims\x10[\x12\x11\n\rArrayPosition\x10\\\x12\x12\n\x0e\x41rrayPositions\x10]\x12\x10\n\x0c\x41rrayPrepend\x10^\x12\x0f\n\x0b\x41rrayRemove\x10_\x12\x10\n\x0c\x41rrayReplace\x10`\x12\x0f\n\x0b\x43\x61rdinality\x10\x62\x12\x10\n\x0c\x41rrayElement\x10\x63\x12\x0e\n\nArraySlice\x10\x64\x12\x07\n\x03\x43ot\x10g\x12\x0c\n\x08\x41rrayHas\x10h\x12\x0f\n\x0b\x41rrayHasAny\x10i\x12\x0f\n\x0b\x41rrayHasAll\x10j\x12\x10\n\x0c\x41rrayRemoveN\x10k\x12\x11\n\rArrayReplaceN\x10l\x12\x12\n\x0e\x41rrayRemoveAll\x10m\x12\x13\n\x0f\x41rrayReplaceAll\x10n\x12\t\n\x05Nanvl\x10o\x12\x0b\n\x07\x46latten\x10p\x12\n\n\x06Iszero\x10r\x12\x0e\n\nArrayEmpty\x10s\x12\x10\n\x0c\x41rrayPopBack\x10t\x12\x11\n\rStringToArray\x10u\x12\x14\n\x10ToTimestampNanos\x10v\x12\x12\n\x0e\x41rrayIntersect\x10w\x12\x0e\n\nArrayUnion\x10x\x12\x0b\n\x07OverLay\x10y\x12\t\n\x05Range\x10z\x12\x0f\n\x0b\x41rrayExcept\x10{\x12\x11\n\rArrayPopFront\x10|\x12\x0f\n\x0bLevenshtein\x10}\x12\x0f\n\x0bSubstrIndex\x10~\x12\r\n\tFindInSet\x10\x7f\x12\x0e\n\tArraySort\x10\x80\x01\x12\x12\n\rArrayDistinct\x10\x81\x01\x12\x10\n\x0b\x41rrayResize\x10\x82\x01\x12\r\n\x08\x45ndsWith\x10\x83\x01\x12\n\n\x05InStr\x10\x84\x01\x12\r\n\x08MakeDate\x10\x85\x01\x12\x11\n\x0c\x41rrayReverse\x10\x86\x01\x12\x0f\n\nRegexpLike\x10\x87\x01\x12\x0b\n\x06ToChar\x10\x88\x01*\xde\x04\n\x11\x41ggregateFunction\x12\x07\n\x03MIN\x10\x00\x12\x07\n\x03MAX\x10\x01\x12\x07\n\x03SUM\x10\x02\x12\x07\n\x03\x41VG\x10\x03\x12\t\n\x05\x43OUNT\x10\x04\x12\x13\n\x0f\x41PPROX_DISTINCT\x10\x05\x12\r\n\tARRAY_AGG\x10\x06\x12\x0c\n\x08VARIANCE\x10\x07\x12\x10\n\x0cVARIANCE_POP\x10\x08\x12\x0e\n\nCOVARIANCE\x10\t\x12\x12\n\x0e\x43OVARIANCE_POP\x10\n\x12\n\n\x06STDDEV\x10\x0b\x12\x0e\n\nSTDDEV_POP\x10\x0c\x12\x0f\n\x0b\x43ORRELATION\x10\r\x12\x1a\n\x16\x41PPROX_PERCENTILE_CONT\x10\x0e\x12\x11\n\rAPPROX_MEDIAN\x10\x0f\x12&\n\"APPROX_PERCENTILE_CONT_WITH_WEIGHT\x10\x10\x12\x0c\n\x08GROUPING\x10\x11\x12\n\n\x06MEDIAN\x10\x12\x12\x0b\n\x07\x42IT_AND\x10\x13\x12\n\n\x06\x42IT_OR\x10\x14\x12\x0b\n\x07\x42IT_XOR\x10\x15\x12\x0c\n\x08\x42OOL_AND\x10\x16\x12\x0b\n\x07\x42OOL_OR\x10\x17\x12\x13\n\x0f\x46IRST_VALUE_AGG\x10\x18\x12\x12\n\x0eLAST_VALUE_AGG\x10\x19\x12\x0e\n\nREGR_SLOPE\x10\x1a\x12\x12\n\x0eREGR_INTERCEPT\x10\x1b\x12\x0e\n\nREGR_COUNT\x10\x1c\x12\x0b\n\x07REGR_R2\x10\x1d\x12\r\n\tREGR_AVGX\x10\x1e\x12\r\n\tREGR_AVGY\x10\x1f\x12\x0c\n\x08REGR_SXX\x10 \x12\x0c\n\x08REGR_SYY\x10!\x12\x0c\n\x08REGR_SXY\x10\"\x12\x0e\n\nSTRING_AGG\x10#\x12\x11\n\rNTH_VALUE_AGG\x10$*\xb0\x01\n\x15\x42uiltInWindowFunction\x12\x0e\n\nROW_NUMBER\x10\x00\x12\x08\n\x04RANK\x10\x01\x12\x0e\n\nDENSE_RANK\x10\x02\x12\x10\n\x0cPERCENT_RANK\x10\x03\x12\r\n\tCUME_DIST\x10\x04\x12\t\n\x05NTILE\x10\x05\x12\x07\n\x03LAG\x10\x06\x12\x08\n\x04LEAD\x10\x07\x12\x0f\n\x0b\x46IRST_VALUE\x10\x08\x12\x0e\n\nLAST_VALUE\x10\t\x12\r\n\tNTH_VALUE\x10\n*3\n\x10WindowFrameUnits\x12\x08\n\x04ROWS\x10\x00\x12\t\n\x05RANGE\x10\x01\x12\n\n\x06GROUPS\x10\x02*E\n\x14WindowFrameBoundType\x12\x0f\n\x0b\x43URRENT_ROW\x10\x00\x12\r\n\tPRECEDING\x10\x01\x12\r\n\tFOLLOWING\x10\x02*(\n\x08\x44\x61teUnit\x12\x07\n\x03\x44\x61y\x10\x00\x12\x13\n\x0f\x44\x61teMillisecond\x10\x01*H\n\x08TimeUnit\x12\n\n\x06Second\x10\x00\x12\x0f\n\x0bMillisecond\x10\x01\x12\x0f\n\x0bMicrosecond\x10\x02\x12\x0e\n\nNanosecond\x10\x03*<\n\x0cIntervalUnit\x12\r\n\tYearMonth\x10\x00\x12\x0b\n\x07\x44\x61yTime\x10\x01\x12\x10\n\x0cMonthDayNano\x10\x02*\"\n\tUnionMode\x12\n\n\x06sparse\x10\x00\x12\t\n\x05\x64\x65nse\x10\x01*Q\n\x16\x43ompressionTypeVariant\x12\x08\n\x04GZIP\x10\x00\x12\t\n\x05\x42ZIP2\x10\x01\x12\x06\n\x02XZ\x10\x02\x12\x08\n\x04ZSTD\x10\x03\x12\x10\n\x0cUNCOMPRESSED\x10\x04*<\n\rPartitionMode\x12\x10\n\x0c\x43OLLECT_LEFT\x10\x00\x12\x0f\n\x0bPARTITIONED\x10\x01\x12\x08\n\x04\x41UTO\x10\x02*A\n\x13StreamPartitionMode\x12\x14\n\x10SINGLE_PARTITION\x10\x00\x12\x14\n\x10PARTITIONED_EXEC\x10\x01*b\n\rAggregateMode\x12\x0b\n\x07PARTIAL\x10\x00\x12\t\n\x05\x46INAL\x10\x01\x12\x15\n\x11\x46INAL_PARTITIONED\x10\x02\x12\n\n\x06SINGLE\x10\x03\x12\x16\n\x12SINGLE_PARTITIONED\x10\x04*)\n\x08JoinSide\x12\r\n\tLEFT_SIDE\x10\x00\x12\x0e\n\nRIGHT_SIDE\x10\x01*3\n\rPrecisionInfo\x12\t\n\x05\x45XACT\x10\x00\x12\x0b\n\x07INEXACT\x10\x01\x12\n\n\x06\x41\x42SENT\x10\x02\x42\x39\n$org.apache.arrow.datafusion.protobufB\x0f\x44\x61tafusionProtoP\x01\x62\x06proto3') _globals = globals() _builder.BuildMessageAndEnumDescriptors(DESCRIPTOR, _globals) @@ -26,38 +26,46 @@ _DFSCHEMA_METADATAENTRY._serialized_options = b'8\001' _CREATEEXTERNALTABLENODE_OPTIONSENTRY._options = None _CREATEEXTERNALTABLENODE_OPTIONSENTRY._serialized_options = b'8\001' + _CREATEEXTERNALTABLENODE_COLUMNDEFAULTSENTRY._options = None + _CREATEEXTERNALTABLENODE_COLUMNDEFAULTSENTRY._serialized_options = b'8\001' _SCHEMA_METADATAENTRY._options = None _SCHEMA_METADATAENTRY._serialized_options = b'8\001' _FIELD_METADATAENTRY._options = None _FIELD_METADATAENTRY._serialized_options = b'8\001' - _globals['_JOINTYPE']._serialized_start=26361 - _globals['_JOINTYPE']._serialized_end=26471 - _globals['_JOINCONSTRAINT']._serialized_start=26473 - _globals['_JOINCONSTRAINT']._serialized_end=26508 - _globals['_SCALARFUNCTION']._serialized_start=26511 - _globals['_SCALARFUNCTION']._serialized_end=27893 - _globals['_AGGREGATEFUNCTION']._serialized_start=27896 - _globals['_AGGREGATEFUNCTION']._serialized_end=28330 - _globals['_BUILTINWINDOWFUNCTION']._serialized_start=28333 - _globals['_BUILTINWINDOWFUNCTION']._serialized_end=28509 - _globals['_WINDOWFRAMEUNITS']._serialized_start=28511 - _globals['_WINDOWFRAMEUNITS']._serialized_end=28562 - _globals['_WINDOWFRAMEBOUNDTYPE']._serialized_start=28564 - _globals['_WINDOWFRAMEBOUNDTYPE']._serialized_end=28633 - _globals['_DATEUNIT']._serialized_start=28635 - _globals['_DATEUNIT']._serialized_end=28675 - _globals['_TIMEUNIT']._serialized_start=28677 - _globals['_TIMEUNIT']._serialized_end=28749 - _globals['_INTERVALUNIT']._serialized_start=28751 - _globals['_INTERVALUNIT']._serialized_end=28811 - _globals['_UNIONMODE']._serialized_start=28813 - _globals['_UNIONMODE']._serialized_end=28847 - _globals['_PARTITIONMODE']._serialized_start=28849 - _globals['_PARTITIONMODE']._serialized_end=28909 - _globals['_AGGREGATEMODE']._serialized_start=28911 - _globals['_AGGREGATEMODE']._serialized_end=29009 - _globals['_JOINSIDE']._serialized_start=29011 - _globals['_JOINSIDE']._serialized_end=29052 + _globals['_JOINTYPE']._serialized_start=32527 + _globals['_JOINTYPE']._serialized_end=32637 + _globals['_JOINCONSTRAINT']._serialized_start=32639 + _globals['_JOINCONSTRAINT']._serialized_end=32674 + _globals['_SCALARFUNCTION']._serialized_start=32677 + _globals['_SCALARFUNCTION']._serialized_end=34533 + _globals['_AGGREGATEFUNCTION']._serialized_start=34536 + _globals['_AGGREGATEFUNCTION']._serialized_end=35142 + _globals['_BUILTINWINDOWFUNCTION']._serialized_start=35145 + _globals['_BUILTINWINDOWFUNCTION']._serialized_end=35321 + _globals['_WINDOWFRAMEUNITS']._serialized_start=35323 + _globals['_WINDOWFRAMEUNITS']._serialized_end=35374 + _globals['_WINDOWFRAMEBOUNDTYPE']._serialized_start=35376 + _globals['_WINDOWFRAMEBOUNDTYPE']._serialized_end=35445 + _globals['_DATEUNIT']._serialized_start=35447 + _globals['_DATEUNIT']._serialized_end=35487 + _globals['_TIMEUNIT']._serialized_start=35489 + _globals['_TIMEUNIT']._serialized_end=35561 + _globals['_INTERVALUNIT']._serialized_start=35563 + _globals['_INTERVALUNIT']._serialized_end=35623 + _globals['_UNIONMODE']._serialized_start=35625 + _globals['_UNIONMODE']._serialized_end=35659 + _globals['_COMPRESSIONTYPEVARIANT']._serialized_start=35661 + _globals['_COMPRESSIONTYPEVARIANT']._serialized_end=35742 + _globals['_PARTITIONMODE']._serialized_start=35744 + _globals['_PARTITIONMODE']._serialized_end=35804 + _globals['_STREAMPARTITIONMODE']._serialized_start=35806 + _globals['_STREAMPARTITIONMODE']._serialized_end=35871 + _globals['_AGGREGATEMODE']._serialized_start=35873 + _globals['_AGGREGATEMODE']._serialized_end=35971 + _globals['_JOINSIDE']._serialized_start=35973 + _globals['_JOINSIDE']._serialized_end=36014 + _globals['_PRECISIONINFO']._serialized_start=36016 + _globals['_PRECISIONINFO']._serialized_end=36067 _globals['_COLUMNRELATION']._serialized_start=38 _globals['_COLUMNRELATION']._serialized_end=72 _globals['_COLUMN']._serialized_start=74 @@ -69,329 +77,403 @@ _globals['_DFSCHEMA_METADATAENTRY']._serialized_start=341 _globals['_DFSCHEMA_METADATAENTRY']._serialized_end=388 _globals['_LOGICALPLANNODE']._serialized_start=391 - _globals['_LOGICALPLANNODE']._serialized_end=1722 - _globals['_LOGICALEXTENSIONNODE']._serialized_start=1724 - _globals['_LOGICALEXTENSIONNODE']._serialized_end=1805 - _globals['_PROJECTIONCOLUMNS']._serialized_start=1807 - _globals['_PROJECTIONCOLUMNS']._serialized_end=1843 - _globals['_CSVFORMAT']._serialized_start=1845 - _globals['_CSVFORMAT']._serialized_end=1895 - _globals['_PARQUETFORMAT']._serialized_start=1897 - _globals['_PARQUETFORMAT']._serialized_end=1918 - _globals['_AVROFORMAT']._serialized_start=1920 - _globals['_AVROFORMAT']._serialized_end=1932 - _globals['_LOGICALEXPRNODECOLLECTION']._serialized_start=1934 - _globals['_LOGICALEXPRNODECOLLECTION']._serialized_end=2018 - _globals['_LISTINGTABLESCANNODE']._serialized_start=2021 - _globals['_LISTINGTABLESCANNODE']._serialized_end=2559 - _globals['_VIEWTABLESCANNODE']._serialized_start=2562 - _globals['_VIEWTABLESCANNODE']._serialized_end=2791 - _globals['_CUSTOMTABLESCANNODE']._serialized_start=2794 - _globals['_CUSTOMTABLESCANNODE']._serialized_end=3034 - _globals['_PROJECTIONNODE']._serialized_start=3037 - _globals['_PROJECTIONNODE']._serialized_end=3175 - _globals['_SELECTIONNODE']._serialized_start=3177 - _globals['_SELECTIONNODE']._serialized_end=3279 - _globals['_SORTNODE']._serialized_start=3281 - _globals['_SORTNODE']._serialized_end=3393 - _globals['_REPARTITIONNODE']._serialized_start=3396 - _globals['_REPARTITIONNODE']._serialized_end=3545 - _globals['_HASHREPARTITION']._serialized_start=3547 - _globals['_HASHREPARTITION']._serialized_end=3637 - _globals['_EMPTYRELATIONNODE']._serialized_start=3639 - _globals['_EMPTYRELATIONNODE']._serialized_end=3683 - _globals['_CREATEEXTERNALTABLENODE']._serialized_start=3686 - _globals['_CREATEEXTERNALTABLENODE']._serialized_end=4176 - _globals['_CREATEEXTERNALTABLENODE_OPTIONSENTRY']._serialized_start=4124 - _globals['_CREATEEXTERNALTABLENODE_OPTIONSENTRY']._serialized_end=4170 - _globals['_PREPARENODE']._serialized_start=4178 - _globals['_PREPARENODE']._serialized_end=4292 - _globals['_CREATECATALOGSCHEMANODE']._serialized_start=4294 - _globals['_CREATECATALOGSCHEMANODE']._serialized_end=4401 - _globals['_CREATECATALOGNODE']._serialized_start=4403 - _globals['_CREATECATALOGNODE']._serialized_end=4505 - _globals['_DROPVIEWNODE']._serialized_start=4507 - _globals['_DROPVIEWNODE']._serialized_end=4625 - _globals['_CREATEVIEWNODE']._serialized_start=4628 - _globals['_CREATEVIEWNODE']._serialized_end=4781 - _globals['_VALUESNODE']._serialized_start=4783 - _globals['_VALUESNODE']._serialized_end=4861 - _globals['_ANALYZENODE']._serialized_start=4863 - _globals['_ANALYZENODE']._serialized_end=4937 - _globals['_EXPLAINNODE']._serialized_start=4939 - _globals['_EXPLAINNODE']._serialized_end=5013 - _globals['_AGGREGATENODE']._serialized_start=5016 - _globals['_AGGREGATENODE']._serialized_end=5172 - _globals['_WINDOWNODE']._serialized_start=5174 - _globals['_WINDOWNODE']._serialized_end=5280 - _globals['_JOINNODE']._serialized_start=5283 - _globals['_JOINNODE']._serialized_end=5650 - _globals['_DISTINCTNODE']._serialized_start=5652 - _globals['_DISTINCTNODE']._serialized_end=5710 - _globals['_UNIONNODE']._serialized_start=5712 - _globals['_UNIONNODE']._serialized_end=5768 - _globals['_CROSSJOINNODE']._serialized_start=5770 - _globals['_CROSSJOINNODE']._serialized_end=5872 - _globals['_LIMITNODE']._serialized_start=5874 - _globals['_LIMITNODE']._serialized_end=5958 - _globals['_SELECTIONEXECNODE']._serialized_start=5960 - _globals['_SELECTIONEXECNODE']._serialized_end=6022 - _globals['_SUBQUERYALIASNODE']._serialized_start=6024 - _globals['_SUBQUERYALIASNODE']._serialized_end=6141 - _globals['_LOGICALEXPRNODE']._serialized_start=6144 - _globals['_LOGICALEXPRNODE']._serialized_end=7725 - _globals['_PLACEHOLDERNODE']._serialized_start=7727 - _globals['_PLACEHOLDERNODE']._serialized_end=7798 - _globals['_LOGICALEXPRLIST']._serialized_start=7800 - _globals['_LOGICALEXPRLIST']._serialized_end=7860 - _globals['_GROUPINGSETNODE']._serialized_start=7862 - _globals['_GROUPINGSETNODE']._serialized_end=7922 - _globals['_CUBENODE']._serialized_start=7924 - _globals['_CUBENODE']._serialized_end=7977 - _globals['_ROLLUPNODE']._serialized_start=7979 - _globals['_ROLLUPNODE']._serialized_end=8034 - _globals['_GETINDEXEDFIELD']._serialized_start=8036 - _globals['_GETINDEXEDFIELD']._serialized_end=8134 - _globals['_ISNULL']._serialized_start=8136 - _globals['_ISNULL']._serialized_end=8187 - _globals['_ISNOTNULL']._serialized_start=8189 - _globals['_ISNOTNULL']._serialized_end=8243 - _globals['_ISTRUE']._serialized_start=8245 - _globals['_ISTRUE']._serialized_end=8296 - _globals['_ISFALSE']._serialized_start=8298 - _globals['_ISFALSE']._serialized_end=8350 - _globals['_ISUNKNOWN']._serialized_start=8352 - _globals['_ISUNKNOWN']._serialized_end=8406 - _globals['_ISNOTTRUE']._serialized_start=8408 - _globals['_ISNOTTRUE']._serialized_end=8462 - _globals['_ISNOTFALSE']._serialized_start=8464 - _globals['_ISNOTFALSE']._serialized_end=8519 - _globals['_ISNOTUNKNOWN']._serialized_start=8521 - _globals['_ISNOTUNKNOWN']._serialized_end=8578 - _globals['_NOT']._serialized_start=8580 - _globals['_NOT']._serialized_end=8628 - _globals['_ALIASNODE']._serialized_start=8630 - _globals['_ALIASNODE']._serialized_end=8699 - _globals['_BINARYEXPRNODE']._serialized_start=8701 - _globals['_BINARYEXPRNODE']._serialized_end=8776 - _globals['_NEGATIVENODE']._serialized_start=8778 - _globals['_NEGATIVENODE']._serialized_end=8835 - _globals['_INLISTNODE']._serialized_start=8837 - _globals['_INLISTNODE']._serialized_end=8952 - _globals['_SCALARFUNCTIONNODE']._serialized_start=8954 - _globals['_SCALARFUNCTIONNODE']._serialized_end=9058 - _globals['_AGGREGATEEXPRNODE']._serialized_start=9061 - _globals['_AGGREGATEEXPRNODE']._serialized_end=9287 - _globals['_AGGREGATEUDFEXPRNODE']._serialized_start=9290 - _globals['_AGGREGATEUDFEXPRNODE']._serialized_end=9465 - _globals['_SCALARUDFEXPRNODE']._serialized_start=9467 - _globals['_SCALARUDFEXPRNODE']._serialized_end=9547 - _globals['_WINDOWEXPRNODE']._serialized_start=9550 - _globals['_WINDOWEXPRNODE']._serialized_end=9925 - _globals['_BETWEENNODE']._serialized_start=9928 - _globals['_BETWEENNODE']._serialized_end=10086 - _globals['_LIKENODE']._serialized_start=10089 - _globals['_LIKENODE']._serialized_end=10226 - _globals['_ILIKENODE']._serialized_start=10229 - _globals['_ILIKENODE']._serialized_end=10367 - _globals['_SIMILARTONODE']._serialized_start=10370 - _globals['_SIMILARTONODE']._serialized_end=10512 - _globals['_CASENODE']._serialized_start=10515 - _globals['_CASENODE']._serialized_end=10662 - _globals['_WHENTHEN']._serialized_start=10664 - _globals['_WHENTHEN']._serialized_end=10770 - _globals['_CASTNODE']._serialized_start=10772 - _globals['_CASTNODE']._serialized_end=10868 - _globals['_TRYCASTNODE']._serialized_start=10870 - _globals['_TRYCASTNODE']._serialized_end=10969 - _globals['_SORTEXPRNODE']._serialized_start=10971 - _globals['_SORTEXPRNODE']._serialized_end=11062 - _globals['_WINDOWFRAME']._serialized_start=11065 - _globals['_WINDOWFRAME']._serialized_end=11247 - _globals['_WINDOWFRAMEBOUND']._serialized_start=11250 - _globals['_WINDOWFRAMEBOUND']._serialized_end=11381 - _globals['_SCHEMA']._serialized_start=11384 - _globals['_SCHEMA']._serialized_end=11529 + _globals['_LOGICALPLANNODE']._serialized_end=1816 + _globals['_LOGICALEXTENSIONNODE']._serialized_start=1818 + _globals['_LOGICALEXTENSIONNODE']._serialized_end=1899 + _globals['_PROJECTIONCOLUMNS']._serialized_start=1901 + _globals['_PROJECTIONCOLUMNS']._serialized_end=1937 + _globals['_CSVFORMAT']._serialized_start=1939 + _globals['_CSVFORMAT']._serialized_end=2041 + _globals['_PARQUETFORMAT']._serialized_start=2043 + _globals['_PARQUETFORMAT']._serialized_end=2064 + _globals['_AVROFORMAT']._serialized_start=2066 + _globals['_AVROFORMAT']._serialized_end=2078 + _globals['_LOGICALEXPRNODECOLLECTION']._serialized_start=2080 + _globals['_LOGICALEXPRNODECOLLECTION']._serialized_end=2164 + _globals['_LISTINGTABLESCANNODE']._serialized_start=2167 + _globals['_LISTINGTABLESCANNODE']._serialized_end=2705 + _globals['_VIEWTABLESCANNODE']._serialized_start=2708 + _globals['_VIEWTABLESCANNODE']._serialized_end=2937 + _globals['_CUSTOMTABLESCANNODE']._serialized_start=2940 + _globals['_CUSTOMTABLESCANNODE']._serialized_end=3180 + _globals['_PROJECTIONNODE']._serialized_start=3183 + _globals['_PROJECTIONNODE']._serialized_end=3321 + _globals['_SELECTIONNODE']._serialized_start=3323 + _globals['_SELECTIONNODE']._serialized_end=3425 + _globals['_SORTNODE']._serialized_start=3427 + _globals['_SORTNODE']._serialized_end=3539 + _globals['_REPARTITIONNODE']._serialized_start=3542 + _globals['_REPARTITIONNODE']._serialized_end=3691 + _globals['_HASHREPARTITION']._serialized_start=3693 + _globals['_HASHREPARTITION']._serialized_end=3783 + _globals['_EMPTYRELATIONNODE']._serialized_start=3785 + _globals['_EMPTYRELATIONNODE']._serialized_end=3829 + _globals['_PRIMARYKEYCONSTRAINT']._serialized_start=3831 + _globals['_PRIMARYKEYCONSTRAINT']._serialized_end=3870 + _globals['_UNIQUECONSTRAINT']._serialized_start=3872 + _globals['_UNIQUECONSTRAINT']._serialized_end=3907 + _globals['_CONSTRAINT']._serialized_start=3910 + _globals['_CONSTRAINT']._serialized_end=4046 + _globals['_CONSTRAINTS']._serialized_start=4048 + _globals['_CONSTRAINTS']._serialized_end=4106 + _globals['_CREATEEXTERNALTABLENODE']._serialized_start=4109 + _globals['_CREATEEXTERNALTABLENODE']._serialized_end=4853 + _globals['_CREATEEXTERNALTABLENODE_OPTIONSENTRY']._serialized_start=4711 + _globals['_CREATEEXTERNALTABLENODE_OPTIONSENTRY']._serialized_end=4757 + _globals['_CREATEEXTERNALTABLENODE_COLUMNDEFAULTSENTRY']._serialized_start=4759 + _globals['_CREATEEXTERNALTABLENODE_COLUMNDEFAULTSENTRY']._serialized_end=4841 + _globals['_PREPARENODE']._serialized_start=4855 + _globals['_PREPARENODE']._serialized_end=4969 + _globals['_CREATECATALOGSCHEMANODE']._serialized_start=4971 + _globals['_CREATECATALOGSCHEMANODE']._serialized_end=5078 + _globals['_CREATECATALOGNODE']._serialized_start=5080 + _globals['_CREATECATALOGNODE']._serialized_end=5182 + _globals['_DROPVIEWNODE']._serialized_start=5184 + _globals['_DROPVIEWNODE']._serialized_end=5302 + _globals['_CREATEVIEWNODE']._serialized_start=5305 + _globals['_CREATEVIEWNODE']._serialized_end=5458 + _globals['_VALUESNODE']._serialized_start=5460 + _globals['_VALUESNODE']._serialized_end=5538 + _globals['_ANALYZENODE']._serialized_start=5540 + _globals['_ANALYZENODE']._serialized_end=5614 + _globals['_EXPLAINNODE']._serialized_start=5616 + _globals['_EXPLAINNODE']._serialized_end=5690 + _globals['_AGGREGATENODE']._serialized_start=5693 + _globals['_AGGREGATENODE']._serialized_end=5849 + _globals['_WINDOWNODE']._serialized_start=5851 + _globals['_WINDOWNODE']._serialized_end=5957 + _globals['_JOINNODE']._serialized_start=5960 + _globals['_JOINNODE']._serialized_end=6327 + _globals['_DISTINCTNODE']._serialized_start=6329 + _globals['_DISTINCTNODE']._serialized_end=6387 + _globals['_DISTINCTONNODE']._serialized_start=6390 + _globals['_DISTINCTONNODE']._serialized_end=6594 + _globals['_COPYTONODE']._serialized_start=6597 + _globals['_COPYTONODE']._serialized_end=6815 + _globals['_SQLOPTIONS']._serialized_start=6817 + _globals['_SQLOPTIONS']._serialized_end=6868 + _globals['_SQLOPTION']._serialized_start=6870 + _globals['_SQLOPTION']._serialized_end=6909 + _globals['_UNIONNODE']._serialized_start=6911 + _globals['_UNIONNODE']._serialized_end=6967 + _globals['_CROSSJOINNODE']._serialized_start=6969 + _globals['_CROSSJOINNODE']._serialized_end=7071 + _globals['_LIMITNODE']._serialized_start=7073 + _globals['_LIMITNODE']._serialized_end=7157 + _globals['_SELECTIONEXECNODE']._serialized_start=7159 + _globals['_SELECTIONEXECNODE']._serialized_end=7221 + _globals['_SUBQUERYALIASNODE']._serialized_start=7223 + _globals['_SUBQUERYALIASNODE']._serialized_end=7340 + _globals['_LOGICALEXPRNODE']._serialized_start=7343 + _globals['_LOGICALEXPRNODE']._serialized_end=8984 + _globals['_WILDCARD']._serialized_start=8986 + _globals['_WILDCARD']._serialized_end=9015 + _globals['_PLACEHOLDERNODE']._serialized_start=9017 + _globals['_PLACEHOLDERNODE']._serialized_end=9088 + _globals['_LOGICALEXPRLIST']._serialized_start=9090 + _globals['_LOGICALEXPRLIST']._serialized_end=9150 + _globals['_GROUPINGSETNODE']._serialized_start=9152 + _globals['_GROUPINGSETNODE']._serialized_end=9212 + _globals['_CUBENODE']._serialized_start=9214 + _globals['_CUBENODE']._serialized_end=9267 + _globals['_ROLLUPNODE']._serialized_start=9269 + _globals['_ROLLUPNODE']._serialized_end=9324 + _globals['_NAMEDSTRUCTFIELD']._serialized_start=9326 + _globals['_NAMEDSTRUCTFIELD']._serialized_end=9383 + _globals['_LISTINDEX']._serialized_start=9385 + _globals['_LISTINDEX']._serialized_end=9438 + _globals['_LISTRANGE']._serialized_start=9441 + _globals['_LISTRANGE']._serialized_end=9584 + _globals['_GETINDEXEDFIELD']._serialized_start=9587 + _globals['_GETINDEXEDFIELD']._serialized_end=9806 + _globals['_ISNULL']._serialized_start=9808 + _globals['_ISNULL']._serialized_end=9859 + _globals['_ISNOTNULL']._serialized_start=9861 + _globals['_ISNOTNULL']._serialized_end=9915 + _globals['_ISTRUE']._serialized_start=9917 + _globals['_ISTRUE']._serialized_end=9968 + _globals['_ISFALSE']._serialized_start=9970 + _globals['_ISFALSE']._serialized_end=10022 + _globals['_ISUNKNOWN']._serialized_start=10024 + _globals['_ISUNKNOWN']._serialized_end=10078 + _globals['_ISNOTTRUE']._serialized_start=10080 + _globals['_ISNOTTRUE']._serialized_end=10134 + _globals['_ISNOTFALSE']._serialized_start=10136 + _globals['_ISNOTFALSE']._serialized_end=10191 + _globals['_ISNOTUNKNOWN']._serialized_start=10193 + _globals['_ISNOTUNKNOWN']._serialized_end=10250 + _globals['_NOT']._serialized_start=10252 + _globals['_NOT']._serialized_end=10300 + _globals['_ALIASNODE']._serialized_start=10302 + _globals['_ALIASNODE']._serialized_end=10422 + _globals['_BINARYEXPRNODE']._serialized_start=10424 + _globals['_BINARYEXPRNODE']._serialized_end=10499 + _globals['_NEGATIVENODE']._serialized_start=10501 + _globals['_NEGATIVENODE']._serialized_end=10558 + _globals['_UNNEST']._serialized_start=10560 + _globals['_UNNEST']._serialized_end=10612 + _globals['_INLISTNODE']._serialized_start=10614 + _globals['_INLISTNODE']._serialized_end=10729 + _globals['_SCALARFUNCTIONNODE']._serialized_start=10731 + _globals['_SCALARFUNCTIONNODE']._serialized_end=10835 + _globals['_AGGREGATEEXPRNODE']._serialized_start=10838 + _globals['_AGGREGATEEXPRNODE']._serialized_end=11064 + _globals['_AGGREGATEUDFEXPRNODE']._serialized_start=11067 + _globals['_AGGREGATEUDFEXPRNODE']._serialized_end=11242 + _globals['_SCALARUDFEXPRNODE']._serialized_start=11244 + _globals['_SCALARUDFEXPRNODE']._serialized_end=11324 + _globals['_WINDOWEXPRNODE']._serialized_start=11327 + _globals['_WINDOWEXPRNODE']._serialized_end=11702 + _globals['_BETWEENNODE']._serialized_start=11705 + _globals['_BETWEENNODE']._serialized_end=11863 + _globals['_LIKENODE']._serialized_start=11866 + _globals['_LIKENODE']._serialized_end=12003 + _globals['_ILIKENODE']._serialized_start=12006 + _globals['_ILIKENODE']._serialized_end=12144 + _globals['_SIMILARTONODE']._serialized_start=12147 + _globals['_SIMILARTONODE']._serialized_end=12289 + _globals['_CASENODE']._serialized_start=12292 + _globals['_CASENODE']._serialized_end=12439 + _globals['_WHENTHEN']._serialized_start=12441 + _globals['_WHENTHEN']._serialized_end=12547 + _globals['_CASTNODE']._serialized_start=12549 + _globals['_CASTNODE']._serialized_end=12645 + _globals['_TRYCASTNODE']._serialized_start=12647 + _globals['_TRYCASTNODE']._serialized_end=12746 + _globals['_SORTEXPRNODE']._serialized_start=12748 + _globals['_SORTEXPRNODE']._serialized_end=12839 + _globals['_WINDOWFRAME']._serialized_start=12842 + _globals['_WINDOWFRAME']._serialized_end=13024 + _globals['_WINDOWFRAMEBOUND']._serialized_start=13027 + _globals['_WINDOWFRAMEBOUND']._serialized_end=13158 + _globals['_SCHEMA']._serialized_start=13161 + _globals['_SCHEMA']._serialized_end=13306 _globals['_SCHEMA_METADATAENTRY']._serialized_start=341 _globals['_SCHEMA_METADATAENTRY']._serialized_end=388 - _globals['_FIELD']._serialized_start=11532 - _globals['_FIELD']._serialized_end=11751 + _globals['_FIELD']._serialized_start=13309 + _globals['_FIELD']._serialized_end=13567 _globals['_FIELD_METADATAENTRY']._serialized_start=341 _globals['_FIELD_METADATAENTRY']._serialized_end=388 - _globals['_FIXEDSIZEBINARY']._serialized_start=11753 - _globals['_FIXEDSIZEBINARY']._serialized_end=11786 - _globals['_TIMESTAMP']._serialized_start=11788 - _globals['_TIMESTAMP']._serialized_end=11858 - _globals['_DECIMAL']._serialized_start=11860 - _globals['_DECIMAL']._serialized_end=11915 - _globals['_LIST']._serialized_start=11917 - _globals['_LIST']._serialized_end=11962 - _globals['_FIXEDSIZELIST']._serialized_start=11964 - _globals['_FIXEDSIZELIST']._serialized_end=12037 - _globals['_DICTIONARY']._serialized_start=12039 - _globals['_DICTIONARY']._serialized_end=12125 - _globals['_STRUCT']._serialized_start=12127 - _globals['_STRUCT']._serialized_end=12179 - _globals['_MAP']._serialized_start=12181 - _globals['_MAP']._serialized_end=12246 - _globals['_UNION']._serialized_start=12248 - _globals['_UNION']._serialized_end=12356 - _globals['_SCALARLISTVALUE']._serialized_start=12358 - _globals['_SCALARLISTVALUE']._serialized_end=12467 - _globals['_SCALARTIME32VALUE']._serialized_start=12469 - _globals['_SCALARTIME32VALUE']._serialized_end=12564 - _globals['_SCALARTIME64VALUE']._serialized_start=12566 - _globals['_SCALARTIME64VALUE']._serialized_end=12665 - _globals['_SCALARTIMESTAMPVALUE']._serialized_start=12668 - _globals['_SCALARTIMESTAMPVALUE']._serialized_end=12847 - _globals['_SCALARDICTIONARYVALUE']._serialized_start=12849 - _globals['_SCALARDICTIONARYVALUE']._serialized_end=12955 - _globals['_INTERVALMONTHDAYNANOVALUE']._serialized_start=12957 - _globals['_INTERVALMONTHDAYNANOVALUE']._serialized_end=13029 - _globals['_STRUCTVALUE']._serialized_start=13031 - _globals['_STRUCTVALUE']._serialized_end=13126 - _globals['_SCALARFIXEDSIZEBINARY']._serialized_start=13128 - _globals['_SCALARFIXEDSIZEBINARY']._serialized_end=13183 - _globals['_SCALARVALUE']._serialized_start=13186 - _globals['_SCALARVALUE']._serialized_end=14417 - _globals['_DECIMAL128']._serialized_start=14419 - _globals['_DECIMAL128']._serialized_end=14468 - _globals['_ARROWTYPE']._serialized_start=14471 - _globals['_ARROWTYPE']._serialized_end=15899 - _globals['_EMPTYMESSAGE']._serialized_start=15901 - _globals['_EMPTYMESSAGE']._serialized_end=15915 - _globals['_ANALYZEDLOGICALPLANTYPE']._serialized_start=15917 - _globals['_ANALYZEDLOGICALPLANTYPE']._serialized_end=15965 - _globals['_OPTIMIZEDLOGICALPLANTYPE']._serialized_start=15967 - _globals['_OPTIMIZEDLOGICALPLANTYPE']._serialized_end=16017 - _globals['_OPTIMIZEDPHYSICALPLANTYPE']._serialized_start=16019 - _globals['_OPTIMIZEDPHYSICALPLANTYPE']._serialized_end=16070 - _globals['_PLANTYPE']._serialized_start=16073 - _globals['_PLANTYPE']._serialized_end=16595 - _globals['_STRINGIFIEDPLAN']._serialized_start=16597 - _globals['_STRINGIFIEDPLAN']._serialized_end=16669 - _globals['_BARETABLEREFERENCE']._serialized_start=16671 - _globals['_BARETABLEREFERENCE']._serialized_end=16706 - _globals['_PARTIALTABLEREFERENCE']._serialized_start=16708 - _globals['_PARTIALTABLEREFERENCE']._serialized_end=16762 - _globals['_FULLTABLEREFERENCE']._serialized_start=16764 - _globals['_FULLTABLEREFERENCE']._serialized_end=16832 - _globals['_OWNEDTABLEREFERENCE']._serialized_start=16835 - _globals['_OWNEDTABLEREFERENCE']._serialized_end=17030 - _globals['_PHYSICALPLANNODE']._serialized_start=17033 - _globals['_PHYSICALPLANNODE']._serialized_end=18195 - _globals['_PHYSICALEXTENSIONNODE']._serialized_start=18197 - _globals['_PHYSICALEXTENSIONNODE']._serialized_end=18280 - _globals['_PHYSICALEXPRNODE']._serialized_start=18283 - _globals['_PHYSICALEXPRNODE']._serialized_end=19382 - _globals['_PHYSICALSCALARUDFNODE']._serialized_start=19384 - _globals['_PHYSICALSCALARUDFNODE']._serialized_end=19509 - _globals['_PHYSICALAGGREGATEEXPRNODE']._serialized_start=19512 - _globals['_PHYSICALAGGREGATEEXPRNODE']._serialized_end=19772 - _globals['_PHYSICALWINDOWEXPRNODE']._serialized_start=19775 - _globals['_PHYSICALWINDOWEXPRNODE']._serialized_end=19982 - _globals['_PHYSICALISNULL']._serialized_start=19984 - _globals['_PHYSICALISNULL']._serialized_end=20044 - _globals['_PHYSICALISNOTNULL']._serialized_start=20046 - _globals['_PHYSICALISNOTNULL']._serialized_end=20109 - _globals['_PHYSICALNOT']._serialized_start=20111 - _globals['_PHYSICALNOT']._serialized_end=20168 - _globals['_PHYSICALALIASNODE']._serialized_start=20170 - _globals['_PHYSICALALIASNODE']._serialized_end=20248 - _globals['_PHYSICALBINARYEXPRNODE']._serialized_start=20250 - _globals['_PHYSICALBINARYEXPRNODE']._serialized_end=20368 - _globals['_PHYSICALDATETIMEINTERVALEXPRNODE']._serialized_start=20371 - _globals['_PHYSICALDATETIMEINTERVALEXPRNODE']._serialized_end=20499 - _globals['_PHYSICALLIKEEXPRNODE']._serialized_start=20502 - _globals['_PHYSICALLIKEEXPRNODE']._serialized_end=20658 - _globals['_PHYSICALSORTEXPRNODE']._serialized_start=20660 - _globals['_PHYSICALSORTEXPRNODE']._serialized_end=20760 - _globals['_PHYSICALWHENTHEN']._serialized_start=20762 - _globals['_PHYSICALWHENTHEN']._serialized_end=20878 - _globals['_PHYSICALINLISTNODE']._serialized_start=20880 - _globals['_PHYSICALINLISTNODE']._serialized_end=21005 - _globals['_PHYSICALCASENODE']._serialized_start=21008 - _globals['_PHYSICALCASENODE']._serialized_end=21173 - _globals['_PHYSICALSCALARFUNCTIONNODE']._serialized_start=21176 - _globals['_PHYSICALSCALARFUNCTIONNODE']._serialized_end=21347 - _globals['_PHYSICALTRYCASTNODE']._serialized_start=21349 - _globals['_PHYSICALTRYCASTNODE']._serialized_end=21457 - _globals['_PHYSICALCASTNODE']._serialized_start=21459 - _globals['_PHYSICALCASTNODE']._serialized_end=21564 - _globals['_PHYSICALNEGATIVENODE']._serialized_start=21566 - _globals['_PHYSICALNEGATIVENODE']._serialized_end=21632 - _globals['_FILTEREXECNODE']._serialized_start=21634 - _globals['_FILTEREXECNODE']._serialized_end=21739 - _globals['_FILEGROUP']._serialized_start=21741 - _globals['_FILEGROUP']._serialized_end=21796 - _globals['_SCANLIMIT']._serialized_start=21798 - _globals['_SCANLIMIT']._serialized_end=21824 - _globals['_PHYSICALSORTEXPRNODECOLLECTION']._serialized_start=21826 - _globals['_PHYSICALSORTEXPRNODECOLLECTION']._serialized_end=21926 - _globals['_FILESCANEXECCONF']._serialized_start=21929 - _globals['_FILESCANEXECCONF']._serialized_end=22260 - _globals['_PARQUETSCANEXECNODE']._serialized_start=22262 - _globals['_PARQUETSCANEXECNODE']._serialized_end=22387 - _globals['_CSVSCANEXECNODE']._serialized_start=22389 - _globals['_CSVSCANEXECNODE']._serialized_end=22494 - _globals['_AVROSCANEXECNODE']._serialized_start=22496 - _globals['_AVROSCANEXECNODE']._serialized_end=22563 - _globals['_HASHJOINEXECNODE']._serialized_start=22566 - _globals['_HASHJOINEXECNODE']._serialized_end=22863 - _globals['_UNIONEXECNODE']._serialized_start=22865 - _globals['_UNIONEXECNODE']._serialized_end=22926 - _globals['_EXPLAINEXECNODE']._serialized_start=22928 - _globals['_EXPLAINEXECNODE']._serialized_end=23054 - _globals['_CROSSJOINEXECNODE']._serialized_start=23056 - _globals['_CROSSJOINEXECNODE']._serialized_end=23164 - _globals['_PHYSICALCOLUMN']._serialized_start=23166 - _globals['_PHYSICALCOLUMN']._serialized_end=23211 - _globals['_JOINON']._serialized_start=23213 - _globals['_JOINON']._serialized_end=23306 - _globals['_EMPTYEXECNODE']._serialized_start=23308 - _globals['_EMPTYEXECNODE']._serialized_end=23384 - _globals['_PROJECTIONEXECNODE']._serialized_start=23387 - _globals['_PROJECTIONEXECNODE']._serialized_end=23515 - _globals['_WINDOWAGGEXECNODE']._serialized_start=23518 - _globals['_WINDOWAGGEXECNODE']._serialized_end=23701 - _globals['_MAYBEFILTER']._serialized_start=23703 - _globals['_MAYBEFILTER']._serialized_end=23760 - _globals['_MAYBEPHYSICALSORTEXPRS']._serialized_start=23762 - _globals['_MAYBEPHYSICALSORTEXPRS']._serialized_end=23839 - _globals['_AGGREGATEEXECNODE']._serialized_start=23842 - _globals['_AGGREGATEEXECNODE']._serialized_end=24307 - _globals['_GLOBALLIMITEXECNODE']._serialized_start=24309 - _globals['_GLOBALLIMITEXECNODE']._serialized_end=24404 - _globals['_LOCALLIMITEXECNODE']._serialized_start=24406 - _globals['_LOCALLIMITEXECNODE']._serialized_end=24486 - _globals['_SORTEXECNODE']._serialized_start=24489 - _globals['_SORTEXECNODE']._serialized_end=24638 - _globals['_SORTPRESERVINGMERGEEXECNODE']._serialized_start=24641 - _globals['_SORTPRESERVINGMERGEEXECNODE']._serialized_end=24774 - _globals['_NESTEDLOOPJOINEXECNODE']._serialized_start=24777 - _globals['_NESTEDLOOPJOINEXECNODE']._serialized_end=24971 - _globals['_COALESCEBATCHESEXECNODE']._serialized_start=24973 - _globals['_COALESCEBATCHESEXECNODE']._serialized_end=25070 - _globals['_COALESCEPARTITIONSEXECNODE']._serialized_start=25072 - _globals['_COALESCEPARTITIONSEXECNODE']._serialized_end=25145 - _globals['_PHYSICALHASHREPARTITION']._serialized_start=25147 - _globals['_PHYSICALHASHREPARTITION']._serialized_end=25246 - _globals['_REPARTITIONEXECNODE']._serialized_start=25249 - _globals['_REPARTITIONEXECNODE']._serialized_end=25430 - _globals['_JOINFILTER']._serialized_start=25433 - _globals['_JOINFILTER']._serialized_end=25580 - _globals['_COLUMNINDEX']._serialized_start=25582 - _globals['_COLUMNINDEX']._serialized_end=25646 - _globals['_PARTITIONEDFILE']._serialized_start=25649 - _globals['_PARTITIONEDFILE']._serialized_end=25809 - _globals['_FILERANGE']._serialized_start=25811 - _globals['_FILERANGE']._serialized_end=25850 - _globals['_PARTITIONSTATS']._serialized_start=25852 - _globals['_PARTITIONSTATS']._serialized_end=25973 - _globals['_STATISTICS']._serialized_start=25975 - _globals['_STATISTICS']._serialized_end=26095 - _globals['_COLUMNSTATS']._serialized_start=26098 - _globals['_COLUMNSTATS']._serialized_end=26243 - _globals['_PHYSICALGETINDEXEDFIELDEXPRNODE']._serialized_start=26245 - _globals['_PHYSICALGETINDEXEDFIELDEXPRNODE']._serialized_end=26359 + _globals['_FIXEDSIZEBINARY']._serialized_start=13569 + _globals['_FIXEDSIZEBINARY']._serialized_end=13602 + _globals['_TIMESTAMP']._serialized_start=13604 + _globals['_TIMESTAMP']._serialized_end=13674 + _globals['_DECIMAL']._serialized_start=13676 + _globals['_DECIMAL']._serialized_end=13731 + _globals['_LIST']._serialized_start=13733 + _globals['_LIST']._serialized_end=13778 + _globals['_FIXEDSIZELIST']._serialized_start=13780 + _globals['_FIXEDSIZELIST']._serialized_end=13853 + _globals['_DICTIONARY']._serialized_start=13855 + _globals['_DICTIONARY']._serialized_end=13941 + _globals['_STRUCT']._serialized_start=13943 + _globals['_STRUCT']._serialized_end=13995 + _globals['_MAP']._serialized_start=13997 + _globals['_MAP']._serialized_end=14062 + _globals['_UNION']._serialized_start=14064 + _globals['_UNION']._serialized_end=14172 + _globals['_SCALARNESTEDVALUE']._serialized_start=14174 + _globals['_SCALARNESTEDVALUE']._serialized_end=14270 + _globals['_SCALARTIME32VALUE']._serialized_start=14272 + _globals['_SCALARTIME32VALUE']._serialized_end=14367 + _globals['_SCALARTIME64VALUE']._serialized_start=14369 + _globals['_SCALARTIME64VALUE']._serialized_end=14468 + _globals['_SCALARTIMESTAMPVALUE']._serialized_start=14471 + _globals['_SCALARTIMESTAMPVALUE']._serialized_end=14650 + _globals['_SCALARDICTIONARYVALUE']._serialized_start=14652 + _globals['_SCALARDICTIONARYVALUE']._serialized_end=14758 + _globals['_INTERVALMONTHDAYNANOVALUE']._serialized_start=14760 + _globals['_INTERVALMONTHDAYNANOVALUE']._serialized_end=14832 + _globals['_SCALARFIXEDSIZEBINARY']._serialized_start=14834 + _globals['_SCALARFIXEDSIZEBINARY']._serialized_end=14889 + _globals['_SCALARVALUE']._serialized_start=14892 + _globals['_SCALARVALUE']._serialized_end=16306 + _globals['_DECIMAL128']._serialized_start=16308 + _globals['_DECIMAL128']._serialized_end=16357 + _globals['_DECIMAL256']._serialized_start=16359 + _globals['_DECIMAL256']._serialized_end=16408 + _globals['_ARROWTYPE']._serialized_start=16411 + _globals['_ARROWTYPE']._serialized_end=17839 + _globals['_EMPTYMESSAGE']._serialized_start=17841 + _globals['_EMPTYMESSAGE']._serialized_end=17855 + _globals['_ANALYZEDLOGICALPLANTYPE']._serialized_start=17857 + _globals['_ANALYZEDLOGICALPLANTYPE']._serialized_end=17905 + _globals['_OPTIMIZEDLOGICALPLANTYPE']._serialized_start=17907 + _globals['_OPTIMIZEDLOGICALPLANTYPE']._serialized_end=17957 + _globals['_OPTIMIZEDPHYSICALPLANTYPE']._serialized_start=17959 + _globals['_OPTIMIZEDPHYSICALPLANTYPE']._serialized_end=18010 + _globals['_PLANTYPE']._serialized_start=18013 + _globals['_PLANTYPE']._serialized_end=18665 + _globals['_STRINGIFIEDPLAN']._serialized_start=18667 + _globals['_STRINGIFIEDPLAN']._serialized_end=18739 + _globals['_BARETABLEREFERENCE']._serialized_start=18741 + _globals['_BARETABLEREFERENCE']._serialized_end=18776 + _globals['_PARTIALTABLEREFERENCE']._serialized_start=18778 + _globals['_PARTIALTABLEREFERENCE']._serialized_end=18832 + _globals['_FULLTABLEREFERENCE']._serialized_start=18834 + _globals['_FULLTABLEREFERENCE']._serialized_end=18902 + _globals['_OWNEDTABLEREFERENCE']._serialized_start=18905 + _globals['_OWNEDTABLEREFERENCE']._serialized_end=19100 + _globals['_PHYSICALPLANNODE']._serialized_start=19103 + _globals['_PHYSICALPLANNODE']._serialized_end=20657 + _globals['_PARTITIONCOLUMN']._serialized_start=20659 + _globals['_PARTITIONCOLUMN']._serialized_end=20733 + _globals['_FILETYPEWRITEROPTIONS']._serialized_start=20736 + _globals['_FILETYPEWRITEROPTIONS']._serialized_end=20997 + _globals['_JSONWRITEROPTIONS']._serialized_start=20999 + _globals['_JSONWRITEROPTIONS']._serialized_end=21075 + _globals['_PARQUETWRITEROPTIONS']._serialized_start=21077 + _globals['_PARQUETWRITEROPTIONS']._serialized_end=21156 + _globals['_CSVWRITEROPTIONS']._serialized_start=21159 + _globals['_CSVWRITEROPTIONS']._serialized_end=21386 + _globals['_ARROWWRITEROPTIONS']._serialized_start=21388 + _globals['_ARROWWRITEROPTIONS']._serialized_end=21408 + _globals['_WRITERPROPERTIES']._serialized_start=21411 + _globals['_WRITERPROPERTIES']._serialized_end=21628 + _globals['_FILESINKCONFIG']._serialized_start=21631 + _globals['_FILESINKCONFIG']._serialized_end=21940 + _globals['_JSONSINK']._serialized_start=21942 + _globals['_JSONSINK']._serialized_end=21996 + _globals['_JSONSINKEXECNODE']._serialized_start=21999 + _globals['_JSONSINKEXECNODE']._serialized_end=22203 + _globals['_CSVSINK']._serialized_start=22205 + _globals['_CSVSINK']._serialized_end=22258 + _globals['_CSVSINKEXECNODE']._serialized_start=22261 + _globals['_CSVSINKEXECNODE']._serialized_end=22463 + _globals['_PARQUETSINK']._serialized_start=22465 + _globals['_PARQUETSINK']._serialized_end=22522 + _globals['_PARQUETSINKEXECNODE']._serialized_start=22525 + _globals['_PARQUETSINKEXECNODE']._serialized_end=22735 + _globals['_PHYSICALEXTENSIONNODE']._serialized_start=22737 + _globals['_PHYSICALEXTENSIONNODE']._serialized_end=22820 + _globals['_PHYSICALEXPRNODE']._serialized_start=22823 + _globals['_PHYSICALEXPRNODE']._serialized_end=23847 + _globals['_PHYSICALSCALARUDFNODE']._serialized_start=23849 + _globals['_PHYSICALSCALARUDFNODE']._serialized_end=23974 + _globals['_PHYSICALAGGREGATEEXPRNODE']._serialized_start=23977 + _globals['_PHYSICALAGGREGATEEXPRNODE']._serialized_end=24237 + _globals['_PHYSICALWINDOWEXPRNODE']._serialized_start=24240 + _globals['_PHYSICALWINDOWEXPRNODE']._serialized_end=24612 + _globals['_PHYSICALISNULL']._serialized_start=24614 + _globals['_PHYSICALISNULL']._serialized_end=24674 + _globals['_PHYSICALISNOTNULL']._serialized_start=24676 + _globals['_PHYSICALISNOTNULL']._serialized_end=24739 + _globals['_PHYSICALNOT']._serialized_start=24741 + _globals['_PHYSICALNOT']._serialized_end=24798 + _globals['_PHYSICALALIASNODE']._serialized_start=24800 + _globals['_PHYSICALALIASNODE']._serialized_end=24878 + _globals['_PHYSICALBINARYEXPRNODE']._serialized_start=24880 + _globals['_PHYSICALBINARYEXPRNODE']._serialized_end=24998 + _globals['_PHYSICALDATETIMEINTERVALEXPRNODE']._serialized_start=25001 + _globals['_PHYSICALDATETIMEINTERVALEXPRNODE']._serialized_end=25129 + _globals['_PHYSICALLIKEEXPRNODE']._serialized_start=25132 + _globals['_PHYSICALLIKEEXPRNODE']._serialized_end=25288 + _globals['_PHYSICALSORTEXPRNODE']._serialized_start=25290 + _globals['_PHYSICALSORTEXPRNODE']._serialized_end=25390 + _globals['_PHYSICALWHENTHEN']._serialized_start=25392 + _globals['_PHYSICALWHENTHEN']._serialized_end=25508 + _globals['_PHYSICALINLISTNODE']._serialized_start=25510 + _globals['_PHYSICALINLISTNODE']._serialized_end=25635 + _globals['_PHYSICALCASENODE']._serialized_start=25638 + _globals['_PHYSICALCASENODE']._serialized_end=25803 + _globals['_PHYSICALSCALARFUNCTIONNODE']._serialized_start=25806 + _globals['_PHYSICALSCALARFUNCTIONNODE']._serialized_end=25977 + _globals['_PHYSICALTRYCASTNODE']._serialized_start=25979 + _globals['_PHYSICALTRYCASTNODE']._serialized_end=26087 + _globals['_PHYSICALCASTNODE']._serialized_start=26089 + _globals['_PHYSICALCASTNODE']._serialized_end=26194 + _globals['_PHYSICALNEGATIVENODE']._serialized_start=26196 + _globals['_PHYSICALNEGATIVENODE']._serialized_end=26262 + _globals['_FILTEREXECNODE']._serialized_start=26265 + _globals['_FILTEREXECNODE']._serialized_end=26406 + _globals['_FILEGROUP']._serialized_start=26408 + _globals['_FILEGROUP']._serialized_end=26463 + _globals['_SCANLIMIT']._serialized_start=26465 + _globals['_SCANLIMIT']._serialized_end=26491 + _globals['_PHYSICALSORTEXPRNODECOLLECTION']._serialized_start=26493 + _globals['_PHYSICALSORTEXPRNODECOLLECTION']._serialized_end=26593 + _globals['_FILESCANEXECCONF']._serialized_start=26596 + _globals['_FILESCANEXECCONF']._serialized_end=26927 + _globals['_PARQUETSCANEXECNODE']._serialized_start=26929 + _globals['_PARQUETSCANEXECNODE']._serialized_end=27054 + _globals['_CSVSCANEXECNODE']._serialized_start=27057 + _globals['_CSVSCANEXECNODE']._serialized_end=27214 + _globals['_AVROSCANEXECNODE']._serialized_start=27216 + _globals['_AVROSCANEXECNODE']._serialized_end=27283 + _globals['_HASHJOINEXECNODE']._serialized_start=27286 + _globals['_HASHJOINEXECNODE']._serialized_end=27583 + _globals['_SYMMETRICHASHJOINEXECNODE']._serialized_start=27586 + _globals['_SYMMETRICHASHJOINEXECNODE']._serialized_end=28017 + _globals['_INTERLEAVEEXECNODE']._serialized_start=28019 + _globals['_INTERLEAVEEXECNODE']._serialized_end=28085 + _globals['_UNIONEXECNODE']._serialized_start=28087 + _globals['_UNIONEXECNODE']._serialized_end=28148 + _globals['_EXPLAINEXECNODE']._serialized_start=28150 + _globals['_EXPLAINEXECNODE']._serialized_end=28276 + _globals['_ANALYZEEXECNODE']._serialized_start=28279 + _globals['_ANALYZEEXECNODE']._serialized_end=28419 + _globals['_CROSSJOINEXECNODE']._serialized_start=28421 + _globals['_CROSSJOINEXECNODE']._serialized_end=28529 + _globals['_PHYSICALCOLUMN']._serialized_start=28531 + _globals['_PHYSICALCOLUMN']._serialized_end=28576 + _globals['_JOINON']._serialized_start=28578 + _globals['_JOINON']._serialized_end=28675 + _globals['_EMPTYEXECNODE']._serialized_start=28677 + _globals['_EMPTYEXECNODE']._serialized_end=28728 + _globals['_PLACEHOLDERROWEXECNODE']._serialized_start=28730 + _globals['_PLACEHOLDERROWEXECNODE']._serialized_end=28790 + _globals['_PROJECTIONEXECNODE']._serialized_start=28793 + _globals['_PROJECTIONEXECNODE']._serialized_end=28921 + _globals['_PARTIALLYSORTEDINPUTORDERMODE']._serialized_start=28923 + _globals['_PARTIALLYSORTEDINPUTORDERMODE']._serialized_end=28971 + _globals['_WINDOWAGGEXECNODE']._serialized_start=28974 + _globals['_WINDOWAGGEXECNODE']._serialized_end=29328 + _globals['_MAYBEFILTER']._serialized_start=29330 + _globals['_MAYBEFILTER']._serialized_end=29387 + _globals['_MAYBEPHYSICALSORTEXPRS']._serialized_start=29389 + _globals['_MAYBEPHYSICALSORTEXPRS']._serialized_end=29466 + _globals['_AGGREGATEEXECNODE']._serialized_start=29469 + _globals['_AGGREGATEEXECNODE']._serialized_end=29875 + _globals['_GLOBALLIMITEXECNODE']._serialized_start=29877 + _globals['_GLOBALLIMITEXECNODE']._serialized_end=29972 + _globals['_LOCALLIMITEXECNODE']._serialized_start=29974 + _globals['_LOCALLIMITEXECNODE']._serialized_end=30054 + _globals['_SORTEXECNODE']._serialized_start=30057 + _globals['_SORTEXECNODE']._serialized_end=30206 + _globals['_SORTPRESERVINGMERGEEXECNODE']._serialized_start=30209 + _globals['_SORTPRESERVINGMERGEEXECNODE']._serialized_end=30342 + _globals['_NESTEDLOOPJOINEXECNODE']._serialized_start=30345 + _globals['_NESTEDLOOPJOINEXECNODE']._serialized_end=30539 + _globals['_COALESCEBATCHESEXECNODE']._serialized_start=30541 + _globals['_COALESCEBATCHESEXECNODE']._serialized_end=30638 + _globals['_COALESCEPARTITIONSEXECNODE']._serialized_start=30640 + _globals['_COALESCEPARTITIONSEXECNODE']._serialized_end=30713 + _globals['_PHYSICALHASHREPARTITION']._serialized_start=30715 + _globals['_PHYSICALHASHREPARTITION']._serialized_end=30814 + _globals['_REPARTITIONEXECNODE']._serialized_start=30817 + _globals['_REPARTITIONEXECNODE']._serialized_end=30998 + _globals['_JOINFILTER']._serialized_start=31001 + _globals['_JOINFILTER']._serialized_end=31148 + _globals['_COLUMNINDEX']._serialized_start=31150 + _globals['_COLUMNINDEX']._serialized_end=31214 + _globals['_PARTITIONEDFILE']._serialized_start=31217 + _globals['_PARTITIONEDFILE']._serialized_end=31377 + _globals['_FILERANGE']._serialized_start=31379 + _globals['_FILERANGE']._serialized_end=31418 + _globals['_PARTITIONSTATS']._serialized_start=31420 + _globals['_PARTITIONSTATS']._serialized_end=31541 + _globals['_PRECISION']._serialized_start=31543 + _globals['_PRECISION']._serialized_end=31643 + _globals['_STATISTICS']._serialized_start=31646 + _globals['_STATISTICS']._serialized_end=31794 + _globals['_COLUMNSTATS']._serialized_start=31797 + _globals['_COLUMNSTATS']._serialized_end=31984 + _globals['_NAMEDSTRUCTFIELDEXPR']._serialized_start=31986 + _globals['_NAMEDSTRUCTFIELDEXPR']._serialized_end=32047 + _globals['_LISTINDEXEXPR']._serialized_start=32049 + _globals['_LISTINDEXEXPR']._serialized_end=32107 + _globals['_LISTRANGEEXPR']._serialized_start=32110 + _globals['_LISTRANGEEXPR']._serialized_end=32260 + _globals['_PHYSICALGETINDEXEDFIELDEXPRNODE']._serialized_start=32263 + _globals['_PHYSICALGETINDEXEDFIELDEXPRNODE']._serialized_end=32525 # @@protoc_insertion_point(module_scope)