diff --git a/duckdb b/duckdb index 197b79b..fb0385b 160000 --- a/duckdb +++ b/duckdb @@ -1 +1 @@ -Subproject commit 197b79bfafe6519b0704209c2a3b2eea6b632baf +Subproject commit fb0385b4ddf6790eba2f74a0b677ecc876dd574b diff --git a/duckdb-r b/duckdb-r index 3e6b47d..f1deae2 160000 --- a/duckdb-r +++ b/duckdb-r @@ -1 +1 @@ -Subproject commit 3e6b47da6c96128616e446833ff6991ed22aa810 +Subproject commit f1deae29ebb6f5fb716c92d14ebc37a0ce8f3437 diff --git a/scripts/build_local.py b/scripts/build_local.py new file mode 100644 index 0000000..fe671cb --- /dev/null +++ b/scripts/build_local.py @@ -0,0 +1,37 @@ +# Requires protoc 3.19.04 +# https://github.com/protocolbuffers/protobuf/releases/tag/v3.19.4 + +import os +import shutil +from os import walk + +# Change to substrait folder +sub_folder = os.path.join(os.path.dirname(os.path.realpath(__file__)),'..','third_party','substrait') +# Delete Current CPP files +shutil.rmtree(os.path.join(sub_folder,'substrait')) + +git_folder = os.path.join(os.path.dirname(os.path.realpath(__file__)),'..','substrait') + +proto_folder = os.path.join(git_folder,'proto') +substrait_proto_folder = os.path.join(proto_folder,'substrait') +substrait_extensions_proto_folder = os.path.join(substrait_proto_folder,'extensions') + +# Generate Proto Files on a specific git tag +os.chdir(sub_folder) + +os.mkdir("substrait") +os.mkdir("substrait/extensions") + +# Generate all files +proto_sub_list = next(walk(substrait_proto_folder), (None, None, []))[2] + +proto_sub_extensions = next(walk(substrait_extensions_proto_folder), (None, None, []))[2] + +# /usr/local/bin/protoc +print("Protoc version" + os.popen('protoc --version').read()) + +for proto in proto_sub_list: + os.system("/usr/local/bin/protoc -I="+ proto_folder+ " --cpp_out="+sub_folder +" "+ os.path.join(substrait_proto_folder,proto)) + +for proto in proto_sub_extensions: + os.system("/usr/local/bin/protoc -I="+ proto_folder+ " --cpp_out="+sub_folder +" "+ os.path.join(substrait_extensions_proto_folder,proto)) \ No newline at end of file diff --git a/src/from_substrait.cpp b/src/from_substrait.cpp index 40f78d4..1cf5635 100644 --- a/src/from_substrait.cpp +++ b/src/from_substrait.cpp @@ -20,11 +20,11 @@ #include "duckdb/common/enums/set_operation_type.hpp" #include "duckdb/parser/expression/comparison_expression.hpp" - -#include "substrait/plan.pb.h" +#include "duckdb/main/client_data.hpp" #include "google/protobuf/util/json_util.h" +#include "substrait/plan.pb.h" +#include "duckdb/main/relation/delim_get_relation.hpp" #include "duckdb/main/client_data.hpp" -#include "duckdb/common/http_state.hpp" namespace duckdb { const std::unordered_map SubstraitToDuckDB::function_names_remap = { @@ -66,9 +66,6 @@ std::string SubstraitToDuckDB::RemoveExtension(std::string &function_name) { } SubstraitToDuckDB::SubstraitToDuckDB(Connection &con_p, const string &serialized, bool json) : con(con_p) { - auto http_state = HTTPState::TryGetState(*con_p.context); - http_state->Reset(); - if (!json) { if (!plan.ParseFromString(serialized)) { throw std::runtime_error("Was not possible to convert binary into Substrait plan"); @@ -413,15 +410,18 @@ shared_ptr SubstraitToDuckDB::TransformJoinOp(const substrait::Rel &so case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_RIGHT: djointype = JoinType::RIGHT; break; - case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_SINGLE: + case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE: djointype = JoinType::SINGLE; break; - case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_SEMI: + case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI: djointype = JoinType::SEMI; break; - case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_MARK: + case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_MARK: djointype = JoinType::MARK; break; + case substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI: + djointype = JoinType::RIGHT_SEMI; + break; default: throw InternalException("Unsupported join type"); } @@ -431,6 +431,73 @@ shared_ptr SubstraitToDuckDB::TransformJoinOp(const substrait::Rel &so return make_shared_ptr(std::move(left_op), std::move(right_op), std::move(join_condition), djointype); } +shared_ptr SubstraitToDuckDB::TransformDelimJoinOp(const substrait::Rel &sop) { + auto &sjoin = sop.duplicate_eliminated_join(); + + vector> duplicate_eliminated_columns; + for (auto &col : sjoin.duplicate_eliminated_columns()) { + duplicate_eliminated_columns.emplace_back( + make_uniq(col.direct_reference().struct_field().field() + 1)); + } + duplicate_eliminated_columns_ptr = &duplicate_eliminated_columns; + JoinType djointype; + switch (sjoin.type()) { + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_INNER: + djointype = JoinType::INNER; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT: + djointype = JoinType::LEFT; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT: + djointype = JoinType::RIGHT; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE: + djointype = JoinType::SINGLE; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI: + djointype = JoinType::RIGHT_SEMI; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_MARK: + djointype = JoinType::MARK; + break; + case substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI: + djointype = JoinType::RIGHT_ANTI; + break; + default: + throw InternalException("Unsupported join type"); + } + unique_ptr join_condition = TransformExpr(sjoin.expression()); + auto left_op = TransformOp(sjoin.left())->Alias("left"); + auto right_op = TransformOp(sjoin.right())->Alias("right"); + auto join = + make_shared_ptr(std::move(left_op), std::move(right_op), std::move(join_condition), djointype); + if (sjoin.duplicate_eliminated_side() == substrait::DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_RIGHT) { + join->delim_flipped = true; + } else if (sjoin.duplicate_eliminated_side() == + substrait::DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_LEFT) { + join->delim_flipped = false; + } else { + throw InvalidInputException("The plan has a delimiter join with an invalid type for it's delimiter side."); + } + join->duplicate_eliminated_columns = std::move(duplicate_eliminated_columns); + return join; +} + +shared_ptr SubstraitToDuckDB::TransformDelimGetOp(const substrait::Rel &sop) { + auto &delim_get = sop.duplicate_eliminated_get(); + auto subtree = TransformReferenceOp(delim_get.input()); + + auto &client_context = con.context; + vector chunk_types; + auto &input_columns = subtree->Columns(); + for (auto &col : *duplicate_eliminated_columns_ptr) { + auto& col_ref = col->Cast(); + chunk_types.emplace_back(input_columns[col_ref.index - 1].Type()); + } + duplicate_eliminated_columns_ptr = nullptr; + return make_shared_ptr(client_context, chunk_types); +} + shared_ptr SubstraitToDuckDB::TransformCrossProductOp(const substrait::Rel &sop) { auto &sub_cross = sop.cross(); @@ -594,6 +661,12 @@ shared_ptr SubstraitToDuckDB::TransformSetOp(const substrait::Rel &sop return make_shared_ptr(std::move(lhs), std::move(rhs), type); } +shared_ptr SubstraitToDuckDB::TransformReferenceOp(const substrait::ReferenceRel &reference) { + // Lets get the subtree + auto subtree = plan.relations(reference.subtree_ordinal()); + return TransformOp(subtree.rel()); +} + shared_ptr SubstraitToDuckDB::TransformOp(const substrait::Rel &sop) { switch (sop.rel_type_case()) { case substrait::Rel::RelTypeCase::kJoin: @@ -614,6 +687,12 @@ shared_ptr SubstraitToDuckDB::TransformOp(const substrait::Rel &sop) { return TransformSortOp(sop); case substrait::Rel::RelTypeCase::kSet: return TransformSetOp(sop); + case substrait::Rel::RelTypeCase::kDuplicateEliminatedJoin: + return TransformDelimJoinOp(sop); + case substrait::Rel::RelTypeCase::kDuplicateEliminatedGet: + return TransformDelimGetOp(sop); + case substrait::Rel::RelTypeCase::kReference: + return TransformReferenceOp(sop.reference()); default: throw InternalException("Unsupported relation type " + to_string(sop.rel_type_case())); } @@ -640,13 +719,12 @@ shared_ptr SubstraitToDuckDB::TransformPlan() { d_plan = TransformRootOp(plan.relations(0).root()); } catch (std::exception &ex) { // Ideally we don't have to do that, we should change to capture the error and throw it here at some point - ::duckdb::ErrorData parsed_error(ex); + ErrorData parsed_error(ex); throw NotImplementedException("Something in this plan is not yet implemented in the Substrait->DuckDB plan " "conversion.\n Substrait Plan:\n" + plan.DebugString() + "Not Implemented error message: " + parsed_error.RawMessage()); } - return d_plan; } diff --git a/src/include/from_substrait.hpp b/src/include/from_substrait.hpp index 3a632ce..96280ea 100644 --- a/src/include/from_substrait.hpp +++ b/src/include/from_substrait.hpp @@ -20,6 +20,8 @@ class SubstraitToDuckDB { //! Transform Substrait Operations to DuckDB Relations shared_ptr TransformOp(const substrait::Rel &sop); shared_ptr TransformJoinOp(const substrait::Rel &sop); + shared_ptr TransformDelimJoinOp(const substrait::Rel &sop); + shared_ptr TransformDelimGetOp(const substrait::Rel &sop); shared_ptr TransformCrossProductOp(const substrait::Rel &sop); shared_ptr TransformFetchOp(const substrait::Rel &sop); shared_ptr TransformFilterOp(const substrait::Rel &sop); @@ -28,6 +30,7 @@ class SubstraitToDuckDB { shared_ptr TransformReadOp(const substrait::Rel &sop); shared_ptr TransformSortOp(const substrait::Rel &sop); shared_ptr TransformSetOp(const substrait::Rel &sop); + shared_ptr TransformReferenceOp(const substrait::ReferenceRel &reference); //! Transform Substrait Expressions to DuckDB Expressions unique_ptr TransformExpr(const substrait::Expression &sexpr); @@ -57,5 +60,7 @@ class SubstraitToDuckDB { //! names static const unordered_map function_names_remap; static const case_insensitive_set_t valid_extract_subfields; + //! Pointer to last seen duplicate_eliminated_columns + vector> *duplicate_eliminated_columns_ptr = nullptr; }; } // namespace duckdb diff --git a/src/include/to_substrait.hpp b/src/include/to_substrait.hpp index ddd4049..2f19126 100644 --- a/src/include/to_substrait.hpp +++ b/src/include/to_substrait.hpp @@ -37,7 +37,7 @@ class DuckDBToSubstrait { void CreateFieldRef(substrait::Expression *expr, uint64_t col_idx); //! Transforms Relation Root - substrait::RelRoot *TransformRootOp(LogicalOperator &dop); + void TransformRootOp(substrait::RelRoot &root_rel, LogicalOperator &dop); //! Methods to Transform Logical Operators to Substrait Relations substrait::Rel *TransformOp(duckdb::LogicalOperator &dop); @@ -47,6 +47,7 @@ class DuckDBToSubstrait { substrait::Rel *TransformLimit(duckdb::LogicalOperator &dop); substrait::Rel *TransformOrderBy(duckdb::LogicalOperator &dop); substrait::Rel *TransformComparisonJoin(duckdb::LogicalOperator &dop); + substrait::Rel *TransformDelimiterJoin(duckdb::LogicalOperator &dop); substrait::Rel *TransformAggregateGroup(duckdb::LogicalOperator &dop); substrait::Rel *TransformGet(duckdb::LogicalOperator &dop); substrait::Rel *TransformCrossProduct(duckdb::LogicalOperator &dop); @@ -54,6 +55,11 @@ class DuckDBToSubstrait { substrait::Rel *TransformDistinct(duckdb::LogicalOperator &dop); substrait::Rel *TransformExcept(LogicalOperator &dop); substrait::Rel *TransformIntersect(LogicalOperator &dop); + substrait::Rel *TransformDelimGet(); + + //! Auxiliary function to create Projection on top of a Join + substrait::Rel *ProjectJoinRelation(LogicalComparisonJoin &djoin, substrait::Rel *join_relation, + idx_t left_col_count); //! Methods to transform different LogicalGet Types (e.g., Table, Parquet) //! To Substrait; @@ -160,5 +166,11 @@ class DuckDBToSubstrait { //! things don't go perfectly shiny bool strict; string errors; + //! Index of the current subtree relation we are looking at + //! This really only matters for delim joins/gets, since these are + //! the only splits we currently support. + int32_t cur_subtree_relation = 1; + //! The pointer to a delim join + LogicalComparisonJoin *duplicate_eliminated_parent_ptr = nullptr; }; } // namespace duckdb diff --git a/src/substrait_extension.cpp b/src/substrait_extension.cpp index 327df86..671dbde 100644 --- a/src/substrait_extension.cpp +++ b/src/substrait_extension.cpp @@ -89,26 +89,28 @@ shared_ptr SubstraitPlanToDuckDBRel(Connection &conn, const string &se static void VerifySubstraitRoundtrip(unique_ptr &query_plan, Connection &con, ToSubstraitFunctionData &data, const string &serialized, bool is_json) { - // We round-trip the generated json and verify if the result is the same - auto actual_result = con.Query(data.query); - + bool is_optimizer_enabled = con.context->config.enable_optimizer; + con.context->config.enable_optimizer = false; auto sub_relation = SubstraitPlanToDuckDBRel(con, serialized, is_json); unique_ptr substrait_result; + try { substrait_result = sub_relation->Execute(); - } catch (std::exception &ex) { // Ideally we don't have to do that, we should change to capture the error and throw it here at some point query_plan->Print(); sub_relation->Print(); - throw InternalException("Substrait Plan Execution Failed"); } + con.context->config.enable_optimizer = is_optimizer_enabled; + // We round-trip the generated json and verify if the result is the same + auto actual_result = con.Query(data.query); + substrait_result->names = actual_result->names; unique_ptr substrait_materialized; if (substrait_result->type == QueryResultType::STREAM_RESULT) { - auto &stream_query = substrait_result->Cast(); + auto &stream_query = substrait_result->Cast(); substrait_materialized = stream_query.Materialize(); } else if (substrait_result->type == QueryResultType::MATERIALIZED_RESULT) { @@ -152,6 +154,7 @@ static DuckDBToSubstrait InitPlanExtractor(ClientContext &context, ToSubstraitFu set disabled_optimizers = DBConfig::GetConfig(context).options.disabled_optimizers; disabled_optimizers.insert(OptimizerType::IN_CLAUSE); disabled_optimizers.insert(OptimizerType::COMPRESSED_MATERIALIZATION); + disabled_optimizers.insert(OptimizerType::MATERIALIZED_CTE); DBConfig::GetConfig(*new_conn.context).options.disabled_optimizers = disabled_optimizers; query_plan = new_conn.context->ExtractPlan(data.query); diff --git a/src/to_substrait.cpp b/src/to_substrait.cpp index dcd1f3d..beebe5d 100644 --- a/src/to_substrait.cpp +++ b/src/to_substrait.cpp @@ -689,6 +689,9 @@ substrait::Expression *DuckDBToSubstrait::TransformJoinCond(JoinCondition &dcond case ExpressionType::COMPARE_LESSTHAN: join_comparision = "lt"; break; + case ExpressionType::COMPARE_NOTEQUAL: + join_comparision = "not_equal"; + break; default: throw InternalException("Unsupported join comparison: " + ExpressionTypeToOperator(dcond.comparison)); } @@ -854,6 +857,42 @@ substrait::Rel *DuckDBToSubstrait::TransformOrderBy(LogicalOperator &dop) { return res; } +substrait::Rel *DuckDBToSubstrait::ProjectJoinRelation(LogicalComparisonJoin &djoin, substrait::Rel *join_relation, + idx_t left_col_count) { + + // somewhat odd semantics on our side + if (djoin.left_projection_map.empty()) { + for (uint64_t i = 0; i < djoin.children[0]->types.size(); i++) { + djoin.left_projection_map.push_back(i); + } + } + if (djoin.right_projection_map.empty()) { + for (uint64_t i = 0; i < djoin.children[1]->types.size(); i++) { + djoin.right_projection_map.push_back(i); + } + } + auto proj_rel = new substrait::Rel(); + auto projection = proj_rel->mutable_project(); + if (djoin.join_type == JoinType::RIGHT_SEMI || djoin.join_type == JoinType::RIGHT_ANTI) { + // We project everything from the right table + for (auto right_idx : djoin.right_projection_map) { + CreateFieldRef(projection->add_expressions(), right_idx); + } + } else { + for (auto left_idx : djoin.left_projection_map) { + CreateFieldRef(projection->add_expressions(), left_idx); + } + if (djoin.join_type != JoinType::SEMI) { + for (auto right_idx : djoin.right_projection_map) { + CreateFieldRef(projection->add_expressions(), right_idx + left_col_count); + } + } + } + + projection->set_allocated_input(join_relation); + return proj_rel; +} + substrait::Rel *DuckDBToSubstrait::TransformComparisonJoin(LogicalOperator &dop) { auto res = new substrait::Rel(); auto sjoin = res->mutable_join(); @@ -884,41 +923,101 @@ substrait::Rel *DuckDBToSubstrait::TransformComparisonJoin(LogicalOperator &dop) sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_RIGHT); break; case JoinType::SINGLE: - sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_SINGLE); + sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE); break; case JoinType::SEMI: - sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_SEMI); + sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI); break; case JoinType::MARK: - sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_MARK); + sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_LEFT_MARK); + break; + case JoinType::RIGHT_SEMI: + sjoin->set_type(substrait::JoinRel::JoinType::JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI); break; default: throw InternalException("Unsupported join type " + JoinTypeToString(djoin.join_type)); } - // somewhat odd semantics on our side - if (djoin.left_projection_map.empty()) { - for (uint64_t i = 0; i < dop.children[0]->types.size(); i++) { - djoin.left_projection_map.push_back(i); - } + return ProjectJoinRelation(djoin, res, left_col_count); +} + +substrait::Rel *DuckDBToSubstrait::TransformDelimiterJoin(LogicalOperator &dop) { + auto &djoin = dop.Cast(); + duplicate_eliminated_parent_ptr = &djoin; + auto res = new substrait::Rel(); + + auto sjoin = res->mutable_duplicate_eliminated_join(); + + auto lhs_child = TransformOp(*dop.children[0]); + auto rhs_child = TransformOp(*dop.children[1]); + if (djoin.delim_flipped) { + // right side is where our delim is + sjoin->set_duplicate_eliminated_side(substrait::DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_RIGHT); + plan.add_relations()->set_allocated_rel(rhs_child); + sjoin->set_allocated_left(lhs_child); + auto rhs_res = new substrait::Rel(); + auto rhs_ref_rel = rhs_res->mutable_reference(); + rhs_ref_rel->set_subtree_ordinal(cur_subtree_relation++); + sjoin->set_allocated_right(rhs_res); + } else { + // left side is where our delim is + sjoin->set_duplicate_eliminated_side(substrait::DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_LEFT); + plan.add_relations()->set_allocated_rel(lhs_child); + sjoin->set_allocated_right(rhs_child); + auto lhs_res = new substrait::Rel(); + auto lhs_ref_rel = lhs_res->mutable_reference(); + lhs_ref_rel->set_subtree_ordinal(cur_subtree_relation++); + sjoin->set_allocated_left(lhs_res); } - if (djoin.right_projection_map.empty()) { - for (uint64_t i = 0; i < dop.children[1]->types.size(); i++) { - djoin.right_projection_map.push_back(i); - } + + auto left_col_count = dop.children[0]->types.size(); + if (dop.children[0]->type == LogicalOperatorType::LOGICAL_COMPARISON_JOIN) { + auto child_join = (LogicalComparisonJoin *)dop.children[0].get(); + left_col_count = child_join->left_projection_map.size() + child_join->right_projection_map.size(); } - auto proj_rel = new substrait::Rel(); - auto projection = proj_rel->mutable_project(); - for (auto left_idx : djoin.left_projection_map) { - CreateFieldRef(projection->add_expressions(), left_idx); + sjoin->set_allocated_expression( + CreateConjunction(djoin.conditions, [&](JoinCondition &in) { return TransformJoinCond(in, left_col_count); })); + + switch (djoin.join_type) { + case JoinType::INNER: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_INNER); + break; + case JoinType::LEFT: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT); + break; + case JoinType::RIGHT: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT); + break; + case JoinType::SINGLE: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE); + break; + case JoinType::RIGHT_SEMI: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI); + break; + case JoinType::MARK: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_MARK); + break; + case JoinType::RIGHT_ANTI: + sjoin->set_type( + substrait::DuplicateEliminatedJoinRel_JoinType::DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI); + break; + default: + throw InternalException("Unsupported join type " + JoinTypeToString(djoin.join_type)); } - if (djoin.join_type != JoinType::SEMI) { - for (auto right_idx : djoin.right_projection_map) { - CreateFieldRef(projection->add_expressions(), right_idx + left_col_count); - } + + // Have to add duplicate_eliminated_columns if any + for (auto &dup_col : djoin.duplicate_eliminated_columns) { + auto &dref = dup_col->Cast(); + auto s_dup_col = sjoin->add_duplicate_eliminated_columns(); + s_dup_col->mutable_direct_reference()->mutable_struct_field()->set_field(static_cast(dref.index)); } - projection->set_allocated_input(res); - return proj_rel; + return ProjectJoinRelation(djoin, res, left_col_count); } substrait::Rel *DuckDBToSubstrait::TransformAggregateGroup(LogicalOperator &dop) { @@ -1192,9 +1291,10 @@ substrait::Rel *DuckDBToSubstrait::TransformGet(LogicalOperator &dop) { // fixme: whatever this means projection->set_maintain_singular_struct(true); auto select = new substrait::Expression_MaskExpression_StructSelect(); + auto &column_ids = dget.GetColumnIds(); for (auto col_idx : dget.projection_ids) { auto struct_item = select->add_struct_items(); - struct_item->set_field((int32_t)dget.column_ids[col_idx]); + struct_item->set_field(static_cast(column_ids[col_idx])); // FIXME do we need to set the child? if yes, to what? } projection->set_allocated_select(select); @@ -1294,6 +1394,15 @@ substrait::Rel *DuckDBToSubstrait::TransformIntersect(LogicalOperator &dop) { return rel; } +substrait::Rel *DuckDBToSubstrait::TransformDelimGet() { + auto rel = new substrait::Rel(); + auto delim_get = rel->mutable_duplicate_eliminated_get(); + auto ref_input = delim_get->mutable_input(); + ref_input->set_subtree_ordinal(cur_subtree_relation); + duplicate_eliminated_parent_ptr = nullptr; + return rel; +} + substrait::Rel *DuckDBToSubstrait::TransformOp(LogicalOperator &dop) { switch (dop.type) { case LogicalOperatorType::LOGICAL_FILTER: @@ -1306,6 +1415,8 @@ substrait::Rel *DuckDBToSubstrait::TransformOp(LogicalOperator &dop) { return TransformOrderBy(dop); case LogicalOperatorType::LOGICAL_PROJECTION: return TransformProjection(dop); + case LogicalOperatorType::LOGICAL_DELIM_JOIN: + return TransformDelimiterJoin(dop); case LogicalOperatorType::LOGICAL_COMPARISON_JOIN: return TransformComparisonJoin(dop); case LogicalOperatorType::LOGICAL_AGGREGATE_AND_GROUP_BY: @@ -1322,6 +1433,8 @@ substrait::Rel *DuckDBToSubstrait::TransformOp(LogicalOperator &dop) { return TransformExcept(dop); case LogicalOperatorType::LOGICAL_INTERSECT: return TransformIntersect(dop); + case LogicalOperatorType::LOGICAL_DELIM_GET: + return TransformDelimGet(); default: throw InternalException(LogicalOperatorToString(dop.type)); } @@ -1332,8 +1445,7 @@ static bool IsSetOperation(LogicalOperator &op) { op.type == LogicalOperatorType::LOGICAL_INTERSECT; } -substrait::RelRoot *DuckDBToSubstrait::TransformRootOp(LogicalOperator &dop) { - auto root_rel = new substrait::RelRoot(); +void DuckDBToSubstrait::TransformRootOp(substrait::RelRoot &root_rel, LogicalOperator &dop) { LogicalOperator *current_op = &dop; bool weird_scenario = current_op->type == LogicalOperatorType::LOGICAL_PROJECTION && current_op->children[0]->type == LogicalOperatorType::LOGICAL_TOP_N; @@ -1358,35 +1470,33 @@ substrait::RelRoot *DuckDBToSubstrait::TransformRootOp(LogicalOperator &dop) { } current_op = current_op->children[0].get(); } - root_rel->set_allocated_input(TransformOp(dop)); + root_rel.set_allocated_input(TransformOp(dop)); auto &dproj = (LogicalProjection &)*current_op; if (!weird_scenario) { for (auto &expression : dproj.expressions) { - root_rel->add_names(expression->GetName()); + root_rel.add_names(expression->GetName()); } } else { for (auto &expression : dop.expressions) { D_ASSERT(expression->type == ExpressionType::BOUND_REF); auto b_expr = (BoundReferenceExpression *)expression.get(); - root_rel->add_names(dproj.expressions[b_expr->index]->GetName()); + root_rel.add_names(dproj.expressions[b_expr->index]->GetName()); } } - - return root_rel; } void DuckDBToSubstrait::TransformPlan(LogicalOperator &dop) { - substrait::RelRoot *root; + auto root = new substrait::RelRoot(); + plan.add_relations()->set_allocated_root(root); try { - root = TransformRootOp(dop); + TransformRootOp(*root, dop); } catch (std::exception &ex) { // Ideally we don't have to do that, we should change to capture the error and throw it here at some point - ::duckdb::ErrorData parsed_error(ex); + ErrorData parsed_error(ex); throw NotImplementedException("Something in this plan is not yet implemented in the DuckDB -> Substrait plan " "conversion.\n DuckDB Plan:\n" + dop.ToString() + "Not Implemented error message: " + parsed_error.RawMessage()); } - plan.add_relations()->set_allocated_root(root); if (strict && !errors.empty()) { throw InvalidInputException("Strict Mode is set to true, and the following warnings/errors happened. \n" + errors); diff --git a/substrait b/substrait index 1a51b3d..d81a9d0 160000 --- a/substrait +++ b/substrait @@ -1 +1 @@ -Subproject commit 1a51b3d49ba8323134314ef94acf49c910ff520d +Subproject commit d81a9d0ec435e6541134d6663420e8ea06e81532 diff --git a/test/sql/test_substrait_subqueries.test b/test/sql/test_substrait_subqueries.test index 305eb96..7b4d46b 100644 --- a/test/sql/test_substrait_subqueries.test +++ b/test/sql/test_substrait_subqueries.test @@ -11,15 +11,33 @@ statement ok CREATE TABLE integers (i integer); statement ok -insert into integers values (1),(2),(3),(NULL); +INSERT INTO integers VALUES (NULL) + +statement ok +CREATE TABLE integers_2 (j integer); + +statement ok +INSERT INTO integers_2 VALUES (NULL) + +statement ok +CALL get_substrait('SELECT (select SUM(i) OR true as i from integers where i = integers_2.j) FROM integers_2;') + +statement ok +drop table integers + +statement ok +CREATE TABLE integers (i integer, j integer); + +statement ok +insert into integers values (1,1),(2,2),(3,3),(NULL,NULL); # Uncorrelated Scalar statement ok CALL get_substrait('select i, i + (select MIN(i) from integers) from integers order by i') # Uncorrelated Any (Missing Mark Join) -statement ok -CALL get_substrait('select i = ANY(select * from integers where i is not null) from integers') +#statement ok +#CALL get_substrait('select i = ANY(select * from integers where i is not null) from integers') # Uncorrelated Exist statement ok diff --git a/test/sql/test_substrait_substring.test b/test/sql/test_substrait_substring.test new file mode 100644 index 0000000..9831968 --- /dev/null +++ b/test/sql/test_substrait_substring.test @@ -0,0 +1,27 @@ +# name: test/sql/test_substrait_substring.test +# description: Test get_substrait with simple subqueries +# group: [sql] + +require substrait + +statement ok +PRAGMA enable_verification + +statement ok +create table t (a varchar) + +statement ok +insert into t values ('pedro'), ('mark'), ('diego') + +query I +select substring(a FROM 1 FOR 2) from t; +---- +pe +ma +di + +statement ok +CALL get_substrait('select substring(a FROM 1 FOR 2) from t'); + +statement ok +CALL get_substrait('select substring(a FROM 1 FOR 2) in (''pe'', ''ma'') from t'); \ No newline at end of file diff --git a/test/sql/test_substrait_tpch.test b/test/sql/test_substrait_tpch.test index dcc3323..48baab5 100644 --- a/test/sql/test_substrait_tpch.test +++ b/test/sql/test_substrait_tpch.test @@ -2,12 +2,6 @@ # description: Test get_substrait with TPC-H queries # group: [sql] -# test skipped since PR https://github.com/duckdb/duckdb/pull/9993 -# the PR re-introduces DelimJoins in TPC-H again for performance reasons -# if there is a selection in the duplicate-eliminated side, we keep the DelimJoin -# this is checked in Deliminator::HasSelection -# if this function returns false, all DelimJoins are removed from TPC-H - require substrait require tpch @@ -22,17 +16,17 @@ CALL dbgen(sf=0.01) statement ok CALL get_substrait('SELECT l_returnflag, l_linestatus, sum(l_quantity) AS sum_qty, sum(l_extendedprice) AS sum_base_price, sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, avg(l_quantity) AS avg_qty, avg(l_extendedprice) AS avg_price, avg(l_discount) AS avg_disc, count(*) AS count_order FROM lineitem WHERE l_shipdate <= CAST(''1998-09-02'' AS date) GROUP BY l_returnflag, l_linestatus ORDER BY l_returnflag, l_linestatus;') -#Q 02 (DELIM_JOIN) -#statement ok -#CALL get_substrait('SELECT s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment FROM part, supplier, partsupp, nation, region WHERE p_partkey = ps_partkey AND s_suppkey = ps_suppkey AND p_size = 15 AND p_type LIKE ''%BRASS'' AND s_nationkey = n_nationkey AND n_regionkey = r_regionkey AND r_name = ''EUROPE'' AND ps_supplycost = ( SELECT min(ps_supplycost) FROM partsupp, supplier, nation, region WHERE p_partkey = ps_partkey AND s_suppkey = ps_suppkey AND s_nationkey = n_nationkey AND n_regionkey = r_regionkey AND r_name = ''EUROPE'') ORDER BY s_acctbal DESC, n_name, s_name, p_partkey LIMIT 100;') +#Q 02 +statement ok +CALL get_substrait('SELECT s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment FROM part, supplier, partsupp, nation, region WHERE p_partkey = ps_partkey AND s_suppkey = ps_suppkey AND p_size = 15 AND p_type LIKE ''%BRASS'' AND s_nationkey = n_nationkey AND n_regionkey = r_regionkey AND r_name = ''EUROPE'' AND ps_supplycost = ( SELECT min(ps_supplycost) FROM partsupp, supplier, nation, region WHERE p_partkey = ps_partkey AND s_suppkey = ps_suppkey AND s_nationkey = n_nationkey AND n_regionkey = r_regionkey AND r_name = ''EUROPE'') ORDER BY s_acctbal DESC, n_name, s_name, p_partkey LIMIT 100;') #Q 03 statement ok CALL get_substrait('SELECT l_orderkey, sum(l_extendedprice * (1 - l_discount)) AS revenue, o_orderdate, o_shippriority FROM customer, orders, lineitem WHERE c_mktsegment = ''BUILDING'' AND c_custkey = o_custkey AND l_orderkey = o_orderkey AND o_orderdate < CAST(''1995-03-15'' AS date) AND l_shipdate > CAST(''1995-03-15'' AS date) GROUP BY l_orderkey, o_orderdate, o_shippriority ORDER BY revenue DESC, o_orderdate LIMIT 10;') -#Q 04 DELIM_JOIN -#statement ok -#CALL get_substrait('SELECT o_orderpriority, count(*) AS order_count FROM orders WHERE o_orderdate >= CAST(''1993-07-01'' AS date) AND o_orderdate < CAST(''1993-10-01'' AS date) AND EXISTS ( SELECT * FROM lineitem WHERE l_orderkey = o_orderkey AND l_commitdate < l_receiptdate) GROUP BY o_orderpriority ORDER BY o_orderpriority;') +#Q 04 +statement ok +CALL get_substrait('SELECT o_orderpriority, count(*) AS order_count FROM orders WHERE o_orderdate >= CAST(''1993-07-01'' AS date) AND o_orderdate < CAST(''1993-10-01'' AS date) AND EXISTS ( SELECT * FROM lineitem WHERE l_orderkey = o_orderkey AND l_commitdate < l_receiptdate) GROUP BY o_orderpriority ORDER BY o_orderpriority;') #Q 05 statement ok @@ -76,15 +70,15 @@ CALL get_substrait('SELECT 100.00 * sum( CASE WHEN p_type LIKE ''PROMO%'' THEN l #Q 15 statement ok -CALL get_substrait('SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM supplier, ( SELECT l_suppkey AS supplier_no, sum(l_extendedprice * (1 - l_discount)) AS total_revenue FROM lineitem WHERE l_shipdate >= CAST(''1996-01-01'' AS date) AND l_shipdate < CAST(''1996-04-01'' AS date) GROUP BY supplier_no) revenue0 WHERE s_suppkey = supplier_no AND total_revenue = ( SELECT max(total_revenue) FROM ( SELECT l_suppkey AS supplier_no, sum(l_extendedprice * (1 - l_discount)) AS total_revenue FROM lineitem WHERE l_shipdate >= CAST(''1996-01-01'' AS date) AND l_shipdate < CAST(''1996-04-01'' AS date) GROUP BY supplier_no) revenue1) ORDER BY s_suppkey;') +CALL get_substrait('WITH revenue AS ( SELECT l_suppkey AS supplier_no, sum(l_extendedprice * (1 - l_discount)) AS total_revenue FROM lineitem WHERE l_shipdate >= CAST(''1996-01-01'' AS date) AND l_shipdate < CAST(''1996-04-01'' AS date) GROUP BY supplier_no) SELECT s_suppkey, s_name, s_address, s_phone, total_revenue FROM supplier, revenue WHERE s_suppkey = supplier_no AND total_revenue = ( SELECT max(total_revenue) FROM revenue) ORDER BY s_suppkey;') #Q 16 statement ok CALL get_substrait('SELECT p_brand, p_type, p_size, count(DISTINCT ps_suppkey) AS supplier_cnt FROM partsupp, part WHERE p_partkey = ps_partkey AND p_brand <> ''Brand#45'' AND p_type NOT LIKE ''MEDIUM POLISHED%'' AND p_size IN (49, 14, 23, 45, 19, 3, 36, 9) AND ps_suppkey NOT IN ( SELECT s_suppkey FROM supplier WHERE s_comment LIKE ''%Customer%Complaints%'') GROUP BY p_brand, p_type, p_size ORDER BY supplier_cnt DESC, p_brand, p_type, p_size;') -#Q 17 (DELIM_JOIN) -#statement ok -#CALL get_substrait('SELECT sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, part WHERE p_partkey = l_partkey AND p_brand = ''Brand#23'' AND p_container = ''MED BOX'' AND l_quantity < ( SELECT 0.2 * avg(l_quantity) FROM lineitem WHERE l_partkey = p_partkey);') +#Q 17 +statement ok +CALL get_substrait('SELECT sum(l_extendedprice) / 7.0 AS avg_yearly FROM lineitem, part WHERE p_partkey = l_partkey AND p_brand = ''Brand#23'' AND p_container = ''MED BOX'' AND l_quantity < ( SELECT 0.2 * avg(l_quantity) FROM lineitem WHERE l_partkey = p_partkey);') #Q 18 statement ok @@ -94,17 +88,17 @@ CALL get_substrait('SELECT c_name, c_custkey, o_orderkey, o_orderdate, o_totalpr statement ok CALL get_substrait('SELECT sum(l_extendedprice * (1 - l_discount)) AS revenue FROM lineitem, part WHERE (p_partkey = l_partkey AND p_brand = ''Brand#12'' AND p_container IN (''SM CASE'', ''SM BOX'', ''SM PACK'', ''SM PKG'') AND l_quantity >= 1 AND l_quantity <= 1 + 10 AND p_size BETWEEN 1 AND 5 AND l_shipmode IN (''AIR'', ''AIR REG'') AND l_shipinstruct = ''DELIVER IN PERSON'') OR (p_partkey = l_partkey AND p_brand = ''Brand#23'' AND p_container IN (''MED BAG'', ''MED BOX'', ''MED PKG'', ''MED PACK'') AND l_quantity >= 10 AND l_quantity <= 10 + 10 AND p_size BETWEEN 1 AND 10 AND l_shipmode IN (''AIR'', ''AIR REG'') AND l_shipinstruct = ''DELIVER IN PERSON'') OR (p_partkey = l_partkey AND p_brand = ''Brand#34'' AND p_container IN (''LG CASE'', ''LG BOX'', ''LG PACK'', ''LG PKG'') AND l_quantity >= 20 AND l_quantity <= 20 + 10 AND p_size BETWEEN 1 AND 15 AND l_shipmode IN (''AIR'', ''AIR REG'') AND l_shipinstruct = ''DELIVER IN PERSON'');') -#Q 20 DELIM_JOIN -#statement ok -#CALL get_substrait('SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN ( SELECT ps_suppkey FROM partsupp WHERE ps_partkey IN ( SELECT p_partkey FROM part WHERE p_name LIKE ''forest%'') AND ps_availqty > ( SELECT 0.5 * sum(l_quantity) FROM lineitem WHERE l_partkey = ps_partkey AND l_suppkey = ps_suppkey AND l_shipdate >= CAST(''1994-01-01'' AS date) AND l_shipdate < CAST(''1995-01-01'' AS date))) AND s_nationkey = n_nationkey AND n_name = ''CANADA'' ORDER BY s_name;') +#Q 20 +statement ok +CALL get_substrait('SELECT s_name, s_address FROM supplier, nation WHERE s_suppkey IN ( SELECT ps_suppkey FROM partsupp WHERE ps_partkey IN ( SELECT p_partkey FROM part WHERE p_name LIKE ''forest%'') AND ps_availqty > ( SELECT 0.5 * sum(l_quantity) FROM lineitem WHERE l_partkey = ps_partkey AND l_suppkey = ps_suppkey AND l_shipdate >= CAST(''1994-01-01'' AS date) AND l_shipdate < CAST(''1995-01-01'' AS date))) AND s_nationkey = n_nationkey AND n_name = ''CANADA'' ORDER BY s_name;') -#Q 21 Delim Join -#statement ok -#CALL get_substrait('SELECT s_name, count(*) AS numwait FROM supplier, lineitem l1, orders, nation WHERE s_suppkey = l1.l_suppkey AND o_orderkey = l1.l_orderkey AND o_orderstatus = ''F'' AND l1.l_receiptdate > l1.l_commitdate AND EXISTS ( SELECT * FROM lineitem l2 WHERE l2.l_orderkey = l1.l_orderkey AND l2.l_suppkey <> l1.l_suppkey) AND NOT EXISTS ( SELECT * FROM lineitem l3 WHERE l3.l_orderkey = l1.l_orderkey AND l3.l_suppkey <> l1.l_suppkey AND l3.l_receiptdate > l3.l_commitdate) AND s_nationkey = n_nationkey AND n_name = ''SAUDI ARABIA'' GROUP BY s_name ORDER BY numwait DESC, s_name LIMIT 100;') +#Q 21 +statement ok +CALL get_substrait('SELECT s_name, count(*) AS numwait FROM supplier, lineitem l1, orders, nation WHERE s_suppkey = l1.l_suppkey AND o_orderkey = l1.l_orderkey AND o_orderstatus = ''F'' AND l1.l_receiptdate > l1.l_commitdate AND EXISTS ( SELECT * FROM lineitem l2 WHERE l2.l_orderkey = l1.l_orderkey AND l2.l_suppkey <> l1.l_suppkey) AND NOT EXISTS ( SELECT * FROM lineitem l3 WHERE l3.l_orderkey = l1.l_orderkey AND l3.l_suppkey <> l1.l_suppkey AND l3.l_receiptdate > l3.l_commitdate) AND s_nationkey = n_nationkey AND n_name = ''SAUDI ARABIA'' GROUP BY s_name ORDER BY numwait DESC, s_name LIMIT 100;') -#Q 22 - Delim Join -#statement ok -#CALL get_substrait('SELECT cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal FROM ( SELECT substring(c_phone FROM 1 FOR 2) AS cntrycode, c_acctbal FROM customer WHERE substring(c_phone FROM 1 FOR 2) IN (''13'', ''31'', ''23'', ''29'', ''30'', ''18'', ''17'') AND c_acctbal > ( SELECT avg(c_acctbal) FROM customer WHERE c_acctbal > 0.00 AND substring(c_phone FROM 1 FOR 2) IN (''13'', ''31'', ''23'', ''29'', ''30'', ''18'', ''17'')) AND NOT EXISTS ( SELECT * FROM orders WHERE o_custkey = c_custkey)) AS custsale GROUP BY cntrycode ORDER BY cntrycode;') +#Q 22 +statement ok +CALL get_substrait('SELECT cntrycode, count(*) AS numcust, sum(c_acctbal) AS totacctbal FROM ( SELECT substring(c_phone FROM 1 FOR 2) AS cntrycode, c_acctbal FROM customer WHERE substring(c_phone FROM 1 FOR 2) IN (''13'', ''31'', ''23'', ''29'', ''30'', ''18'', ''17'') AND c_acctbal > ( SELECT avg(c_acctbal) FROM customer WHERE c_acctbal > 0.00 AND substring(c_phone FROM 1 FOR 2) IN (''13'', ''31'', ''23'', ''29'', ''30'', ''18'', ''17'')) AND NOT EXISTS ( SELECT * FROM orders WHERE o_custkey = c_custkey)) AS custsale GROUP BY cntrycode ORDER BY cntrycode;') # Test Get JSON statement ok diff --git a/third_party/substrait/substrait/algebra.pb.cc b/third_party/substrait/substrait/algebra.pb.cc index b93a9e2..50e370b 100644 --- a/third_party/substrait/substrait/algebra.pb.cc +++ b/third_party/substrait/substrait/algebra.pb.cc @@ -69,7 +69,8 @@ struct RelCommon_Hint_RuntimeConstraintDefaultTypeInternal { PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT RelCommon_Hint_RuntimeConstraintDefaultTypeInternal _RelCommon_Hint_RuntimeConstraint_default_instance_; constexpr RelCommon_Hint::RelCommon_Hint( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) - : stats_(nullptr) + : alias_(&::PROTOBUF_NAMESPACE_ID::internal::fixed_address_empty_string) + , stats_(nullptr) , constraint_(nullptr) , advanced_extension_(nullptr){} struct RelCommon_HintDefaultTypeInternal { @@ -176,6 +177,23 @@ struct ReadRel_LocalFiles_FileOrFiles_DwrfReadOptionsDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ReadRel_LocalFiles_FileOrFiles_DwrfReadOptionsDefaultTypeInternal _ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions_default_instance_; +constexpr ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : field_delimiter_(&::PROTOBUF_NAMESPACE_ID::internal::fixed_address_empty_string) + , quote_(&::PROTOBUF_NAMESPACE_ID::internal::fixed_address_empty_string) + , escape_(&::PROTOBUF_NAMESPACE_ID::internal::fixed_address_empty_string) + , value_treated_as_null_(&::PROTOBUF_NAMESPACE_ID::internal::fixed_address_empty_string) + , max_line_size_(uint64_t{0u}) + , header_lines_to_skip_(uint64_t{0u}){} +struct ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal { + constexpr ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal() {} + union { + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal _ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions_default_instance_; constexpr ReadRel_LocalFiles_FileOrFiles::ReadRel_LocalFiles_FileOrFiles( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : partition_index_(uint64_t{0u}) @@ -759,42 +777,41 @@ struct NestedLoopJoinRelDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT NestedLoopJoinRelDefaultTypeInternal _NestedLoopJoinRel_default_instance_; -constexpr MarkJoinRel::MarkJoinRel( +constexpr DuplicateEliminatedGetRel::DuplicateEliminatedGetRel( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : common_(nullptr) - , left_(nullptr) - , right_(nullptr) - , expression_(nullptr) - , advanced_extension_(nullptr){} -struct MarkJoinRelDefaultTypeInternal { - constexpr MarkJoinRelDefaultTypeInternal() + , input_(nullptr){} +struct DuplicateEliminatedGetRelDefaultTypeInternal { + constexpr DuplicateEliminatedGetRelDefaultTypeInternal() : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} - ~MarkJoinRelDefaultTypeInternal() {} + ~DuplicateEliminatedGetRelDefaultTypeInternal() {} union { - MarkJoinRel _instance; + DuplicateEliminatedGetRel _instance; }; }; -PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT MarkJoinRelDefaultTypeInternal _MarkJoinRel_default_instance_; -constexpr DelimiterJoinRel::DelimiterJoinRel( +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DuplicateEliminatedGetRelDefaultTypeInternal _DuplicateEliminatedGetRel_default_instance_; +constexpr DuplicateEliminatedJoinRel::DuplicateEliminatedJoinRel( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) - : left_keys_() - , right_keys_() + : duplicate_eliminated_columns_() , common_(nullptr) , left_(nullptr) , right_(nullptr) - , delimiter_field_(nullptr) + , expression_(nullptr) + , post_join_filter_(nullptr) , advanced_extension_(nullptr) , type_(0) + + , duplicate_eliminated_side_(0) {} -struct DelimiterJoinRelDefaultTypeInternal { - constexpr DelimiterJoinRelDefaultTypeInternal() +struct DuplicateEliminatedJoinRelDefaultTypeInternal { + constexpr DuplicateEliminatedJoinRelDefaultTypeInternal() : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} - ~DelimiterJoinRelDefaultTypeInternal() {} + ~DuplicateEliminatedJoinRelDefaultTypeInternal() {} union { - DelimiterJoinRel _instance; + DuplicateEliminatedJoinRel _instance; }; }; -PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DelimiterJoinRelDefaultTypeInternal _DelimiterJoinRel_default_instance_; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DuplicateEliminatedJoinRelDefaultTypeInternal _DuplicateEliminatedJoinRel_default_instance_; constexpr FunctionArgument::FunctionArgument( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : _oneof_case_{}{} @@ -870,6 +887,19 @@ struct Expression_Literal_DecimalDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Expression_Literal_DecimalDefaultTypeInternal _Expression_Literal_Decimal_default_instance_; +constexpr Expression_Literal_PrecisionTimestamp::Expression_Literal_PrecisionTimestamp( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : value_(int64_t{0}) + , precision_(0){} +struct Expression_Literal_PrecisionTimestampDefaultTypeInternal { + constexpr Expression_Literal_PrecisionTimestampDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~Expression_Literal_PrecisionTimestampDefaultTypeInternal() {} + union { + Expression_Literal_PrecisionTimestamp _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Expression_Literal_PrecisionTimestampDefaultTypeInternal _Expression_Literal_PrecisionTimestamp_default_instance_; constexpr Expression_Literal_Map_KeyValue::Expression_Literal_Map_KeyValue( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : key_(nullptr) @@ -912,7 +942,8 @@ constexpr Expression_Literal_IntervalDayToSecond::Expression_Literal_IntervalDay ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : days_(0) , seconds_(0) - , microseconds_(0){} + , subseconds_(int64_t{0}) + , _oneof_case_{}{} struct Expression_Literal_IntervalDayToSecondDefaultTypeInternal { constexpr Expression_Literal_IntervalDayToSecondDefaultTypeInternal() : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} @@ -922,6 +953,19 @@ struct Expression_Literal_IntervalDayToSecondDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Expression_Literal_IntervalDayToSecondDefaultTypeInternal _Expression_Literal_IntervalDayToSecond_default_instance_; +constexpr Expression_Literal_IntervalCompound::Expression_Literal_IntervalCompound( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : interval_year_to_month_(nullptr) + , interval_day_to_second_(nullptr){} +struct Expression_Literal_IntervalCompoundDefaultTypeInternal { + constexpr Expression_Literal_IntervalCompoundDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~Expression_Literal_IntervalCompoundDefaultTypeInternal() {} + union { + Expression_Literal_IntervalCompound _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Expression_Literal_IntervalCompoundDefaultTypeInternal _Expression_Literal_IntervalCompound_default_instance_; constexpr Expression_Literal_Struct::Expression_Literal_Struct( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : fields_(){} @@ -1633,8 +1677,8 @@ struct ReferenceRelDefaultTypeInternal { }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ReferenceRelDefaultTypeInternal _ReferenceRel_default_instance_; } // namespace substrait -static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2falgebra_2eproto[118]; -static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_substrait_2falgebra_2eproto[19]; +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2falgebra_2eproto[121]; +static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_substrait_2falgebra_2eproto[20]; static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_substrait_2falgebra_2eproto = nullptr; const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { @@ -1675,6 +1719,7 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI ~0u, // no _inlined_string_donated_ PROTOBUF_FIELD_OFFSET(::substrait::RelCommon_Hint, stats_), PROTOBUF_FIELD_OFFSET(::substrait::RelCommon_Hint, constraint_), + PROTOBUF_FIELD_OFFSET(::substrait::RelCommon_Hint, alias_), PROTOBUF_FIELD_OFFSET(::substrait::RelCommon_Hint, advanced_extension_), ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::RelCommon, _internal_metadata_), @@ -1733,6 +1778,24 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI ~0u, // no _oneof_case_ ~0u, // no _weak_field_map_ ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, _has_bits_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, field_delimiter_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, max_line_size_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, quote_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, header_lines_to_skip_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, escape_), + PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, value_treated_as_null_), + ~0u, + ~0u, + ~0u, + ~0u, + ~0u, + 0, ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles, _internal_metadata_), ~0u, // no _extensions_ @@ -1751,6 +1814,7 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles, path_type_), PROTOBUF_FIELD_OFFSET(::substrait::ReadRel_LocalFiles_FileOrFiles, file_format_), ~0u, // no _has_bits_ @@ -2161,30 +2225,28 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI PROTOBUF_FIELD_OFFSET(::substrait::NestedLoopJoinRel, type_), PROTOBUF_FIELD_OFFSET(::substrait::NestedLoopJoinRel, advanced_extension_), ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, _internal_metadata_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedGetRel, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ ~0u, // no _weak_field_map_ ~0u, // no _inlined_string_donated_ - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, common_), - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, left_), - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, right_), - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, expression_), - PROTOBUF_FIELD_OFFSET(::substrait::MarkJoinRel, advanced_extension_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedGetRel, common_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedGetRel, input_), ~0u, // no _has_bits_ - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, _internal_metadata_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ ~0u, // no _weak_field_map_ ~0u, // no _inlined_string_donated_ - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, common_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, left_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, right_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, left_keys_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, right_keys_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, type_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, delimiter_field_), - PROTOBUF_FIELD_OFFSET(::substrait::DelimiterJoinRel, advanced_extension_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, common_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, left_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, right_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, expression_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, post_join_filter_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, type_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, duplicate_eliminated_columns_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, duplicate_eliminated_side_), + PROTOBUF_FIELD_OFFSET(::substrait::DuplicateEliminatedJoinRel, advanced_extension_), ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::FunctionArgument, _internal_metadata_), ~0u, // no _extensions_ @@ -2236,6 +2298,14 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_Decimal, precision_), PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_Decimal, scale_), ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_PrecisionTimestamp, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_PrecisionTimestamp, precision_), + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_PrecisionTimestamp, value_), + ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_Map_KeyValue, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ @@ -2261,12 +2331,23 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, _internal_metadata_), ~0u, // no _extensions_ - ~0u, // no _oneof_case_ + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, _oneof_case_[0]), ~0u, // no _weak_field_map_ ~0u, // no _inlined_string_donated_ PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, days_), PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, seconds_), - PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, microseconds_), + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, subseconds_), + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalDayToSecond, precision_mode_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalCompound, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalCompound, interval_year_to_month_), + PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_IntervalCompound, interval_day_to_second_), ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal_Struct, _internal_metadata_), ~0u, // no _extensions_ @@ -2327,6 +2408,7 @@ const uint32_t TableStruct_substrait_2falgebra_2eproto::offsets[] PROTOBUF_SECTI ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal, nullable_), PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal, type_variation_reference_), PROTOBUF_FIELD_OFFSET(::substrait::Expression_Literal, literal_type_), @@ -2779,119 +2861,122 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOB { 13, -1, -1, sizeof(::substrait::RelCommon_Hint_Stats)}, { 22, -1, -1, sizeof(::substrait::RelCommon_Hint_RuntimeConstraint)}, { 29, -1, -1, sizeof(::substrait::RelCommon_Hint)}, - { 38, -1, -1, sizeof(::substrait::RelCommon)}, - { 49, -1, -1, sizeof(::substrait::ReadRel_NamedTable)}, - { 57, -1, -1, sizeof(::substrait::ReadRel_VirtualTable)}, - { 64, -1, -1, sizeof(::substrait::ReadRel_ExtensionTable)}, - { 71, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions)}, - { 77, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions)}, - { 83, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions)}, - { 89, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions)}, - { 95, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles)}, - { 115, -1, -1, sizeof(::substrait::ReadRel_LocalFiles)}, - { 123, -1, -1, sizeof(::substrait::ReadRel)}, - { 140, -1, -1, sizeof(::substrait::ProjectRel)}, - { 150, -1, -1, sizeof(::substrait::JoinRel)}, - { 163, -1, -1, sizeof(::substrait::CrossRel)}, - { 173, -1, -1, sizeof(::substrait::FetchRel)}, - { 184, -1, -1, sizeof(::substrait::AggregateRel_Grouping)}, - { 191, -1, -1, sizeof(::substrait::AggregateRel_Measure)}, - { 199, -1, -1, sizeof(::substrait::AggregateRel)}, - { 210, -1, -1, sizeof(::substrait::ConsistentPartitionWindowRel_WindowRelFunction)}, - { 225, -1, -1, sizeof(::substrait::ConsistentPartitionWindowRel)}, - { 237, -1, -1, sizeof(::substrait::SortRel)}, - { 247, -1, -1, sizeof(::substrait::FilterRel)}, - { 257, -1, -1, sizeof(::substrait::SetRel)}, - { 267, -1, -1, sizeof(::substrait::ExtensionSingleRel)}, - { 276, -1, -1, sizeof(::substrait::ExtensionLeafRel)}, - { 284, -1, -1, sizeof(::substrait::ExtensionMultiRel)}, - { 293, -1, -1, sizeof(::substrait::ExchangeRel_ScatterFields)}, - { 300, -1, -1, sizeof(::substrait::ExchangeRel_SingleBucketExpression)}, - { 307, -1, -1, sizeof(::substrait::ExchangeRel_MultiBucketExpression)}, - { 315, -1, -1, sizeof(::substrait::ExchangeRel_Broadcast)}, - { 321, -1, -1, sizeof(::substrait::ExchangeRel_RoundRobin)}, - { 328, -1, -1, sizeof(::substrait::ExchangeRel_ExchangeTarget)}, - { 338, -1, -1, sizeof(::substrait::ExchangeRel)}, - { 355, -1, -1, sizeof(::substrait::ExpandRel_ExpandField)}, - { 364, -1, -1, sizeof(::substrait::ExpandRel_SwitchingField)}, - { 371, -1, -1, sizeof(::substrait::ExpandRel)}, - { 380, -1, -1, sizeof(::substrait::RelRoot)}, - { 388, -1, -1, sizeof(::substrait::Rel)}, - { 418, -1, -1, sizeof(::substrait::NamedObjectWrite)}, - { 426, -1, -1, sizeof(::substrait::ExtensionObject)}, - { 433, -1, -1, sizeof(::substrait::DdlRel)}, - { 448, -1, -1, sizeof(::substrait::WriteRel)}, - { 462, -1, -1, sizeof(::substrait::ComparisonJoinKey_ComparisonType)}, - { 471, -1, -1, sizeof(::substrait::ComparisonJoinKey)}, - { 480, -1, -1, sizeof(::substrait::HashJoinRel)}, - { 495, -1, -1, sizeof(::substrait::MergeJoinRel)}, - { 510, -1, -1, sizeof(::substrait::NestedLoopJoinRel)}, - { 522, -1, -1, sizeof(::substrait::MarkJoinRel)}, - { 533, -1, -1, sizeof(::substrait::DelimiterJoinRel)}, - { 547, -1, -1, sizeof(::substrait::FunctionArgument)}, - { 557, -1, -1, sizeof(::substrait::FunctionOption)}, - { 565, -1, -1, sizeof(::substrait::Expression_Enum_Empty)}, - { 571, -1, -1, sizeof(::substrait::Expression_Enum)}, - { 580, -1, -1, sizeof(::substrait::Expression_Literal_VarChar)}, - { 588, -1, -1, sizeof(::substrait::Expression_Literal_Decimal)}, - { 597, -1, -1, sizeof(::substrait::Expression_Literal_Map_KeyValue)}, - { 605, -1, -1, sizeof(::substrait::Expression_Literal_Map)}, - { 612, -1, -1, sizeof(::substrait::Expression_Literal_IntervalYearToMonth)}, - { 620, -1, -1, sizeof(::substrait::Expression_Literal_IntervalDayToSecond)}, - { 629, -1, -1, sizeof(::substrait::Expression_Literal_Struct)}, - { 636, -1, -1, sizeof(::substrait::Expression_Literal_List)}, - { 643, -1, -1, sizeof(::substrait::Expression_Literal_UserDefined)}, - { 654, -1, -1, sizeof(::substrait::Expression_Literal)}, - { 692, -1, -1, sizeof(::substrait::Expression_Nested_Map_KeyValue)}, - { 700, -1, -1, sizeof(::substrait::Expression_Nested_Map)}, - { 707, -1, -1, sizeof(::substrait::Expression_Nested_Struct)}, - { 714, -1, -1, sizeof(::substrait::Expression_Nested_List)}, - { 721, -1, -1, sizeof(::substrait::Expression_Nested)}, - { 733, -1, -1, sizeof(::substrait::Expression_ScalarFunction)}, - { 744, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Preceding)}, - { 751, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Following)}, - { 758, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_CurrentRow)}, - { 764, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Unbounded)}, - { 770, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound)}, - { 781, -1, -1, sizeof(::substrait::Expression_WindowFunction)}, - { 799, -1, -1, sizeof(::substrait::Expression_IfThen_IfClause)}, - { 807, -1, -1, sizeof(::substrait::Expression_IfThen)}, - { 815, -1, -1, sizeof(::substrait::Expression_Cast)}, - { 824, -1, -1, sizeof(::substrait::Expression_SwitchExpression_IfValue)}, - { 832, -1, -1, sizeof(::substrait::Expression_SwitchExpression)}, - { 841, -1, -1, sizeof(::substrait::Expression_SingularOrList)}, - { 849, -1, -1, sizeof(::substrait::Expression_MultiOrList_Record)}, - { 856, -1, -1, sizeof(::substrait::Expression_MultiOrList)}, - { 864, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction_PythonPickleFunction)}, - { 872, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction_WebAssemblyFunction)}, - { 880, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction)}, - { 891, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_MapKey)}, - { 899, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_StructField)}, - { 907, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_ListElement)}, - { 915, -1, -1, sizeof(::substrait::Expression_ReferenceSegment)}, - { 925, -1, -1, sizeof(::substrait::Expression_MaskExpression_Select)}, - { 935, -1, -1, sizeof(::substrait::Expression_MaskExpression_StructSelect)}, - { 942, -1, -1, sizeof(::substrait::Expression_MaskExpression_StructItem)}, - { 950, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem_ListElement)}, - { 957, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice)}, - { 965, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem)}, - { 974, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect)}, - { 982, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect_MapKey)}, - { 989, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect_MapKeyExpression)}, - { 996, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect)}, - { 1006, -1, -1, sizeof(::substrait::Expression_MaskExpression)}, - { 1014, -1, -1, sizeof(::substrait::Expression_FieldReference_RootReference)}, - { 1020, -1, -1, sizeof(::substrait::Expression_FieldReference_OuterReference)}, - { 1027, -1, -1, sizeof(::substrait::Expression_FieldReference)}, - { 1040, -1, -1, sizeof(::substrait::Expression_Subquery_Scalar)}, - { 1047, -1, -1, sizeof(::substrait::Expression_Subquery_InPredicate)}, - { 1055, -1, -1, sizeof(::substrait::Expression_Subquery_SetPredicate)}, - { 1063, -1, -1, sizeof(::substrait::Expression_Subquery_SetComparison)}, - { 1073, -1, -1, sizeof(::substrait::Expression_Subquery)}, - { 1084, -1, -1, sizeof(::substrait::Expression)}, - { 1103, -1, -1, sizeof(::substrait::SortField)}, - { 1113, -1, -1, sizeof(::substrait::AggregateFunction)}, - { 1127, -1, -1, sizeof(::substrait::ReferenceRel)}, + { 39, -1, -1, sizeof(::substrait::RelCommon)}, + { 50, -1, -1, sizeof(::substrait::ReadRel_NamedTable)}, + { 58, -1, -1, sizeof(::substrait::ReadRel_VirtualTable)}, + { 65, -1, -1, sizeof(::substrait::ReadRel_ExtensionTable)}, + { 72, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions)}, + { 78, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions)}, + { 84, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions)}, + { 90, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions)}, + { 96, 108, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions)}, + { 114, -1, -1, sizeof(::substrait::ReadRel_LocalFiles_FileOrFiles)}, + { 135, -1, -1, sizeof(::substrait::ReadRel_LocalFiles)}, + { 143, -1, -1, sizeof(::substrait::ReadRel)}, + { 160, -1, -1, sizeof(::substrait::ProjectRel)}, + { 170, -1, -1, sizeof(::substrait::JoinRel)}, + { 183, -1, -1, sizeof(::substrait::CrossRel)}, + { 193, -1, -1, sizeof(::substrait::FetchRel)}, + { 204, -1, -1, sizeof(::substrait::AggregateRel_Grouping)}, + { 211, -1, -1, sizeof(::substrait::AggregateRel_Measure)}, + { 219, -1, -1, sizeof(::substrait::AggregateRel)}, + { 230, -1, -1, sizeof(::substrait::ConsistentPartitionWindowRel_WindowRelFunction)}, + { 245, -1, -1, sizeof(::substrait::ConsistentPartitionWindowRel)}, + { 257, -1, -1, sizeof(::substrait::SortRel)}, + { 267, -1, -1, sizeof(::substrait::FilterRel)}, + { 277, -1, -1, sizeof(::substrait::SetRel)}, + { 287, -1, -1, sizeof(::substrait::ExtensionSingleRel)}, + { 296, -1, -1, sizeof(::substrait::ExtensionLeafRel)}, + { 304, -1, -1, sizeof(::substrait::ExtensionMultiRel)}, + { 313, -1, -1, sizeof(::substrait::ExchangeRel_ScatterFields)}, + { 320, -1, -1, sizeof(::substrait::ExchangeRel_SingleBucketExpression)}, + { 327, -1, -1, sizeof(::substrait::ExchangeRel_MultiBucketExpression)}, + { 335, -1, -1, sizeof(::substrait::ExchangeRel_Broadcast)}, + { 341, -1, -1, sizeof(::substrait::ExchangeRel_RoundRobin)}, + { 348, -1, -1, sizeof(::substrait::ExchangeRel_ExchangeTarget)}, + { 358, -1, -1, sizeof(::substrait::ExchangeRel)}, + { 375, -1, -1, sizeof(::substrait::ExpandRel_ExpandField)}, + { 384, -1, -1, sizeof(::substrait::ExpandRel_SwitchingField)}, + { 391, -1, -1, sizeof(::substrait::ExpandRel)}, + { 400, -1, -1, sizeof(::substrait::RelRoot)}, + { 408, -1, -1, sizeof(::substrait::Rel)}, + { 438, -1, -1, sizeof(::substrait::NamedObjectWrite)}, + { 446, -1, -1, sizeof(::substrait::ExtensionObject)}, + { 453, -1, -1, sizeof(::substrait::DdlRel)}, + { 468, -1, -1, sizeof(::substrait::WriteRel)}, + { 482, -1, -1, sizeof(::substrait::ComparisonJoinKey_ComparisonType)}, + { 491, -1, -1, sizeof(::substrait::ComparisonJoinKey)}, + { 500, -1, -1, sizeof(::substrait::HashJoinRel)}, + { 515, -1, -1, sizeof(::substrait::MergeJoinRel)}, + { 530, -1, -1, sizeof(::substrait::NestedLoopJoinRel)}, + { 542, -1, -1, sizeof(::substrait::DuplicateEliminatedGetRel)}, + { 550, -1, -1, sizeof(::substrait::DuplicateEliminatedJoinRel)}, + { 565, -1, -1, sizeof(::substrait::FunctionArgument)}, + { 575, -1, -1, sizeof(::substrait::FunctionOption)}, + { 583, -1, -1, sizeof(::substrait::Expression_Enum_Empty)}, + { 589, -1, -1, sizeof(::substrait::Expression_Enum)}, + { 598, -1, -1, sizeof(::substrait::Expression_Literal_VarChar)}, + { 606, -1, -1, sizeof(::substrait::Expression_Literal_Decimal)}, + { 615, -1, -1, sizeof(::substrait::Expression_Literal_PrecisionTimestamp)}, + { 623, -1, -1, sizeof(::substrait::Expression_Literal_Map_KeyValue)}, + { 631, -1, -1, sizeof(::substrait::Expression_Literal_Map)}, + { 638, -1, -1, sizeof(::substrait::Expression_Literal_IntervalYearToMonth)}, + { 646, -1, -1, sizeof(::substrait::Expression_Literal_IntervalDayToSecond)}, + { 658, -1, -1, sizeof(::substrait::Expression_Literal_IntervalCompound)}, + { 666, -1, -1, sizeof(::substrait::Expression_Literal_Struct)}, + { 673, -1, -1, sizeof(::substrait::Expression_Literal_List)}, + { 680, -1, -1, sizeof(::substrait::Expression_Literal_UserDefined)}, + { 691, -1, -1, sizeof(::substrait::Expression_Literal)}, + { 730, -1, -1, sizeof(::substrait::Expression_Nested_Map_KeyValue)}, + { 738, -1, -1, sizeof(::substrait::Expression_Nested_Map)}, + { 745, -1, -1, sizeof(::substrait::Expression_Nested_Struct)}, + { 752, -1, -1, sizeof(::substrait::Expression_Nested_List)}, + { 759, -1, -1, sizeof(::substrait::Expression_Nested)}, + { 771, -1, -1, sizeof(::substrait::Expression_ScalarFunction)}, + { 782, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Preceding)}, + { 789, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Following)}, + { 796, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_CurrentRow)}, + { 802, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound_Unbounded)}, + { 808, -1, -1, sizeof(::substrait::Expression_WindowFunction_Bound)}, + { 819, -1, -1, sizeof(::substrait::Expression_WindowFunction)}, + { 837, -1, -1, sizeof(::substrait::Expression_IfThen_IfClause)}, + { 845, -1, -1, sizeof(::substrait::Expression_IfThen)}, + { 853, -1, -1, sizeof(::substrait::Expression_Cast)}, + { 862, -1, -1, sizeof(::substrait::Expression_SwitchExpression_IfValue)}, + { 870, -1, -1, sizeof(::substrait::Expression_SwitchExpression)}, + { 879, -1, -1, sizeof(::substrait::Expression_SingularOrList)}, + { 887, -1, -1, sizeof(::substrait::Expression_MultiOrList_Record)}, + { 894, -1, -1, sizeof(::substrait::Expression_MultiOrList)}, + { 902, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction_PythonPickleFunction)}, + { 910, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction_WebAssemblyFunction)}, + { 918, -1, -1, sizeof(::substrait::Expression_EmbeddedFunction)}, + { 929, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_MapKey)}, + { 937, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_StructField)}, + { 945, -1, -1, sizeof(::substrait::Expression_ReferenceSegment_ListElement)}, + { 953, -1, -1, sizeof(::substrait::Expression_ReferenceSegment)}, + { 963, -1, -1, sizeof(::substrait::Expression_MaskExpression_Select)}, + { 973, -1, -1, sizeof(::substrait::Expression_MaskExpression_StructSelect)}, + { 980, -1, -1, sizeof(::substrait::Expression_MaskExpression_StructItem)}, + { 988, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem_ListElement)}, + { 995, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice)}, + { 1003, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect_ListSelectItem)}, + { 1012, -1, -1, sizeof(::substrait::Expression_MaskExpression_ListSelect)}, + { 1020, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect_MapKey)}, + { 1027, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect_MapKeyExpression)}, + { 1034, -1, -1, sizeof(::substrait::Expression_MaskExpression_MapSelect)}, + { 1044, -1, -1, sizeof(::substrait::Expression_MaskExpression)}, + { 1052, -1, -1, sizeof(::substrait::Expression_FieldReference_RootReference)}, + { 1058, -1, -1, sizeof(::substrait::Expression_FieldReference_OuterReference)}, + { 1065, -1, -1, sizeof(::substrait::Expression_FieldReference)}, + { 1078, -1, -1, sizeof(::substrait::Expression_Subquery_Scalar)}, + { 1085, -1, -1, sizeof(::substrait::Expression_Subquery_InPredicate)}, + { 1093, -1, -1, sizeof(::substrait::Expression_Subquery_SetPredicate)}, + { 1101, -1, -1, sizeof(::substrait::Expression_Subquery_SetComparison)}, + { 1111, -1, -1, sizeof(::substrait::Expression_Subquery)}, + { 1122, -1, -1, sizeof(::substrait::Expression)}, + { 1141, -1, -1, sizeof(::substrait::SortField)}, + { 1151, -1, -1, sizeof(::substrait::AggregateFunction)}, + { 1165, -1, -1, sizeof(::substrait::ReferenceRel)}, }; static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { @@ -2908,6 +2993,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = reinterpret_cast(&::substrait::_ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions_default_instance_), reinterpret_cast(&::substrait::_ReadRel_LocalFiles_FileOrFiles_OrcReadOptions_default_instance_), reinterpret_cast(&::substrait::_ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions_default_instance_), + reinterpret_cast(&::substrait::_ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions_default_instance_), reinterpret_cast(&::substrait::_ReadRel_LocalFiles_FileOrFiles_default_instance_), reinterpret_cast(&::substrait::_ReadRel_LocalFiles_default_instance_), reinterpret_cast(&::substrait::_ReadRel_default_instance_), @@ -2947,18 +3033,20 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = reinterpret_cast(&::substrait::_HashJoinRel_default_instance_), reinterpret_cast(&::substrait::_MergeJoinRel_default_instance_), reinterpret_cast(&::substrait::_NestedLoopJoinRel_default_instance_), - reinterpret_cast(&::substrait::_MarkJoinRel_default_instance_), - reinterpret_cast(&::substrait::_DelimiterJoinRel_default_instance_), + reinterpret_cast(&::substrait::_DuplicateEliminatedGetRel_default_instance_), + reinterpret_cast(&::substrait::_DuplicateEliminatedJoinRel_default_instance_), reinterpret_cast(&::substrait::_FunctionArgument_default_instance_), reinterpret_cast(&::substrait::_FunctionOption_default_instance_), reinterpret_cast(&::substrait::_Expression_Enum_Empty_default_instance_), reinterpret_cast(&::substrait::_Expression_Enum_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_VarChar_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_Decimal_default_instance_), + reinterpret_cast(&::substrait::_Expression_Literal_PrecisionTimestamp_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_Map_KeyValue_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_Map_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_IntervalYearToMonth_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_IntervalDayToSecond_default_instance_), + reinterpret_cast(&::substrait::_Expression_Literal_IntervalCompound_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_Struct_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_List_default_instance_), reinterpret_cast(&::substrait::_Expression_Literal_UserDefined_default_instance_), @@ -3019,590 +3107,629 @@ const char descriptor_table_protodef_substrait_2falgebra_2eproto[] PROTOBUF_SECT "\n\027substrait/algebra.proto\022\tsubstrait\032\031go" "ogle/protobuf/any.proto\032%substrait/exten" "sions/extensions.proto\032\024substrait/type.p" - "roto\"\231\005\n\tRelCommon\022-\n\006direct\030\001 \001(\0132\033.sub" + "roto\"\250\005\n\tRelCommon\022-\n\006direct\030\001 \001(\0132\033.sub" "strait.RelCommon.DirectH\000\022)\n\004emit\030\002 \001(\0132" "\031.substrait.RelCommon.EmitH\000\022\'\n\004hint\030\003 \001" "(\0132\031.substrait.RelCommon.Hint\022C\n\022advance" "d_extension\030\004 \001(\0132\'.substrait.extensions" ".AdvancedExtension\032\010\n\006Direct\032\036\n\004Emit\022\026\n\016" - "output_mapping\030\001 \003(\005\032\214\003\n\004Hint\022.\n\005stats\030\001" + "output_mapping\030\001 \003(\005\032\233\003\n\004Hint\022.\n\005stats\030\001" " \001(\0132\037.substrait.RelCommon.Hint.Stats\022\?\n" "\nconstraint\030\002 \001(\0132+.substrait.RelCommon." - "Hint.RuntimeConstraint\022C\n\022advanced_exten" - "sion\030\n \001(\0132\'.substrait.extensions.Advanc" - "edExtension\032t\n\005Stats\022\021\n\trow_count\030\001 \001(\001\022" - "\023\n\013record_size\030\002 \001(\001\022C\n\022advanced_extensi" - "on\030\n \001(\0132\'.substrait.extensions.Advanced" - "Extension\032X\n\021RuntimeConstraint\022C\n\022advanc" - "ed_extension\030\n \001(\0132\'.substrait.extension" - "s.AdvancedExtensionB\013\n\temit_kind\"\211\014\n\007Rea" - "dRel\022$\n\006common\030\001 \001(\0132\024.substrait.RelComm" - "on\022+\n\013base_schema\030\002 \001(\0132\026.substrait.Name" - "dStruct\022%\n\006filter\030\003 \001(\0132\025.substrait.Expr" - "ession\0221\n\022best_effort_filter\030\013 \001(\0132\025.sub" - "strait.Expression\0228\n\nprojection\030\004 \001(\0132$." - "substrait.Expression.MaskExpression\022C\n\022a" + "Hint.RuntimeConstraint\022\r\n\005alias\030\003 \001(\t\022C\n" + "\022advanced_extension\030\n \001(\0132\'.substrait.ex" + "tensions.AdvancedExtension\032t\n\005Stats\022\021\n\tr" + "ow_count\030\001 \001(\001\022\023\n\013record_size\030\002 \001(\001\022C\n\022a" "dvanced_extension\030\n \001(\0132\'.substrait.exte" - "nsions.AdvancedExtension\0228\n\rvirtual_tabl" - "e\030\005 \001(\0132\037.substrait.ReadRel.VirtualTable" - "H\000\0224\n\013local_files\030\006 \001(\0132\035.substrait.Read" - "Rel.LocalFilesH\000\0224\n\013named_table\030\007 \001(\0132\035." - "substrait.ReadRel.NamedTableH\000\022<\n\017extens" - "ion_table\030\010 \001(\0132!.substrait.ReadRel.Exte" - "nsionTableH\000\032`\n\nNamedTable\022\r\n\005names\030\001 \003(" - "\t\022C\n\022advanced_extension\030\n \001(\0132\'.substrai" - "t.extensions.AdvancedExtension\032D\n\014Virtua" - "lTable\0224\n\006values\030\001 \003(\0132$.substrait.Expre" - "ssion.Literal.Struct\0326\n\016ExtensionTable\022$" - "\n\006detail\030\001 \001(\0132\024.google.protobuf.Any\032\200\006\n" - "\nLocalFiles\0228\n\005items\030\001 \003(\0132).substrait.R" - "eadRel.LocalFiles.FileOrFiles\022C\n\022advance" - "d_extension\030\n \001(\0132\'.substrait.extensions" - ".AdvancedExtension\032\362\004\n\013FileOrFiles\022\022\n\010ur" - "i_path\030\001 \001(\tH\000\022\027\n\ruri_path_glob\030\002 \001(\tH\000\022" - "\022\n\010uri_file\030\003 \001(\tH\000\022\024\n\nuri_folder\030\004 \001(\tH" - "\000\022\027\n\017partition_index\030\006 \001(\004\022\r\n\005start\030\007 \001(" - "\004\022\016\n\006length\030\010 \001(\004\022O\n\007parquet\030\t \001(\0132<.sub" - "strait.ReadRel.LocalFiles.FileOrFiles.Pa" - "rquetReadOptionsH\001\022K\n\005arrow\030\n \001(\0132:.subs" - "trait.ReadRel.LocalFiles.FileOrFiles.Arr" - "owReadOptionsH\001\022G\n\003orc\030\013 \001(\01328.substrait" - ".ReadRel.LocalFiles.FileOrFiles.OrcReadO" - "ptionsH\001\022)\n\textension\030\014 \001(\0132\024.google.pro" - "tobuf.AnyH\001\022I\n\004dwrf\030\r \001(\01329.substrait.Re" - "adRel.LocalFiles.FileOrFiles.DwrfReadOpt" - "ionsH\001\032\024\n\022ParquetReadOptions\032\022\n\020ArrowRea" - "dOptions\032\020\n\016OrcReadOptions\032\021\n\017DwrfReadOp" - "tionsB\013\n\tpath_typeB\r\n\013file_formatJ\004\010\005\020\006R" - "\006formatB\013\n\tread_type\"\302\001\n\nProjectRel\022$\n\006c" - "ommon\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005inp" - "ut\030\002 \001(\0132\016.substrait.Rel\022*\n\013expressions\030" - "\003 \003(\0132\025.substrait.Expression\022C\n\022advanced" - "_extension\030\n \001(\0132\'.substrait.extensions." - "AdvancedExtension\"\205\004\n\007JoinRel\022$\n\006common\030" - "\001 \001(\0132\024.substrait.RelCommon\022\034\n\004left\030\002 \001(" - "\0132\016.substrait.Rel\022\035\n\005right\030\003 \001(\0132\016.subst" - "rait.Rel\022)\n\nexpression\030\004 \001(\0132\025.substrait" - ".Expression\022/\n\020post_join_filter\030\005 \001(\0132\025." - "substrait.Expression\022)\n\004type\030\006 \001(\0162\033.sub" - "strait.JoinRel.JoinType\022C\n\022advanced_exte" - "nsion\030\n \001(\0132\'.substrait.extensions.Advan" - "cedExtension\"\312\001\n\010JoinType\022\031\n\025JOIN_TYPE_U" - "NSPECIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017JOI" - "N_TYPE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017JO" - "IN_TYPE_RIGHT\020\004\022\022\n\016JOIN_TYPE_SEMI\020\005\022\022\n\016J" - "OIN_TYPE_ANTI\020\006\022\024\n\020JOIN_TYPE_SINGLE\020\007\022\022\n" - "\016JOIN_TYPE_MARK\020\010\"\262\001\n\010CrossRel\022$\n\006common" - "\030\001 \001(\0132\024.substrait.RelCommon\022\034\n\004left\030\002 \001" - "(\0132\016.substrait.Rel\022\035\n\005right\030\003 \001(\0132\016.subs" - "trait.Rel\022C\n\022advanced_extension\030\n \001(\0132\'." - "substrait.extensions.AdvancedExtension\"\263" - "\001\n\010FetchRel\022$\n\006common\030\001 \001(\0132\024.substrait." - "RelCommon\022\035\n\005input\030\002 \001(\0132\016.substrait.Rel" - "\022\016\n\006offset\030\003 \001(\003\022\r\n\005count\030\004 \001(\003\022C\n\022advan" - "ced_extension\030\n \001(\0132\'.substrait.extensio" - "ns.AdvancedExtension\"\242\003\n\014AggregateRel\022$\n" + "nsions.AdvancedExtension\032X\n\021RuntimeConst" + "raint\022C\n\022advanced_extension\030\n \001(\0132\'.subs" + "trait.extensions.AdvancedExtensionB\013\n\tem" + "it_kind\"\267\016\n\007ReadRel\022$\n\006common\030\001 \001(\0132\024.su" + "bstrait.RelCommon\022+\n\013base_schema\030\002 \001(\0132\026" + ".substrait.NamedStruct\022%\n\006filter\030\003 \001(\0132\025" + ".substrait.Expression\0221\n\022best_effort_fil" + "ter\030\013 \001(\0132\025.substrait.Expression\0228\n\nproj" + "ection\030\004 \001(\0132$.substrait.Expression.Mask" + "Expression\022C\n\022advanced_extension\030\n \001(\0132\'" + ".substrait.extensions.AdvancedExtension\022" + "8\n\rvirtual_table\030\005 \001(\0132\037.substrait.ReadR" + "el.VirtualTableH\000\0224\n\013local_files\030\006 \001(\0132\035" + ".substrait.ReadRel.LocalFilesH\000\0224\n\013named" + "_table\030\007 \001(\0132\035.substrait.ReadRel.NamedTa" + "bleH\000\022<\n\017extension_table\030\010 \001(\0132!.substra" + "it.ReadRel.ExtensionTableH\000\032`\n\nNamedTabl" + "e\022\r\n\005names\030\001 \003(\t\022C\n\022advanced_extension\030\n" + " \001(\0132\'.substrait.extensions.AdvancedExte" + "nsion\032D\n\014VirtualTable\0224\n\006values\030\001 \003(\0132$." + "substrait.Expression.Literal.Struct\0326\n\016E" + "xtensionTable\022$\n\006detail\030\001 \001(\0132\024.google.p" + "rotobuf.Any\032\256\010\n\nLocalFiles\0228\n\005items\030\001 \003(" + "\0132).substrait.ReadRel.LocalFiles.FileOrF" + "iles\022C\n\022advanced_extension\030\n \001(\0132\'.subst" + "rait.extensions.AdvancedExtension\032\240\007\n\013Fi" + "leOrFiles\022\022\n\010uri_path\030\001 \001(\tH\000\022\027\n\ruri_pat" + "h_glob\030\002 \001(\tH\000\022\022\n\010uri_file\030\003 \001(\tH\000\022\024\n\nur" + "i_folder\030\004 \001(\tH\000\022\027\n\017partition_index\030\006 \001(" + "\004\022\r\n\005start\030\007 \001(\004\022\016\n\006length\030\010 \001(\004\022O\n\007parq" + "uet\030\t \001(\0132<.substrait.ReadRel.LocalFiles" + ".FileOrFiles.ParquetReadOptionsH\001\022K\n\005arr" + "ow\030\n \001(\0132:.substrait.ReadRel.LocalFiles." + "FileOrFiles.ArrowReadOptionsH\001\022G\n\003orc\030\013 " + "\001(\01328.substrait.ReadRel.LocalFiles.FileO" + "rFiles.OrcReadOptionsH\001\022)\n\textension\030\014 \001" + "(\0132\024.google.protobuf.AnyH\001\022I\n\004dwrf\030\r \001(\013" + "29.substrait.ReadRel.LocalFiles.FileOrFi" + "les.DwrfReadOptionsH\001\022[\n\004text\030\016 \001(\0132K.su" + "bstrait.ReadRel.LocalFiles.FileOrFiles.D" + "elimiterSeparatedTextReadOptionsH\001\032\024\n\022Pa" + "rquetReadOptions\032\022\n\020ArrowReadOptions\032\020\n\016" + "OrcReadOptions\032\021\n\017DwrfReadOptions\032\316\001\n!De" + "limiterSeparatedTextReadOptions\022\027\n\017field" + "_delimiter\030\001 \001(\t\022\025\n\rmax_line_size\030\002 \001(\004\022" + "\r\n\005quote\030\003 \001(\t\022\034\n\024header_lines_to_skip\030\004" + " \001(\004\022\016\n\006escape\030\005 \001(\t\022\"\n\025value_treated_as" + "_null\030\006 \001(\tH\000\210\001\001B\030\n\026_value_treated_as_nu" + "llB\013\n\tpath_typeB\r\n\013file_formatJ\004\010\005\020\006R\006fo" + "rmatB\013\n\tread_type\"\302\001\n\nProjectRel\022$\n\006comm" + "on\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005input\030" + "\002 \001(\0132\016.substrait.Rel\022*\n\013expressions\030\003 \003" + "(\0132\025.substrait.Expression\022C\n\022advanced_ex" + "tension\030\n \001(\0132\'.substrait.extensions.Adv" + "ancedExtension\"\203\005\n\007JoinRel\022$\n\006common\030\001 \001" + "(\0132\024.substrait.RelCommon\022\034\n\004left\030\002 \001(\0132\016" + ".substrait.Rel\022\035\n\005right\030\003 \001(\0132\016.substrai" + "t.Rel\022)\n\nexpression\030\004 \001(\0132\025.substrait.Ex" + "pression\022/\n\020post_join_filter\030\005 \001(\0132\025.sub" + "strait.Expression\022)\n\004type\030\006 \001(\0162\033.substr" + "ait.JoinRel.JoinType\022C\n\022advanced_extensi" + "on\030\n \001(\0132\'.substrait.extensions.Advanced" + "Extension\"\310\002\n\010JoinType\022\031\n\025JOIN_TYPE_UNSP" + "ECIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017JOIN_T" + "YPE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017JOIN_" + "TYPE_RIGHT\020\004\022\027\n\023JOIN_TYPE_LEFT_SEMI\020\005\022\027\n" + "\023JOIN_TYPE_LEFT_ANTI\020\006\022\031\n\025JOIN_TYPE_LEFT" + "_SINGLE\020\007\022\030\n\024JOIN_TYPE_RIGHT_SEMI\020\010\022\030\n\024J" + "OIN_TYPE_RIGHT_ANTI\020\t\022\032\n\026JOIN_TYPE_RIGHT" + "_SINGLE\020\n\022\027\n\023JOIN_TYPE_LEFT_MARK\020\013\022\030\n\024JO" + "IN_TYPE_RIGHT_MARK\020\014\"\262\001\n\010CrossRel\022$\n\006com" + "mon\030\001 \001(\0132\024.substrait.RelCommon\022\034\n\004left\030" + "\002 \001(\0132\016.substrait.Rel\022\035\n\005right\030\003 \001(\0132\016.s" + "ubstrait.Rel\022C\n\022advanced_extension\030\n \001(\013" + "2\'.substrait.extensions.AdvancedExtensio" + "n\"\263\001\n\010FetchRel\022$\n\006common\030\001 \001(\0132\024.substra" + "it.RelCommon\022\035\n\005input\030\002 \001(\0132\016.substrait." + "Rel\022\016\n\006offset\030\003 \001(\003\022\r\n\005count\030\004 \001(\003\022C\n\022ad" + "vanced_extension\030\n \001(\0132\'.substrait.exten" + "sions.AdvancedExtension\"\242\003\n\014AggregateRel" + "\022$\n\006common\030\001 \001(\0132\024.substrait.RelCommon\022\035" + "\n\005input\030\002 \001(\0132\016.substrait.Rel\0223\n\tgroupin" + "gs\030\003 \003(\0132 .substrait.AggregateRel.Groupi" + "ng\0221\n\010measures\030\004 \003(\0132\037.substrait.Aggrega" + "teRel.Measure\022C\n\022advanced_extension\030\n \001(" + "\0132\'.substrait.extensions.AdvancedExtensi" + "on\032\?\n\010Grouping\0223\n\024grouping_expressions\030\001" + " \003(\0132\025.substrait.Expression\032_\n\007Measure\022-" + "\n\007measure\030\001 \001(\0132\034.substrait.AggregateFun" + "ction\022%\n\006filter\030\002 \001(\0132\025.substrait.Expres" + "sion\"\310\006\n\034ConsistentPartitionWindowRel\022$\n" "\006common\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005i" - "nput\030\002 \001(\0132\016.substrait.Rel\0223\n\tgroupings\030" - "\003 \003(\0132 .substrait.AggregateRel.Grouping\022" - "1\n\010measures\030\004 \003(\0132\037.substrait.AggregateR" - "el.Measure\022C\n\022advanced_extension\030\n \001(\0132\'" - ".substrait.extensions.AdvancedExtension\032" - "\?\n\010Grouping\0223\n\024grouping_expressions\030\001 \003(" - "\0132\025.substrait.Expression\032_\n\007Measure\022-\n\007m" - "easure\030\001 \001(\0132\034.substrait.AggregateFuncti" - "on\022%\n\006filter\030\002 \001(\0132\025.substrait.Expressio" - "n\"\310\006\n\034ConsistentPartitionWindowRel\022$\n\006co" - "mmon\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005inpu" - "t\030\002 \001(\0132\016.substrait.Rel\022S\n\020window_functi" - "ons\030\003 \003(\01329.substrait.ConsistentPartitio" - "nWindowRel.WindowRelFunction\0224\n\025partitio" - "n_expressions\030\004 \003(\0132\025.substrait.Expressi" - "on\022#\n\005sorts\030\005 \003(\0132\024.substrait.SortField\022" - "C\n\022advanced_extension\030\n \001(\0132\'.substrait." - "extensions.AdvancedExtension\032\355\003\n\021WindowR" - "elFunction\022\032\n\022function_reference\030\001 \001(\r\022." - "\n\targuments\030\t \003(\0132\033.substrait.FunctionAr" - "gument\022*\n\007options\030\013 \003(\0132\031.substrait.Func" - "tionOption\022$\n\013output_type\030\007 \001(\0132\017.substr" - "ait.Type\022*\n\005phase\030\006 \001(\0162\033.substrait.Aggr" - "egationPhase\022F\n\ninvocation\030\n \001(\01622.subst" - "rait.AggregateFunction.AggregationInvoca" - "tion\022\?\n\013lower_bound\030\005 \001(\0132*.substrait.Ex" - "pression.WindowFunction.Bound\022\?\n\013upper_b" - "ound\030\004 \001(\0132*.substrait.Expression.Window" - "Function.Bound\022D\n\013bounds_type\030\014 \001(\0162/.su" - "bstrait.Expression.WindowFunction.Bounds" - "Type\"\270\001\n\007SortRel\022$\n\006common\030\001 \001(\0132\024.subst" - "rait.RelCommon\022\035\n\005input\030\002 \001(\0132\016.substrai" - "t.Rel\022#\n\005sorts\030\003 \003(\0132\024.substrait.SortFie" + "nput\030\002 \001(\0132\016.substrait.Rel\022S\n\020window_fun" + "ctions\030\003 \003(\01329.substrait.ConsistentParti" + "tionWindowRel.WindowRelFunction\0224\n\025parti" + "tion_expressions\030\004 \003(\0132\025.substrait.Expre" + "ssion\022#\n\005sorts\030\005 \003(\0132\024.substrait.SortFie" "ld\022C\n\022advanced_extension\030\n \001(\0132\'.substra" - "it.extensions.AdvancedExtension\"\277\001\n\tFilt" - "erRel\022$\n\006common\030\001 \001(\0132\024.substrait.RelCom" - "mon\022\035\n\005input\030\002 \001(\0132\016.substrait.Rel\022(\n\tco" - "ndition\030\003 \001(\0132\025.substrait.Expression\022C\n\022" - "advanced_extension\030\n \001(\0132\'.substrait.ext" - "ensions.AdvancedExtension\"\203\003\n\006SetRel\022$\n\006" - "common\030\001 \001(\0132\024.substrait.RelCommon\022\036\n\006in" - "puts\030\002 \003(\0132\016.substrait.Rel\022#\n\002op\030\003 \001(\0162\027" - ".substrait.SetRel.SetOp\022C\n\022advanced_exte" - "nsion\030\n \001(\0132\'.substrait.extensions.Advan" - "cedExtension\"\310\001\n\005SetOp\022\026\n\022SET_OP_UNSPECI" - "FIED\020\000\022\030\n\024SET_OP_MINUS_PRIMARY\020\001\022\031\n\025SET_" - "OP_MINUS_MULTISET\020\002\022\037\n\033SET_OP_INTERSECTI" - "ON_PRIMARY\020\003\022 \n\034SET_OP_INTERSECTION_MULT" - "ISET\020\004\022\031\n\025SET_OP_UNION_DISTINCT\020\005\022\024\n\020SET" - "_OP_UNION_ALL\020\006\"\177\n\022ExtensionSingleRel\022$\n" - "\006common\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005i" - "nput\030\002 \001(\0132\016.substrait.Rel\022$\n\006detail\030\003 \001" - "(\0132\024.google.protobuf.Any\"^\n\020ExtensionLea" - "fRel\022$\n\006common\030\001 \001(\0132\024.substrait.RelComm" - "on\022$\n\006detail\030\002 \001(\0132\024.google.protobuf.Any" - "\"\177\n\021ExtensionMultiRel\022$\n\006common\030\001 \001(\0132\024." - "substrait.RelCommon\022\036\n\006inputs\030\002 \003(\0132\016.su" - "bstrait.Rel\022$\n\006detail\030\003 \001(\0132\024.google.pro" - "tobuf.Any\"\303\007\n\013ExchangeRel\022$\n\006common\030\001 \001(" - "\0132\024.substrait.RelCommon\022\035\n\005input\030\002 \001(\0132\016" - ".substrait.Rel\022\027\n\017partition_count\030\003 \001(\005\022" - "6\n\007targets\030\004 \003(\0132%.substrait.ExchangeRel" - ".ExchangeTarget\022A\n\021scatter_by_fields\030\005 \001" - "(\0132$.substrait.ExchangeRel.ScatterFields" - "H\000\022F\n\rsingle_target\030\006 \001(\0132-.substrait.Ex" - "changeRel.SingleBucketExpressionH\000\022D\n\014mu" - "lti_target\030\007 \001(\0132,.substrait.ExchangeRel" - ".MultiBucketExpressionH\000\0228\n\013round_robin\030" - "\010 \001(\0132!.substrait.ExchangeRel.RoundRobin" - "H\000\0225\n\tbroadcast\030\t \001(\0132 .substrait.Exchan" - "geRel.BroadcastH\000\022C\n\022advanced_extension\030" - "\n \001(\0132\'.substrait.extensions.AdvancedExt" - "ension\032E\n\rScatterFields\0224\n\006fields\030\001 \003(\0132" - "$.substrait.Expression.FieldReference\032C\n" - "\026SingleBucketExpression\022)\n\nexpression\030\001 " - "\001(\0132\025.substrait.Expression\032`\n\025MultiBucke" - "tExpression\022)\n\nexpression\030\001 \001(\0132\025.substr" - "ait.Expression\022\034\n\024constrained_to_count\030\002" - " \001(\010\032\013\n\tBroadcast\032\033\n\nRoundRobin\022\r\n\005exact" - "\030\001 \001(\010\032n\n\016ExchangeTarget\022\024\n\014partition_id" - "\030\001 \003(\005\022\r\n\003uri\030\002 \001(\tH\000\022(\n\010extended\030\003 \001(\0132" - "\024.google.protobuf.AnyH\000B\r\n\013target_typeB\017" - "\n\rexchange_kind\"\320\002\n\tExpandRel\022$\n\006common\030" - "\001 \001(\0132\024.substrait.RelCommon\022\035\n\005input\030\002 \001" - "(\0132\016.substrait.Rel\0220\n\006fields\030\004 \003(\0132 .sub" - "strait.ExpandRel.ExpandField\032\216\001\n\013ExpandF" - "ield\022>\n\017switching_field\030\002 \001(\0132#.substrai" - "t.ExpandRel.SwitchingFieldH\000\0221\n\020consiste" - "nt_field\030\003 \001(\0132\025.substrait.ExpressionH\000B" - "\014\n\nfield_type\032;\n\016SwitchingField\022)\n\ndupli" - "cates\030\001 \003(\0132\025.substrait.Expression\"7\n\007Re" - "lRoot\022\035\n\005input\030\001 \001(\0132\016.substrait.Rel\022\r\n\005" - "names\030\002 \003(\t\"\225\010\n\003Rel\022\"\n\004read\030\001 \001(\0132\022.subs" - "trait.ReadRelH\000\022&\n\006filter\030\002 \001(\0132\024.substr" - "ait.FilterRelH\000\022$\n\005fetch\030\003 \001(\0132\023.substra" - "it.FetchRelH\000\022,\n\taggregate\030\004 \001(\0132\027.subst" - "rait.AggregateRelH\000\022\"\n\004sort\030\005 \001(\0132\022.subs" - "trait.SortRelH\000\022\"\n\004join\030\006 \001(\0132\022.substrai" - "t.JoinRelH\000\022(\n\007project\030\007 \001(\0132\025.substrait" - ".ProjectRelH\000\022 \n\003set\030\010 \001(\0132\021.substrait.S" - "etRelH\000\0229\n\020extension_single\030\t \001(\0132\035.subs" - "trait.ExtensionSingleRelH\000\0227\n\017extension_" - "multi\030\n \001(\0132\034.substrait.ExtensionMultiRe" - "lH\000\0225\n\016extension_leaf\030\013 \001(\0132\033.substrait." - "ExtensionLeafRelH\000\022$\n\005cross\030\014 \001(\0132\023.subs" - "trait.CrossRelH\000\022,\n\treference\030\025 \001(\0132\027.su" - "bstrait.ReferenceRelH\000\022$\n\005write\030\023 \001(\0132\023." - "substrait.WriteRelH\000\022 \n\003ddl\030\024 \001(\0132\021.subs" - "trait.DdlRelH\000\022+\n\thash_join\030\r \001(\0132\026.subs" - "trait.HashJoinRelH\000\022-\n\nmerge_join\030\016 \001(\0132" - "\027.substrait.MergeJoinRelH\000\0228\n\020nested_loo" - "p_join\030\022 \001(\0132\034.substrait.NestedLoopJoinR" - "elH\000\022+\n\tmark_join\030\027 \001(\0132\026.substrait.Mark" - "JoinRelH\000\0225\n\016delimiter_join\030\030 \001(\0132\033.subs" - "trait.DelimiterJoinRelH\000\0229\n\006window\030\021 \001(\013" - "2\'.substrait.ConsistentPartitionWindowRe" - "lH\000\022*\n\010exchange\030\017 \001(\0132\026.substrait.Exchan" - "geRelH\000\022&\n\006expand\030\020 \001(\0132\024.substrait.Expa" - "ndRelH\000B\n\n\010rel_type\"f\n\020NamedObjectWrite\022" - "\r\n\005names\030\001 \003(\t\022C\n\022advanced_extension\030\n \001" - "(\0132\'.substrait.extensions.AdvancedExtens" - "ion\"7\n\017ExtensionObject\022$\n\006detail\030\001 \001(\0132\024" - ".google.protobuf.Any\"\364\004\n\006DdlRel\0223\n\014named" - "_object\030\001 \001(\0132\033.substrait.NamedObjectWri" - "teH\000\0226\n\020extension_object\030\002 \001(\0132\032.substra" - "it.ExtensionObjectH\000\022,\n\014table_schema\030\003 \001" - "(\0132\026.substrait.NamedStruct\022<\n\016table_defa" - "ults\030\004 \001(\0132$.substrait.Expression.Litera" - "l.Struct\022+\n\006object\030\005 \001(\0162\033.substrait.Ddl" - "Rel.DdlObject\022#\n\002op\030\006 \001(\0162\027.substrait.Dd" - "lRel.DdlOp\022\'\n\017view_definition\030\007 \001(\0132\016.su" - "bstrait.Rel\022$\n\006common\030\010 \001(\0132\024.substrait." - "RelCommon\"R\n\tDdlObject\022\032\n\026DDL_OBJECT_UNS" - "PECIFIED\020\000\022\024\n\020DDL_OBJECT_TABLE\020\001\022\023\n\017DDL_" - "OBJECT_VIEW\020\002\"\215\001\n\005DdlOp\022\026\n\022DDL_OP_UNSPEC" - "IFIED\020\000\022\021\n\rDDL_OP_CREATE\020\001\022\034\n\030DDL_OP_CRE" - "ATE_OR_REPLACE\020\002\022\020\n\014DDL_OP_ALTER\020\003\022\017\n\013DD" - "L_OP_DROP\020\004\022\030\n\024DDL_OP_DROP_IF_EXIST\020\005B\014\n" - "\nwrite_type\"\256\004\n\010WriteRel\0222\n\013named_table\030" - "\001 \001(\0132\033.substrait.NamedObjectWriteH\000\0225\n\017" - "extension_table\030\002 \001(\0132\032.substrait.Extens" - "ionObjectH\000\022,\n\014table_schema\030\003 \001(\0132\026.subs" - "trait.NamedStruct\022\'\n\002op\030\004 \001(\0162\033.substrai" - "t.WriteRel.WriteOp\022\035\n\005input\030\005 \001(\0132\016.subs" - "trait.Rel\022.\n\006output\030\006 \001(\0162\036.substrait.Wr" - "iteRel.OutputMode\022$\n\006common\030\007 \001(\0132\024.subs" - "trait.RelCommon\"u\n\007WriteOp\022\030\n\024WRITE_OP_U" - "NSPECIFIED\020\000\022\023\n\017WRITE_OP_INSERT\020\001\022\023\n\017WRI" - "TE_OP_DELETE\020\002\022\023\n\017WRITE_OP_UPDATE\020\003\022\021\n\rW" - "RITE_OP_CTAS\020\004\"f\n\nOutputMode\022\033\n\027OUTPUT_M" - "ODE_UNSPECIFIED\020\000\022\031\n\025OUTPUT_MODE_NO_OUTP" - "UT\020\001\022 \n\034OUTPUT_MODE_MODIFIED_RECORDS\020\002B\014" - "\n\nwrite_type\"\201\004\n\021ComparisonJoinKey\0222\n\004le" - "ft\030\001 \001(\0132$.substrait.Expression.FieldRef" - "erence\0223\n\005right\030\002 \001(\0132$.substrait.Expres" - "sion.FieldReference\022\?\n\ncomparison\030\003 \001(\0132" - "+.substrait.ComparisonJoinKey.Comparison" - "Type\032\210\001\n\016ComparisonType\022C\n\006simple\030\001 \001(\0162" - "1.substrait.ComparisonJoinKey.SimpleComp" - "arisonTypeH\000\022#\n\031custom_function_referenc" - "e\030\002 \001(\rH\000B\014\n\ninner_type\"\266\001\n\024SimpleCompar" - "isonType\022&\n\"SIMPLE_COMPARISON_TYPE_UNSPE" - "CIFIED\020\000\022\035\n\031SIMPLE_COMPARISON_TYPE_EQ\020\001\022" - "/\n+SIMPLE_COMPARISON_TYPE_IS_NOT_DISTINC" - "T_FROM\020\002\022&\n\"SIMPLE_COMPARISON_TYPE_MIGHT" - "_EQUAL\020\003\"\235\005\n\013HashJoinRel\022$\n\006common\030\001 \001(\013" - "2\024.substrait.RelCommon\022\034\n\004left\030\002 \001(\0132\016.s" - "ubstrait.Rel\022\035\n\005right\030\003 \001(\0132\016.substrait." - "Rel\022;\n\tleft_keys\030\004 \003(\0132$.substrait.Expre" - "ssion.FieldReferenceB\002\030\001\022<\n\nright_keys\030\005" - " \003(\0132$.substrait.Expression.FieldReferen" - "ceB\002\030\001\022*\n\004keys\030\010 \003(\0132\034.substrait.Compari" - "sonJoinKey\022/\n\020post_join_filter\030\006 \001(\0132\025.s" - "ubstrait.Expression\022-\n\004type\030\007 \001(\0162\037.subs" - "trait.HashJoinRel.JoinType\022C\n\022advanced_e" + "it.extensions.AdvancedExtension\032\355\003\n\021Wind" + "owRelFunction\022\032\n\022function_reference\030\001 \001(" + "\r\022.\n\targuments\030\t \003(\0132\033.substrait.Functio" + "nArgument\022*\n\007options\030\013 \003(\0132\031.substrait.F" + "unctionOption\022$\n\013output_type\030\007 \001(\0132\017.sub" + "strait.Type\022*\n\005phase\030\006 \001(\0162\033.substrait.A" + "ggregationPhase\022F\n\ninvocation\030\n \001(\01622.su" + "bstrait.AggregateFunction.AggregationInv" + "ocation\022\?\n\013lower_bound\030\005 \001(\0132*.substrait" + ".Expression.WindowFunction.Bound\022\?\n\013uppe" + "r_bound\030\004 \001(\0132*.substrait.Expression.Win" + "dowFunction.Bound\022D\n\013bounds_type\030\014 \001(\0162/" + ".substrait.Expression.WindowFunction.Bou" + "ndsType\"\270\001\n\007SortRel\022$\n\006common\030\001 \001(\0132\024.su" + "bstrait.RelCommon\022\035\n\005input\030\002 \001(\0132\016.subst" + "rait.Rel\022#\n\005sorts\030\003 \003(\0132\024.substrait.Sort" + "Field\022C\n\022advanced_extension\030\n \001(\0132\'.subs" + "trait.extensions.AdvancedExtension\"\277\001\n\tF" + "ilterRel\022$\n\006common\030\001 \001(\0132\024.substrait.Rel" + "Common\022\035\n\005input\030\002 \001(\0132\016.substrait.Rel\022(\n" + "\tcondition\030\003 \001(\0132\025.substrait.Expression\022" + "C\n\022advanced_extension\030\n \001(\0132\'.substrait." + "extensions.AdvancedExtension\"\203\003\n\006SetRel\022" + "$\n\006common\030\001 \001(\0132\024.substrait.RelCommon\022\036\n" + "\006inputs\030\002 \003(\0132\016.substrait.Rel\022#\n\002op\030\003 \001(" + "\0162\027.substrait.SetRel.SetOp\022C\n\022advanced_e" "xtension\030\n \001(\0132\'.substrait.extensions.Ad" - "vancedExtension\"\336\001\n\010JoinType\022\031\n\025JOIN_TYP" - "E_UNSPECIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017" - "JOIN_TYPE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n" - "\017JOIN_TYPE_RIGHT\020\004\022\027\n\023JOIN_TYPE_LEFT_SEM" - "I\020\005\022\030\n\024JOIN_TYPE_RIGHT_SEMI\020\006\022\027\n\023JOIN_TY" - "PE_LEFT_ANTI\020\007\022\030\n\024JOIN_TYPE_RIGHT_ANTI\020\010" - "\"\237\005\n\014MergeJoinRel\022$\n\006common\030\001 \001(\0132\024.subs" - "trait.RelCommon\022\034\n\004left\030\002 \001(\0132\016.substrai" - "t.Rel\022\035\n\005right\030\003 \001(\0132\016.substrait.Rel\022;\n\t" - "left_keys\030\004 \003(\0132$.substrait.Expression.F" - "ieldReferenceB\002\030\001\022<\n\nright_keys\030\005 \003(\0132$." - "substrait.Expression.FieldReferenceB\002\030\001\022" - "*\n\004keys\030\010 \003(\0132\034.substrait.ComparisonJoin" - "Key\022/\n\020post_join_filter\030\006 \001(\0132\025.substrai" - "t.Expression\022.\n\004type\030\007 \001(\0162 .substrait.M" - "ergeJoinRel.JoinType\022C\n\022advanced_extensi" + "vancedExtension\"\310\001\n\005SetOp\022\026\n\022SET_OP_UNSP" + "ECIFIED\020\000\022\030\n\024SET_OP_MINUS_PRIMARY\020\001\022\031\n\025S" + "ET_OP_MINUS_MULTISET\020\002\022\037\n\033SET_OP_INTERSE" + "CTION_PRIMARY\020\003\022 \n\034SET_OP_INTERSECTION_M" + "ULTISET\020\004\022\031\n\025SET_OP_UNION_DISTINCT\020\005\022\024\n\020" + "SET_OP_UNION_ALL\020\006\"\177\n\022ExtensionSingleRel" + "\022$\n\006common\030\001 \001(\0132\024.substrait.RelCommon\022\035" + "\n\005input\030\002 \001(\0132\016.substrait.Rel\022$\n\006detail\030" + "\003 \001(\0132\024.google.protobuf.Any\"^\n\020Extension" + "LeafRel\022$\n\006common\030\001 \001(\0132\024.substrait.RelC" + "ommon\022$\n\006detail\030\002 \001(\0132\024.google.protobuf." + "Any\"\177\n\021ExtensionMultiRel\022$\n\006common\030\001 \001(\013" + "2\024.substrait.RelCommon\022\036\n\006inputs\030\002 \003(\0132\016" + ".substrait.Rel\022$\n\006detail\030\003 \001(\0132\024.google." + "protobuf.Any\"\303\007\n\013ExchangeRel\022$\n\006common\030\001" + " \001(\0132\024.substrait.RelCommon\022\035\n\005input\030\002 \001(" + "\0132\016.substrait.Rel\022\027\n\017partition_count\030\003 \001" + "(\005\0226\n\007targets\030\004 \003(\0132%.substrait.Exchange" + "Rel.ExchangeTarget\022A\n\021scatter_by_fields\030" + "\005 \001(\0132$.substrait.ExchangeRel.ScatterFie" + "ldsH\000\022F\n\rsingle_target\030\006 \001(\0132-.substrait" + ".ExchangeRel.SingleBucketExpressionH\000\022D\n" + "\014multi_target\030\007 \001(\0132,.substrait.Exchange" + "Rel.MultiBucketExpressionH\000\0228\n\013round_rob" + "in\030\010 \001(\0132!.substrait.ExchangeRel.RoundRo" + "binH\000\0225\n\tbroadcast\030\t \001(\0132 .substrait.Exc" + "hangeRel.BroadcastH\000\022C\n\022advanced_extensi" "on\030\n \001(\0132\'.substrait.extensions.Advanced" - "Extension\"\336\001\n\010JoinType\022\031\n\025JOIN_TYPE_UNSP" - "ECIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017JOIN_T" - "YPE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017JOIN_" - "TYPE_RIGHT\020\004\022\027\n\023JOIN_TYPE_LEFT_SEMI\020\005\022\030\n" - "\024JOIN_TYPE_RIGHT_SEMI\020\006\022\027\n\023JOIN_TYPE_LEF" - "T_ANTI\020\007\022\030\n\024JOIN_TYPE_RIGHT_ANTI\020\010\"\374\003\n\021N" - "estedLoopJoinRel\022$\n\006common\030\001 \001(\0132\024.subst" + "Extension\032E\n\rScatterFields\0224\n\006fields\030\001 \003" + "(\0132$.substrait.Expression.FieldReference" + "\032C\n\026SingleBucketExpression\022)\n\nexpression" + "\030\001 \001(\0132\025.substrait.Expression\032`\n\025MultiBu" + "cketExpression\022)\n\nexpression\030\001 \001(\0132\025.sub" + "strait.Expression\022\034\n\024constrained_to_coun" + "t\030\002 \001(\010\032\013\n\tBroadcast\032\033\n\nRoundRobin\022\r\n\005ex" + "act\030\001 \001(\010\032n\n\016ExchangeTarget\022\024\n\014partition" + "_id\030\001 \003(\005\022\r\n\003uri\030\002 \001(\tH\000\022(\n\010extended\030\003 \001" + "(\0132\024.google.protobuf.AnyH\000B\r\n\013target_typ" + "eB\017\n\rexchange_kind\"\320\002\n\tExpandRel\022$\n\006comm" + "on\030\001 \001(\0132\024.substrait.RelCommon\022\035\n\005input\030" + "\002 \001(\0132\016.substrait.Rel\0220\n\006fields\030\004 \003(\0132 ." + "substrait.ExpandRel.ExpandField\032\216\001\n\013Expa" + "ndField\022>\n\017switching_field\030\002 \001(\0132#.subst" + "rait.ExpandRel.SwitchingFieldH\000\0221\n\020consi" + "stent_field\030\003 \001(\0132\025.substrait.Expression" + "H\000B\014\n\nfield_type\032;\n\016SwitchingField\022)\n\ndu" + "plicates\030\001 \003(\0132\025.substrait.Expression\"7\n" + "\007RelRoot\022\035\n\005input\030\001 \001(\0132\016.substrait.Rel\022" + "\r\n\005names\030\002 \003(\t\"\307\010\n\003Rel\022\"\n\004read\030\001 \001(\0132\022.s" + "ubstrait.ReadRelH\000\022&\n\006filter\030\002 \001(\0132\024.sub" + "strait.FilterRelH\000\022$\n\005fetch\030\003 \001(\0132\023.subs" + "trait.FetchRelH\000\022,\n\taggregate\030\004 \001(\0132\027.su" + "bstrait.AggregateRelH\000\022\"\n\004sort\030\005 \001(\0132\022.s" + "ubstrait.SortRelH\000\022\"\n\004join\030\006 \001(\0132\022.subst" + "rait.JoinRelH\000\022(\n\007project\030\007 \001(\0132\025.substr" + "ait.ProjectRelH\000\022 \n\003set\030\010 \001(\0132\021.substrai" + "t.SetRelH\000\0229\n\020extension_single\030\t \001(\0132\035.s" + "ubstrait.ExtensionSingleRelH\000\0227\n\017extensi" + "on_multi\030\n \001(\0132\034.substrait.ExtensionMult" + "iRelH\000\0225\n\016extension_leaf\030\013 \001(\0132\033.substra" + "it.ExtensionLeafRelH\000\022$\n\005cross\030\014 \001(\0132\023.s" + "ubstrait.CrossRelH\000\022,\n\treference\030\025 \001(\0132\027" + ".substrait.ReferenceRelH\000\022$\n\005write\030\023 \001(\013" + "2\023.substrait.WriteRelH\000\022 \n\003ddl\030\024 \001(\0132\021.s" + "ubstrait.DdlRelH\000\022+\n\thash_join\030\r \001(\0132\026.s" + "ubstrait.HashJoinRelH\000\022-\n\nmerge_join\030\016 \001" + "(\0132\027.substrait.MergeJoinRelH\000\0228\n\020nested_" + "loop_join\030\022 \001(\0132\034.substrait.NestedLoopJo" + "inRelH\000\022H\n\030duplicate_eliminated_get\030\027 \001(" + "\0132$.substrait.DuplicateEliminatedGetRelH" + "\000\022J\n\031duplicate_eliminated_join\030\030 \001(\0132%.s" + "ubstrait.DuplicateEliminatedJoinRelH\000\0229\n" + "\006window\030\021 \001(\0132\'.substrait.ConsistentPart" + "itionWindowRelH\000\022*\n\010exchange\030\017 \001(\0132\026.sub" + "strait.ExchangeRelH\000\022&\n\006expand\030\020 \001(\0132\024.s" + "ubstrait.ExpandRelH\000B\n\n\010rel_type\"f\n\020Name" + "dObjectWrite\022\r\n\005names\030\001 \003(\t\022C\n\022advanced_" + "extension\030\n \001(\0132\'.substrait.extensions.A" + "dvancedExtension\"7\n\017ExtensionObject\022$\n\006d" + "etail\030\001 \001(\0132\024.google.protobuf.Any\"\364\004\n\006Dd" + "lRel\0223\n\014named_object\030\001 \001(\0132\033.substrait.N" + "amedObjectWriteH\000\0226\n\020extension_object\030\002 " + "\001(\0132\032.substrait.ExtensionObjectH\000\022,\n\014tab" + "le_schema\030\003 \001(\0132\026.substrait.NamedStruct\022" + "<\n\016table_defaults\030\004 \001(\0132$.substrait.Expr" + "ession.Literal.Struct\022+\n\006object\030\005 \001(\0162\033." + "substrait.DdlRel.DdlObject\022#\n\002op\030\006 \001(\0162\027" + ".substrait.DdlRel.DdlOp\022\'\n\017view_definiti" + "on\030\007 \001(\0132\016.substrait.Rel\022$\n\006common\030\010 \001(\013" + "2\024.substrait.RelCommon\"R\n\tDdlObject\022\032\n\026D" + "DL_OBJECT_UNSPECIFIED\020\000\022\024\n\020DDL_OBJECT_TA" + "BLE\020\001\022\023\n\017DDL_OBJECT_VIEW\020\002\"\215\001\n\005DdlOp\022\026\n\022" + "DDL_OP_UNSPECIFIED\020\000\022\021\n\rDDL_OP_CREATE\020\001\022" + "\034\n\030DDL_OP_CREATE_OR_REPLACE\020\002\022\020\n\014DDL_OP_" + "ALTER\020\003\022\017\n\013DDL_OP_DROP\020\004\022\030\n\024DDL_OP_DROP_" + "IF_EXIST\020\005B\014\n\nwrite_type\"\256\004\n\010WriteRel\0222\n" + "\013named_table\030\001 \001(\0132\033.substrait.NamedObje" + "ctWriteH\000\0225\n\017extension_table\030\002 \001(\0132\032.sub" + "strait.ExtensionObjectH\000\022,\n\014table_schema" + "\030\003 \001(\0132\026.substrait.NamedStruct\022\'\n\002op\030\004 \001" + "(\0162\033.substrait.WriteRel.WriteOp\022\035\n\005input" + "\030\005 \001(\0132\016.substrait.Rel\022.\n\006output\030\006 \001(\0162\036" + ".substrait.WriteRel.OutputMode\022$\n\006common" + "\030\007 \001(\0132\024.substrait.RelCommon\"u\n\007WriteOp\022" + "\030\n\024WRITE_OP_UNSPECIFIED\020\000\022\023\n\017WRITE_OP_IN" + "SERT\020\001\022\023\n\017WRITE_OP_DELETE\020\002\022\023\n\017WRITE_OP_" + "UPDATE\020\003\022\021\n\rWRITE_OP_CTAS\020\004\"f\n\nOutputMod" + "e\022\033\n\027OUTPUT_MODE_UNSPECIFIED\020\000\022\031\n\025OUTPUT" + "_MODE_NO_OUTPUT\020\001\022 \n\034OUTPUT_MODE_MODIFIE" + "D_RECORDS\020\002B\014\n\nwrite_type\"\201\004\n\021Comparison" + "JoinKey\0222\n\004left\030\001 \001(\0132$.substrait.Expres" + "sion.FieldReference\0223\n\005right\030\002 \001(\0132$.sub" + "strait.Expression.FieldReference\022\?\n\ncomp" + "arison\030\003 \001(\0132+.substrait.ComparisonJoinK" + "ey.ComparisonType\032\210\001\n\016ComparisonType\022C\n\006" + "simple\030\001 \001(\01621.substrait.ComparisonJoinK" + "ey.SimpleComparisonTypeH\000\022#\n\031custom_func" + "tion_reference\030\002 \001(\rH\000B\014\n\ninner_type\"\266\001\n" + "\024SimpleComparisonType\022&\n\"SIMPLE_COMPARIS" + "ON_TYPE_UNSPECIFIED\020\000\022\035\n\031SIMPLE_COMPARIS" + "ON_TYPE_EQ\020\001\022/\n+SIMPLE_COMPARISON_TYPE_I" + "S_NOT_DISTINCT_FROM\020\002\022&\n\"SIMPLE_COMPARIS" + "ON_TYPE_MIGHT_EQUAL\020\003\"\207\006\n\013HashJoinRel\022$\n" + "\006common\030\001 \001(\0132\024.substrait.RelCommon\022\034\n\004l" + "eft\030\002 \001(\0132\016.substrait.Rel\022\035\n\005right\030\003 \001(\013" + "2\016.substrait.Rel\022;\n\tleft_keys\030\004 \003(\0132$.su" + "bstrait.Expression.FieldReferenceB\002\030\001\022<\n" + "\nright_keys\030\005 \003(\0132$.substrait.Expression" + ".FieldReferenceB\002\030\001\022*\n\004keys\030\010 \003(\0132\034.subs" + "trait.ComparisonJoinKey\022/\n\020post_join_fil" + "ter\030\006 \001(\0132\025.substrait.Expression\022-\n\004type" + "\030\007 \001(\0162\037.substrait.HashJoinRel.JoinType\022" + "C\n\022advanced_extension\030\n \001(\0132\'.substrait." + "extensions.AdvancedExtension\"\310\002\n\010JoinTyp" + "e\022\031\n\025JOIN_TYPE_UNSPECIFIED\020\000\022\023\n\017JOIN_TYP" + "E_INNER\020\001\022\023\n\017JOIN_TYPE_OUTER\020\002\022\022\n\016JOIN_T" + "YPE_LEFT\020\003\022\023\n\017JOIN_TYPE_RIGHT\020\004\022\027\n\023JOIN_" + "TYPE_LEFT_SEMI\020\005\022\030\n\024JOIN_TYPE_RIGHT_SEMI" + "\020\006\022\027\n\023JOIN_TYPE_LEFT_ANTI\020\007\022\030\n\024JOIN_TYPE" + "_RIGHT_ANTI\020\010\022\031\n\025JOIN_TYPE_LEFT_SINGLE\020\t" + "\022\032\n\026JOIN_TYPE_RIGHT_SINGLE\020\n\022\027\n\023JOIN_TYP" + "E_LEFT_MARK\020\013\022\030\n\024JOIN_TYPE_RIGHT_MARK\020\014\"" + "\211\006\n\014MergeJoinRel\022$\n\006common\030\001 \001(\0132\024.subst" "rait.RelCommon\022\034\n\004left\030\002 \001(\0132\016.substrait" - ".Rel\022\035\n\005right\030\003 \001(\0132\016.substrait.Rel\022)\n\ne" - "xpression\030\004 \001(\0132\025.substrait.Expression\0223" - "\n\004type\030\005 \001(\0162%.substrait.NestedLoopJoinR" - "el.JoinType\022C\n\022advanced_extension\030\n \001(\0132" - "\'.substrait.extensions.AdvancedExtension" - "\"\336\001\n\010JoinType\022\031\n\025JOIN_TYPE_UNSPECIFIED\020\000" - "\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017JOIN_TYPE_OUTER" - "\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017JOIN_TYPE_RIGH" - "T\020\004\022\027\n\023JOIN_TYPE_LEFT_SEMI\020\005\022\030\n\024JOIN_TYP" - "E_RIGHT_SEMI\020\006\022\027\n\023JOIN_TYPE_LEFT_ANTI\020\007\022" - "\030\n\024JOIN_TYPE_RIGHT_ANTI\020\010\"\340\001\n\013MarkJoinRe" + ".Rel\022\035\n\005right\030\003 \001(\0132\016.substrait.Rel\022;\n\tl" + "eft_keys\030\004 \003(\0132$.substrait.Expression.Fi" + "eldReferenceB\002\030\001\022<\n\nright_keys\030\005 \003(\0132$.s" + "ubstrait.Expression.FieldReferenceB\002\030\001\022*" + "\n\004keys\030\010 \003(\0132\034.substrait.ComparisonJoinK" + "ey\022/\n\020post_join_filter\030\006 \001(\0132\025.substrait" + ".Expression\022.\n\004type\030\007 \001(\0162 .substrait.Me" + "rgeJoinRel.JoinType\022C\n\022advanced_extensio" + "n\030\n \001(\0132\'.substrait.extensions.AdvancedE" + "xtension\"\310\002\n\010JoinType\022\031\n\025JOIN_TYPE_UNSPE" + "CIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017JOIN_TY" + "PE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017JOIN_T" + "YPE_RIGHT\020\004\022\027\n\023JOIN_TYPE_LEFT_SEMI\020\005\022\030\n\024" + "JOIN_TYPE_RIGHT_SEMI\020\006\022\027\n\023JOIN_TYPE_LEFT" + "_ANTI\020\007\022\030\n\024JOIN_TYPE_RIGHT_ANTI\020\010\022\031\n\025JOI" + "N_TYPE_LEFT_SINGLE\020\t\022\032\n\026JOIN_TYPE_RIGHT_" + "SINGLE\020\n\022\027\n\023JOIN_TYPE_LEFT_MARK\020\013\022\030\n\024JOI" + "N_TYPE_RIGHT_MARK\020\014\"\346\004\n\021NestedLoopJoinRe" "l\022$\n\006common\030\001 \001(\0132\024.substrait.RelCommon\022" "\034\n\004left\030\002 \001(\0132\016.substrait.Rel\022\035\n\005right\030\003" " \001(\0132\016.substrait.Rel\022)\n\nexpression\030\004 \001(\013" - "2\025.substrait.Expression\022C\n\022advanced_exte" - "nsion\030\n \001(\0132\'.substrait.extensions.Advan" - "cedExtension\"\201\005\n\020DelimiterJoinRel\022$\n\006com" - "mon\030\001 \001(\0132\024.substrait.RelCommon\022\034\n\004left\030" - "\002 \001(\0132\016.substrait.Rel\022\035\n\005right\030\003 \001(\0132\016.s" - "ubstrait.Rel\0227\n\tleft_keys\030\004 \003(\0132$.substr" - "ait.Expression.FieldReference\0228\n\nright_k" - "eys\030\005 \003(\0132$.substrait.Expression.FieldRe" - "ference\0222\n\004type\030\006 \001(\0162$.substrait.Delimi" - "terJoinRel.JoinType\022=\n\017delimiter_field\030\007" - " \001(\0132$.substrait.Expression.FieldReferen" - "ce\022C\n\022advanced_extension\030\n \001(\0132\'.substra" - "it.extensions.AdvancedExtension\"\336\001\n\010Join" - "Type\022\031\n\025JOIN_TYPE_UNSPECIFIED\020\000\022\023\n\017JOIN_" - "TYPE_INNER\020\001\022\023\n\017JOIN_TYPE_OUTER\020\002\022\022\n\016JOI" - "N_TYPE_LEFT\020\003\022\023\n\017JOIN_TYPE_RIGHT\020\004\022\027\n\023JO" - "IN_TYPE_LEFT_SEMI\020\005\022\030\n\024JOIN_TYPE_RIGHT_S" - "EMI\020\006\022\027\n\023JOIN_TYPE_LEFT_ANTI\020\007\022\030\n\024JOIN_T" - "YPE_RIGHT_ANTI\020\010\"w\n\020FunctionArgument\022\016\n\004" - "enum\030\001 \001(\tH\000\022\037\n\004type\030\002 \001(\0132\017.substrait.T" - "ypeH\000\022&\n\005value\030\003 \001(\0132\025.substrait.Express" - "ionH\000B\n\n\010arg_type\"2\n\016FunctionOption\022\014\n\004n" - "ame\030\001 \001(\t\022\022\n\npreference\030\002 \003(\t\"\371G\n\nExpres" - "sion\0220\n\007literal\030\001 \001(\0132\035.substrait.Expres" - "sion.LiteralH\000\0229\n\tselection\030\002 \001(\0132$.subs" - "trait.Expression.FieldReferenceH\000\022\?\n\017sca" - "lar_function\030\003 \001(\0132$.substrait.Expressio" - "n.ScalarFunctionH\000\022\?\n\017window_function\030\005 " - "\001(\0132$.substrait.Expression.WindowFunctio" - "nH\000\022/\n\007if_then\030\006 \001(\0132\034.substrait.Express" - "ion.IfThenH\000\022C\n\021switch_expression\030\007 \001(\0132" - "&.substrait.Expression.SwitchExpressionH" - "\000\022@\n\020singular_or_list\030\010 \001(\0132$.substrait." - "Expression.SingularOrListH\000\022:\n\rmulti_or_" - "list\030\t \001(\0132!.substrait.Expression.MultiO" - "rListH\000\022*\n\004cast\030\013 \001(\0132\032.substrait.Expres" - "sion.CastH\000\0222\n\010subquery\030\014 \001(\0132\036.substrai" - "t.Expression.SubqueryH\000\022.\n\006nested\030\r \001(\0132" - "\034.substrait.Expression.NestedH\000\022.\n\004enum\030" - "\n \001(\0132\032.substrait.Expression.EnumB\002\030\001H\000\032" - "r\n\004Enum\022\023\n\tspecified\030\001 \001(\tH\000\0227\n\013unspecif" - "ied\030\002 \001(\0132 .substrait.Expression.Enum.Em" - "ptyH\000\032\013\n\005Empty:\002\030\001:\002\030\001B\013\n\tenum_kind\032\362\r\n\007" - "Literal\022\021\n\007boolean\030\001 \001(\010H\000\022\014\n\002i8\030\002 \001(\005H\000" - "\022\r\n\003i16\030\003 \001(\005H\000\022\r\n\003i32\030\005 \001(\005H\000\022\r\n\003i64\030\007 " - "\001(\003H\000\022\016\n\004fp32\030\n \001(\002H\000\022\016\n\004fp64\030\013 \001(\001H\000\022\020\n" - "\006string\030\014 \001(\tH\000\022\020\n\006binary\030\r \001(\014H\000\022\027\n\ttim" - "estamp\030\016 \001(\003B\002\030\001H\000\022\016\n\004date\030\020 \001(\005H\000\022\016\n\004ti" - "me\030\021 \001(\003H\000\022S\n\026interval_year_to_month\030\023 \001" - "(\01321.substrait.Expression.Literal.Interv" - "alYearToMonthH\000\022S\n\026interval_day_to_secon" - "d\030\024 \001(\01321.substrait.Expression.Literal.I" - "ntervalDayToSecondH\000\022\024\n\nfixed_char\030\025 \001(\t" - "H\000\0229\n\010var_char\030\026 \001(\0132%.substrait.Express" - "ion.Literal.VarCharH\000\022\026\n\014fixed_binary\030\027 " - "\001(\014H\000\0228\n\007decimal\030\030 \001(\0132%.substrait.Expre" - "ssion.Literal.DecimalH\000\022\035\n\023precision_tim" - "estamp\030\" \001(\004H\000\022 \n\026precision_timestamp_tz" - "\030# \001(\004H\000\0226\n\006struct\030\031 \001(\0132$.substrait.Exp" - "ression.Literal.StructH\000\0220\n\003map\030\032 \001(\0132!." - "substrait.Expression.Literal.MapH\000\022\032\n\014ti" - "mestamp_tz\030\033 \001(\003B\002\030\001H\000\022\016\n\004uuid\030\034 \001(\014H\000\022\037" - "\n\004null\030\035 \001(\0132\017.substrait.TypeH\000\0222\n\004list\030" - "\036 \001(\0132\".substrait.Expression.Literal.Lis" - "tH\000\022*\n\nempty_list\030\037 \001(\0132\024.substrait.Type" - ".ListH\000\022(\n\tempty_map\030 \001(\0132\023.substrait.T" - "ype.MapH\000\022A\n\014user_defined\030! \001(\0132).substr" - "ait.Expression.Literal.UserDefinedH\000\022\020\n\010" - "nullable\0302 \001(\010\022 \n\030type_variation_referen" - "ce\0303 \001(\r\032(\n\007VarChar\022\r\n\005value\030\001 \001(\t\022\016\n\006le" - "ngth\030\002 \001(\r\032:\n\007Decimal\022\r\n\005value\030\001 \001(\014\022\021\n\t" - "precision\030\002 \001(\005\022\r\n\005scale\030\003 \001(\005\032\253\001\n\003Map\022>" - "\n\nkey_values\030\001 \003(\0132*.substrait.Expressio" - "n.Literal.Map.KeyValue\032d\n\010KeyValue\022*\n\003ke" - "y\030\001 \001(\0132\035.substrait.Expression.Literal\022," - "\n\005value\030\002 \001(\0132\035.substrait.Expression.Lit" - "eral\0324\n\023IntervalYearToMonth\022\r\n\005years\030\001 \001" - "(\005\022\016\n\006months\030\002 \001(\005\032J\n\023IntervalDayToSecon" - "d\022\014\n\004days\030\001 \001(\005\022\017\n\007seconds\030\002 \001(\005\022\024\n\014micr" - "oseconds\030\003 \001(\005\0327\n\006Struct\022-\n\006fields\030\001 \003(\013" - "2\035.substrait.Expression.Literal\0325\n\004List\022" - "-\n\006values\030\001 \003(\0132\035.substrait.Expression.L" - "iteral\032\277\001\n\013UserDefined\022\026\n\016type_reference" - "\030\001 \001(\r\0222\n\017type_parameters\030\003 \003(\0132\031.substr" - "ait.Type.Parameter\022%\n\005value\030\002 \001(\0132\024.goog" - "le.protobuf.AnyH\000\0226\n\006struct\030\004 \001(\0132$.subs" - "trait.Expression.Literal.StructH\000B\005\n\003val" - "B\016\n\014literal_type\032\343\003\n\006Nested\022\020\n\010nullable\030" - "\001 \001(\010\022 \n\030type_variation_reference\030\002 \001(\r\022" - "5\n\006struct\030\003 \001(\0132#.substrait.Expression.N" - "ested.StructH\000\0221\n\004list\030\004 \001(\0132!.substrait" - ".Expression.Nested.ListH\000\022/\n\003map\030\005 \001(\0132 " - ".substrait.Expression.Nested.MapH\000\032\232\001\n\003M" - "ap\022=\n\nkey_values\030\001 \003(\0132).substrait.Expre" - "ssion.Nested.Map.KeyValue\032T\n\010KeyValue\022\"\n" - "\003key\030\001 \001(\0132\025.substrait.Expression\022$\n\005val" - "ue\030\002 \001(\0132\025.substrait.Expression\032/\n\006Struc" - "t\022%\n\006fields\030\001 \003(\0132\025.substrait.Expression" - "\032-\n\004List\022%\n\006values\030\001 \003(\0132\025.substrait.Exp" - "ressionB\r\n\013nested_type\032\327\001\n\016ScalarFunctio" - "n\022\032\n\022function_reference\030\001 \001(\r\022.\n\targumen" - "ts\030\004 \003(\0132\033.substrait.FunctionArgument\022*\n" - "\007options\030\005 \003(\0132\031.substrait.FunctionOptio" - "n\022$\n\013output_type\030\003 \001(\0132\017.substrait.Type\022" - "\'\n\004args\030\002 \003(\0132\025.substrait.ExpressionB\002\030\001" - "\032\321\010\n\016WindowFunction\022\032\n\022function_referenc" - "e\030\001 \001(\r\022.\n\targuments\030\t \003(\0132\033.substrait.F" - "unctionArgument\022*\n\007options\030\013 \003(\0132\031.subst" - "rait.FunctionOption\022$\n\013output_type\030\007 \001(\013" - "2\017.substrait.Type\022*\n\005phase\030\006 \001(\0162\033.subst" - "rait.AggregationPhase\022#\n\005sorts\030\003 \003(\0132\024.s" - "ubstrait.SortField\022F\n\ninvocation\030\n \001(\01622" - ".substrait.AggregateFunction.Aggregation" - "Invocation\022)\n\npartitions\030\002 \003(\0132\025.substra" - "it.Expression\022D\n\013bounds_type\030\014 \001(\0162/.sub" - "strait.Expression.WindowFunction.BoundsT" - "ype\022\?\n\013lower_bound\030\005 \001(\0132*.substrait.Exp" - "ression.WindowFunction.Bound\022\?\n\013upper_bo" - "und\030\004 \001(\0132*.substrait.Expression.WindowF" - "unction.Bound\022\'\n\004args\030\010 \003(\0132\025.substrait." - "ExpressionB\002\030\001\032\223\003\n\005Bound\022I\n\tpreceding\030\001 " - "\001(\01324.substrait.Expression.WindowFunctio" - "n.Bound.PrecedingH\000\022I\n\tfollowing\030\002 \001(\01324" - ".substrait.Expression.WindowFunction.Bou" - "nd.FollowingH\000\022L\n\013current_row\030\003 \001(\01325.su" - "bstrait.Expression.WindowFunction.Bound." - "CurrentRowH\000\022I\n\tunbounded\030\004 \001(\01324.substr" - "ait.Expression.WindowFunction.Bound.Unbo" - "undedH\000\032\033\n\tPreceding\022\016\n\006offset\030\001 \001(\003\032\033\n\t" - "Following\022\016\n\006offset\030\001 \001(\003\032\014\n\nCurrentRow\032" - "\013\n\tUnboundedB\006\n\004kind\"V\n\nBoundsType\022\033\n\027BO" - "UNDS_TYPE_UNSPECIFIED\020\000\022\024\n\020BOUNDS_TYPE_R" - "OWS\020\001\022\025\n\021BOUNDS_TYPE_RANGE\020\002\032\265\001\n\006IfThen\022" - "2\n\003ifs\030\001 \003(\0132%.substrait.Expression.IfTh" - "en.IfClause\022#\n\004else\030\002 \001(\0132\025.substrait.Ex" - "pression\032R\n\010IfClause\022!\n\002if\030\001 \001(\0132\025.subst" - "rait.Expression\022#\n\004then\030\002 \001(\0132\025.substrai" - "t.Expression\032\216\002\n\004Cast\022\035\n\004type\030\001 \001(\0132\017.su" - "bstrait.Type\022$\n\005input\030\002 \001(\0132\025.substrait." - "Expression\022D\n\020failure_behavior\030\003 \001(\0162*.s" - "ubstrait.Expression.Cast.FailureBehavior" - "\"{\n\017FailureBehavior\022 \n\034FAILURE_BEHAVIOR_" - "UNSPECIFIED\020\000\022 \n\034FAILURE_BEHAVIOR_RETURN" - "_NULL\020\001\022$\n FAILURE_BEHAVIOR_THROW_EXCEPT" - "ION\020\002\032\365\001\n\020SwitchExpression\022$\n\005match\030\003 \001(" - "\0132\025.substrait.Expression\022;\n\003ifs\030\001 \003(\0132.." - "substrait.Expression.SwitchExpression.If" - "Value\022#\n\004else\030\002 \001(\0132\025.substrait.Expressi" - "on\032Y\n\007IfValue\022)\n\002if\030\001 \001(\0132\035.substrait.Ex" - "pression.Literal\022#\n\004then\030\002 \001(\0132\025.substra" - "it.Expression\032^\n\016SingularOrList\022$\n\005value" - "\030\001 \001(\0132\025.substrait.Expression\022&\n\007options" - "\030\002 \003(\0132\025.substrait.Expression\032\237\001\n\013MultiO" - "rList\022$\n\005value\030\001 \003(\0132\025.substrait.Express" - "ion\0229\n\007options\030\002 \003(\0132(.substrait.Express" - "ion.MultiOrList.Record\032/\n\006Record\022%\n\006fiel" - "ds\030\001 \003(\0132\025.substrait.Expression\032\243\003\n\020Embe" - "ddedFunction\022(\n\targuments\030\001 \003(\0132\025.substr" - "ait.Expression\022$\n\013output_type\030\002 \001(\0132\017.su" - "bstrait.Type\022]\n\026python_pickle_function\030\003" - " \001(\0132;.substrait.Expression.EmbeddedFunc" - "tion.PythonPickleFunctionH\000\022[\n\025web_assem" - "bly_function\030\004 \001(\0132:.substrait.Expressio" - "n.EmbeddedFunction.WebAssemblyFunctionH\000" - "\032>\n\024PythonPickleFunction\022\020\n\010function\030\001 \001" - "(\014\022\024\n\014prerequisite\030\002 \003(\t\032;\n\023WebAssemblyF" - "unction\022\016\n\006script\030\001 \001(\014\022\024\n\014prerequisite\030" - "\002 \003(\tB\006\n\004kind\032\232\004\n\020ReferenceSegment\022@\n\007ma" - "p_key\030\001 \001(\0132-.substrait.Expression.Refer" - "enceSegment.MapKeyH\000\022J\n\014struct_field\030\002 \001" - "(\01322.substrait.Expression.ReferenceSegme" - "nt.StructFieldH\000\022J\n\014list_element\030\003 \001(\01322" - ".substrait.Expression.ReferenceSegment.L" - "istElementH\000\032o\n\006MapKey\022.\n\007map_key\030\001 \001(\0132" - "\035.substrait.Expression.Literal\0225\n\005child\030" - "\002 \001(\0132&.substrait.Expression.ReferenceSe" - "gment\032S\n\013StructField\022\r\n\005field\030\001 \001(\005\0225\n\005c" - "hild\030\002 \001(\0132&.substrait.Expression.Refere" - "nceSegment\032T\n\013ListElement\022\016\n\006offset\030\001 \001(" - "\005\0225\n\005child\030\002 \001(\0132&.substrait.Expression." - "ReferenceSegmentB\020\n\016reference_type\032\360\t\n\016M" - "askExpression\022A\n\006select\030\001 \001(\01321.substrai" - "t.Expression.MaskExpression.StructSelect" - "\022 \n\030maintain_singular_struct\030\002 \001(\010\032\325\001\n\006S" - "elect\022C\n\006struct\030\001 \001(\01321.substrait.Expres" - "sion.MaskExpression.StructSelectH\000\022\?\n\004li" - "st\030\002 \001(\0132/.substrait.Expression.MaskExpr" - "ession.ListSelectH\000\022=\n\003map\030\003 \001(\0132..subst" - "rait.Expression.MaskExpression.MapSelect" - "H\000B\006\n\004type\032U\n\014StructSelect\022E\n\014struct_ite" - "ms\030\001 \003(\0132/.substrait.Expression.MaskExpr" - "ession.StructItem\032W\n\nStructItem\022\r\n\005field" - "\030\001 \001(\005\022:\n\005child\030\002 \001(\0132+.substrait.Expres" - "sion.MaskExpression.Select\032\264\003\n\nListSelec" - "t\022Q\n\tselection\030\001 \003(\0132>.substrait.Express" - "ion.MaskExpression.ListSelect.ListSelect" - "Item\022:\n\005child\030\002 \001(\0132+.substrait.Expressi" - "on.MaskExpression.Select\032\226\002\n\016ListSelectI" - "tem\022Z\n\004item\030\001 \001(\0132J.substrait.Expression" - ".MaskExpression.ListSelect.ListSelectIte" - "m.ListElementH\000\022Y\n\005slice\030\002 \001(\0132H.substra" - "it.Expression.MaskExpression.ListSelect." - "ListSelectItem.ListSliceH\000\032\034\n\013ListElemen" - "t\022\r\n\005field\030\001 \001(\005\032\'\n\tListSlice\022\r\n\005start\030\001" - " \001(\005\022\013\n\003end\030\002 \001(\005B\006\n\004type\032\271\002\n\tMapSelect\022" - "D\n\003key\030\001 \001(\01325.substrait.Expression.Mask" - "Expression.MapSelect.MapKeyH\000\022U\n\nexpress" - "ion\030\002 \001(\0132\?.substrait.Expression.MaskExp" - "ression.MapSelect.MapKeyExpressionH\000\022:\n\005" - "child\030\003 \001(\0132+.substrait.Expression.MaskE" - "xpression.Select\032\031\n\006MapKey\022\017\n\007map_key\030\001 " - "\001(\t\032.\n\020MapKeyExpression\022\032\n\022map_key_expre" - "ssion\030\001 \001(\tB\010\n\006select\032\266\003\n\016FieldReference" - "\022B\n\020direct_reference\030\001 \001(\0132&.substrait.E" - "xpression.ReferenceSegmentH\000\022@\n\020masked_r" - "eference\030\002 \001(\0132$.substrait.Expression.Ma" - "skExpressionH\000\022+\n\nexpression\030\003 \001(\0132\025.sub" - "strait.ExpressionH\001\022L\n\016root_reference\030\004 " - "\001(\01322.substrait.Expression.FieldReferenc" - "e.RootReferenceH\001\022N\n\017outer_reference\030\005 \001" - "(\01323.substrait.Expression.FieldReference" - ".OuterReferenceH\001\032\017\n\rRootReference\032#\n\016Ou" - "terReference\022\021\n\tsteps_out\030\001 \001(\rB\020\n\016refer" - "ence_typeB\013\n\troot_type\032\214\t\n\010Subquery\0227\n\006s" - "calar\030\001 \001(\0132%.substrait.Expression.Subqu" - "ery.ScalarH\000\022B\n\014in_predicate\030\002 \001(\0132*.sub" - "strait.Expression.Subquery.InPredicateH\000" - "\022D\n\rset_predicate\030\003 \001(\0132+.substrait.Expr" - "ession.Subquery.SetPredicateH\000\022F\n\016set_co" - "mparison\030\004 \001(\0132,.substrait.Expression.Su" - "bquery.SetComparisonH\000\032\'\n\006Scalar\022\035\n\005inpu" - "t\030\001 \001(\0132\016.substrait.Rel\032W\n\013InPredicate\022&" - "\n\007needles\030\001 \003(\0132\025.substrait.Expression\022 " - "\n\010haystack\030\002 \001(\0132\016.substrait.Rel\032\334\001\n\014Set" - "Predicate\022M\n\014predicate_op\030\001 \001(\01627.substr" - "ait.Expression.Subquery.SetPredicate.Pre" - "dicateOp\022\036\n\006tuples\030\002 \001(\0132\016.substrait.Rel" - "\"]\n\013PredicateOp\022\034\n\030PREDICATE_OP_UNSPECIF" - "IED\020\000\022\027\n\023PREDICATE_OP_EXISTS\020\001\022\027\n\023PREDIC" - "ATE_OP_UNIQUE\020\002\032\202\004\n\rSetComparison\022N\n\014red" - "uction_op\030\001 \001(\01628.substrait.Expression.S" - "ubquery.SetComparison.ReductionOp\022P\n\rcom" - "parison_op\030\002 \001(\01629.substrait.Expression." - "Subquery.SetComparison.ComparisonOp\022#\n\004l" - "eft\030\003 \001(\0132\025.substrait.Expression\022\035\n\005righ" - "t\030\004 \001(\0132\016.substrait.Rel\"\261\001\n\014ComparisonOp" - "\022\035\n\031COMPARISON_OP_UNSPECIFIED\020\000\022\024\n\020COMPA" - "RISON_OP_EQ\020\001\022\024\n\020COMPARISON_OP_NE\020\002\022\024\n\020C" - "OMPARISON_OP_LT\020\003\022\024\n\020COMPARISON_OP_GT\020\004\022" - "\024\n\020COMPARISON_OP_LE\020\005\022\024\n\020COMPARISON_OP_G" - "E\020\006\"W\n\013ReductionOp\022\034\n\030REDUCTION_OP_UNSPE" - "CIFIED\020\000\022\024\n\020REDUCTION_OP_ANY\020\001\022\024\n\020REDUCT" - "ION_OP_ALL\020\002B\017\n\rsubquery_typeB\n\n\010rex_typ" - "e\"\377\002\n\tSortField\022#\n\004expr\030\001 \001(\0132\025.substrai" - "t.Expression\0227\n\tdirection\030\002 \001(\0162\".substr" - "ait.SortField.SortDirectionH\000\022\'\n\035compari" - "son_function_reference\030\003 \001(\rH\000\"\335\001\n\rSortD" - "irection\022\036\n\032SORT_DIRECTION_UNSPECIFIED\020\000" - "\022\"\n\036SORT_DIRECTION_ASC_NULLS_FIRST\020\001\022!\n\035" - "SORT_DIRECTION_ASC_NULLS_LAST\020\002\022#\n\037SORT_" - "DIRECTION_DESC_NULLS_FIRST\020\003\022\"\n\036SORT_DIR" - "ECTION_DESC_NULLS_LAST\020\004\022\034\n\030SORT_DIRECTI" - "ON_CLUSTERED\020\005B\013\n\tsort_kind\"\372\003\n\021Aggregat" - "eFunction\022\032\n\022function_reference\030\001 \001(\r\022.\n" - "\targuments\030\007 \003(\0132\033.substrait.FunctionArg" - "ument\022*\n\007options\030\010 \003(\0132\031.substrait.Funct" - "ionOption\022$\n\013output_type\030\005 \001(\0132\017.substra" - "it.Type\022*\n\005phase\030\004 \001(\0162\033.substrait.Aggre" - "gationPhase\022#\n\005sorts\030\003 \003(\0132\024.substrait.S" - "ortField\022F\n\ninvocation\030\006 \001(\01622.substrait" - ".AggregateFunction.AggregationInvocation" - "\022\'\n\004args\030\002 \003(\0132\025.substrait.ExpressionB\002\030" - "\001\"\204\001\n\025AggregationInvocation\022&\n\"AGGREGATI" - "ON_INVOCATION_UNSPECIFIED\020\000\022\036\n\032AGGREGATI" - "ON_INVOCATION_ALL\020\001\022#\n\037AGGREGATION_INVOC" - "ATION_DISTINCT\020\002\"\'\n\014ReferenceRel\022\027\n\017subt" - "ree_ordinal\030\001 \001(\005*\357\001\n\020AggregationPhase\022!" - "\n\035AGGREGATION_PHASE_UNSPECIFIED\020\000\022-\n)AGG" - "REGATION_PHASE_INITIAL_TO_INTERMEDIATE\020\001" - "\0222\n.AGGREGATION_PHASE_INTERMEDIATE_TO_IN" - "TERMEDIATE\020\002\022\'\n#AGGREGATION_PHASE_INITIA" - "L_TO_RESULT\020\003\022,\n(AGGREGATION_PHASE_INTER" - "MEDIATE_TO_RESULT\020\004BW\n\022io.substrait.prot" - "oP\001Z*github.com/substrait-io/substrait-g" - "o/proto\252\002\022Substrait.Protobufb\006proto3" + "2\025.substrait.Expression\0223\n\004type\030\005 \001(\0162%." + "substrait.NestedLoopJoinRel.JoinType\022C\n\022" + "advanced_extension\030\n \001(\0132\'.substrait.ext" + "ensions.AdvancedExtension\"\310\002\n\010JoinType\022\031" + "\n\025JOIN_TYPE_UNSPECIFIED\020\000\022\023\n\017JOIN_TYPE_I" + "NNER\020\001\022\023\n\017JOIN_TYPE_OUTER\020\002\022\022\n\016JOIN_TYPE" + "_LEFT\020\003\022\023\n\017JOIN_TYPE_RIGHT\020\004\022\027\n\023JOIN_TYP" + "E_LEFT_SEMI\020\005\022\030\n\024JOIN_TYPE_RIGHT_SEMI\020\006\022" + "\027\n\023JOIN_TYPE_LEFT_ANTI\020\007\022\030\n\024JOIN_TYPE_RI" + "GHT_ANTI\020\010\022\031\n\025JOIN_TYPE_LEFT_SINGLE\020\t\022\032\n" + "\026JOIN_TYPE_RIGHT_SINGLE\020\n\022\027\n\023JOIN_TYPE_L" + "EFT_MARK\020\013\022\030\n\024JOIN_TYPE_RIGHT_MARK\020\014\"i\n\031" + "DuplicateEliminatedGetRel\022$\n\006common\030\001 \001(" + "\0132\024.substrait.RelCommon\022&\n\005input\030\002 \001(\0132\027" + ".substrait.ReferenceRel\"\347\007\n\032DuplicateEli" + "minatedJoinRel\022$\n\006common\030\001 \001(\0132\024.substra" + "it.RelCommon\022\034\n\004left\030\002 \001(\0132\016.substrait.R" + "el\022\035\n\005right\030\003 \001(\0132\016.substrait.Rel\022)\n\nexp" + "ression\030\004 \001(\0132\025.substrait.Expression\022/\n\020" + "post_join_filter\030\005 \001(\0132\025.substrait.Expre" + "ssion\022<\n\004type\030\006 \001(\0162..substrait.Duplicat" + "eEliminatedJoinRel.JoinType\022J\n\034duplicate" + "_eliminated_columns\030\007 \003(\0132$.substrait.Ex" + "pression.FieldReference\022`\n\031duplicate_eli" + "minated_side\030\010 \001(\0162=.substrait.Duplicate" + "EliminatedJoinRel.DuplicateEliminatedSid" + "e\022C\n\022advanced_extension\030\n \001(\0132\'.substrai" + "t.extensions.AdvancedExtension\"\215\001\n\027Dupli" + "cateEliminatedSide\022)\n%DUPLICATE_ELIMINAT" + "ED_SIDE_UNSPECIFIED\020\000\022\"\n\036DUPLICATE_ELIMI" + "NATED_SIDE_LEFT\020\001\022#\n\037DUPLICATE_ELIMINATE" + "D_SIDE_RIGHT\020\002\"\310\002\n\010JoinType\022\031\n\025JOIN_TYPE" + "_UNSPECIFIED\020\000\022\023\n\017JOIN_TYPE_INNER\020\001\022\023\n\017J" + "OIN_TYPE_OUTER\020\002\022\022\n\016JOIN_TYPE_LEFT\020\003\022\023\n\017" + "JOIN_TYPE_RIGHT\020\004\022\027\n\023JOIN_TYPE_LEFT_SEMI" + "\020\005\022\027\n\023JOIN_TYPE_LEFT_ANTI\020\006\022\031\n\025JOIN_TYPE" + "_LEFT_SINGLE\020\007\022\030\n\024JOIN_TYPE_RIGHT_SEMI\020\010" + "\022\030\n\024JOIN_TYPE_RIGHT_ANTI\020\t\022\032\n\026JOIN_TYPE_" + "RIGHT_SINGLE\020\n\022\027\n\023JOIN_TYPE_LEFT_MARK\020\013\022" + "\030\n\024JOIN_TYPE_RIGHT_MARK\020\014\"w\n\020FunctionArg" + "ument\022\016\n\004enum\030\001 \001(\tH\000\022\037\n\004type\030\002 \001(\0132\017.su" + "bstrait.TypeH\000\022&\n\005value\030\003 \001(\0132\025.substrai" + "t.ExpressionH\000B\n\n\010arg_type\"2\n\016FunctionOp" + "tion\022\014\n\004name\030\001 \001(\t\022\022\n\npreference\030\002 \003(\t\"\337" + "K\n\nExpression\0220\n\007literal\030\001 \001(\0132\035.substra" + "it.Expression.LiteralH\000\0229\n\tselection\030\002 \001" + "(\0132$.substrait.Expression.FieldReference" + "H\000\022\?\n\017scalar_function\030\003 \001(\0132$.substrait." + "Expression.ScalarFunctionH\000\022\?\n\017window_fu" + "nction\030\005 \001(\0132$.substrait.Expression.Wind" + "owFunctionH\000\022/\n\007if_then\030\006 \001(\0132\034.substrai" + "t.Expression.IfThenH\000\022C\n\021switch_expressi" + "on\030\007 \001(\0132&.substrait.Expression.SwitchEx" + "pressionH\000\022@\n\020singular_or_list\030\010 \001(\0132$.s" + "ubstrait.Expression.SingularOrListH\000\022:\n\r" + "multi_or_list\030\t \001(\0132!.substrait.Expressi" + "on.MultiOrListH\000\022*\n\004cast\030\013 \001(\0132\032.substra" + "it.Expression.CastH\000\0222\n\010subquery\030\014 \001(\0132\036" + ".substrait.Expression.SubqueryH\000\022.\n\006nest" + "ed\030\r \001(\0132\034.substrait.Expression.NestedH\000" + "\022.\n\004enum\030\n \001(\0132\032.substrait.Expression.En" + "umB\002\030\001H\000\032r\n\004Enum\022\023\n\tspecified\030\001 \001(\tH\000\0227\n" + "\013unspecified\030\002 \001(\0132 .substrait.Expressio" + "n.Enum.EmptyH\000\032\013\n\005Empty:\002\030\001:\002\030\001B\013\n\tenum_" + "kind\032\330\021\n\007Literal\022\021\n\007boolean\030\001 \001(\010H\000\022\014\n\002i" + "8\030\002 \001(\005H\000\022\r\n\003i16\030\003 \001(\005H\000\022\r\n\003i32\030\005 \001(\005H\000\022" + "\r\n\003i64\030\007 \001(\003H\000\022\016\n\004fp32\030\n \001(\002H\000\022\016\n\004fp64\030\013" + " \001(\001H\000\022\020\n\006string\030\014 \001(\tH\000\022\020\n\006binary\030\r \001(\014" + "H\000\022\027\n\ttimestamp\030\016 \001(\003B\002\030\001H\000\022\016\n\004date\030\020 \001(" + "\005H\000\022\016\n\004time\030\021 \001(\003H\000\022S\n\026interval_year_to_" + "month\030\023 \001(\01321.substrait.Expression.Liter" + "al.IntervalYearToMonthH\000\022S\n\026interval_day" + "_to_second\030\024 \001(\01321.substrait.Expression." + "Literal.IntervalDayToSecondH\000\022K\n\021interva" + "l_compound\030$ \001(\0132..substrait.Expression." + "Literal.IntervalCompoundH\000\022\024\n\nfixed_char" + "\030\025 \001(\tH\000\0229\n\010var_char\030\026 \001(\0132%.substrait.E" + "xpression.Literal.VarCharH\000\022\026\n\014fixed_bin" + "ary\030\027 \001(\014H\000\0228\n\007decimal\030\030 \001(\0132%.substrait" + ".Expression.Literal.DecimalH\000\022O\n\023precisi" + "on_timestamp\030\" \001(\01320.substrait.Expressio" + "n.Literal.PrecisionTimestampH\000\022R\n\026precis" + "ion_timestamp_tz\030# \001(\01320.substrait.Expre" + "ssion.Literal.PrecisionTimestampH\000\0226\n\006st" + "ruct\030\031 \001(\0132$.substrait.Expression.Litera" + "l.StructH\000\0220\n\003map\030\032 \001(\0132!.substrait.Expr" + "ession.Literal.MapH\000\022\032\n\014timestamp_tz\030\033 \001" + "(\003B\002\030\001H\000\022\016\n\004uuid\030\034 \001(\014H\000\022\037\n\004null\030\035 \001(\0132\017" + ".substrait.TypeH\000\0222\n\004list\030\036 \001(\0132\".substr" + "ait.Expression.Literal.ListH\000\022*\n\nempty_l" + "ist\030\037 \001(\0132\024.substrait.Type.ListH\000\022(\n\temp" + "ty_map\030 \001(\0132\023.substrait.Type.MapH\000\022A\n\014u" + "ser_defined\030! \001(\0132).substrait.Expression" + ".Literal.UserDefinedH\000\022\020\n\010nullable\0302 \001(\010" + "\022 \n\030type_variation_reference\0303 \001(\r\032(\n\007Va" + "rChar\022\r\n\005value\030\001 \001(\t\022\016\n\006length\030\002 \001(\r\032:\n\007" + "Decimal\022\r\n\005value\030\001 \001(\014\022\021\n\tprecision\030\002 \001(" + "\005\022\r\n\005scale\030\003 \001(\005\0326\n\022PrecisionTimestamp\022\021" + "\n\tprecision\030\001 \001(\005\022\r\n\005value\030\002 \001(\003\032\253\001\n\003Map" + "\022>\n\nkey_values\030\001 \003(\0132*.substrait.Express" + "ion.Literal.Map.KeyValue\032d\n\010KeyValue\022*\n\003" + "key\030\001 \001(\0132\035.substrait.Expression.Literal" + "\022,\n\005value\030\002 \001(\0132\035.substrait.Expression.L" + "iteral\0324\n\023IntervalYearToMonth\022\r\n\005years\030\001" + " \001(\005\022\016\n\006months\030\002 \001(\005\032\213\001\n\023IntervalDayToSe" + "cond\022\014\n\004days\030\001 \001(\005\022\017\n\007seconds\030\002 \001(\005\022\032\n\014m" + "icroseconds\030\003 \001(\005B\002\030\001H\000\022\023\n\tprecision\030\004 \001" + "(\005H\000\022\022\n\nsubseconds\030\005 \001(\003B\020\n\016precision_mo" + "de\032\270\001\n\020IntervalCompound\022Q\n\026interval_year" + "_to_month\030\001 \001(\01321.substrait.Expression.L" + "iteral.IntervalYearToMonth\022Q\n\026interval_d" + "ay_to_second\030\002 \001(\01321.substrait.Expressio" + "n.Literal.IntervalDayToSecond\0327\n\006Struct\022" + "-\n\006fields\030\001 \003(\0132\035.substrait.Expression.L" + "iteral\0325\n\004List\022-\n\006values\030\001 \003(\0132\035.substra" + "it.Expression.Literal\032\277\001\n\013UserDefined\022\026\n" + "\016type_reference\030\001 \001(\r\0222\n\017type_parameters" + "\030\003 \003(\0132\031.substrait.Type.Parameter\022%\n\005val" + "ue\030\002 \001(\0132\024.google.protobuf.AnyH\000\0226\n\006stru" + "ct\030\004 \001(\0132$.substrait.Expression.Literal." + "StructH\000B\005\n\003valB\016\n\014literal_type\032\343\003\n\006Nest" + "ed\022\020\n\010nullable\030\001 \001(\010\022 \n\030type_variation_r" + "eference\030\002 \001(\r\0225\n\006struct\030\003 \001(\0132#.substra" + "it.Expression.Nested.StructH\000\0221\n\004list\030\004 " + "\001(\0132!.substrait.Expression.Nested.ListH\000" + "\022/\n\003map\030\005 \001(\0132 .substrait.Expression.Nes" + "ted.MapH\000\032\232\001\n\003Map\022=\n\nkey_values\030\001 \003(\0132)." + "substrait.Expression.Nested.Map.KeyValue" + "\032T\n\010KeyValue\022\"\n\003key\030\001 \001(\0132\025.substrait.Ex" + "pression\022$\n\005value\030\002 \001(\0132\025.substrait.Expr" + "ession\032/\n\006Struct\022%\n\006fields\030\001 \003(\0132\025.subst" + "rait.Expression\032-\n\004List\022%\n\006values\030\001 \003(\0132" + "\025.substrait.ExpressionB\r\n\013nested_type\032\327\001" + "\n\016ScalarFunction\022\032\n\022function_reference\030\001" + " \001(\r\022.\n\targuments\030\004 \003(\0132\033.substrait.Func" + "tionArgument\022*\n\007options\030\005 \003(\0132\031.substrai" + "t.FunctionOption\022$\n\013output_type\030\003 \001(\0132\017." + "substrait.Type\022\'\n\004args\030\002 \003(\0132\025.substrait" + ".ExpressionB\002\030\001\032\321\010\n\016WindowFunction\022\032\n\022fu" + "nction_reference\030\001 \001(\r\022.\n\targuments\030\t \003(" + "\0132\033.substrait.FunctionArgument\022*\n\007option" + "s\030\013 \003(\0132\031.substrait.FunctionOption\022$\n\013ou" + "tput_type\030\007 \001(\0132\017.substrait.Type\022*\n\005phas" + "e\030\006 \001(\0162\033.substrait.AggregationPhase\022#\n\005" + "sorts\030\003 \003(\0132\024.substrait.SortField\022F\n\ninv" + "ocation\030\n \001(\01622.substrait.AggregateFunct" + "ion.AggregationInvocation\022)\n\npartitions\030" + "\002 \003(\0132\025.substrait.Expression\022D\n\013bounds_t" + "ype\030\014 \001(\0162/.substrait.Expression.WindowF" + "unction.BoundsType\022\?\n\013lower_bound\030\005 \001(\0132" + "*.substrait.Expression.WindowFunction.Bo" + "und\022\?\n\013upper_bound\030\004 \001(\0132*.substrait.Exp" + "ression.WindowFunction.Bound\022\'\n\004args\030\010 \003" + "(\0132\025.substrait.ExpressionB\002\030\001\032\223\003\n\005Bound\022" + "I\n\tpreceding\030\001 \001(\01324.substrait.Expressio" + "n.WindowFunction.Bound.PrecedingH\000\022I\n\tfo" + "llowing\030\002 \001(\01324.substrait.Expression.Win" + "dowFunction.Bound.FollowingH\000\022L\n\013current" + "_row\030\003 \001(\01325.substrait.Expression.Window" + "Function.Bound.CurrentRowH\000\022I\n\tunbounded" + "\030\004 \001(\01324.substrait.Expression.WindowFunc" + "tion.Bound.UnboundedH\000\032\033\n\tPreceding\022\016\n\006o" + "ffset\030\001 \001(\003\032\033\n\tFollowing\022\016\n\006offset\030\001 \001(\003" + "\032\014\n\nCurrentRow\032\013\n\tUnboundedB\006\n\004kind\"V\n\nB" + "oundsType\022\033\n\027BOUNDS_TYPE_UNSPECIFIED\020\000\022\024" + "\n\020BOUNDS_TYPE_ROWS\020\001\022\025\n\021BOUNDS_TYPE_RANG" + "E\020\002\032\265\001\n\006IfThen\0222\n\003ifs\030\001 \003(\0132%.substrait." + "Expression.IfThen.IfClause\022#\n\004else\030\002 \001(\013" + "2\025.substrait.Expression\032R\n\010IfClause\022!\n\002i" + "f\030\001 \001(\0132\025.substrait.Expression\022#\n\004then\030\002" + " \001(\0132\025.substrait.Expression\032\216\002\n\004Cast\022\035\n\004" + "type\030\001 \001(\0132\017.substrait.Type\022$\n\005input\030\002 \001" + "(\0132\025.substrait.Expression\022D\n\020failure_beh" + "avior\030\003 \001(\0162*.substrait.Expression.Cast." + "FailureBehavior\"{\n\017FailureBehavior\022 \n\034FA" + "ILURE_BEHAVIOR_UNSPECIFIED\020\000\022 \n\034FAILURE_" + "BEHAVIOR_RETURN_NULL\020\001\022$\n FAILURE_BEHAVI" + "OR_THROW_EXCEPTION\020\002\032\365\001\n\020SwitchExpressio" + "n\022$\n\005match\030\003 \001(\0132\025.substrait.Expression\022" + ";\n\003ifs\030\001 \003(\0132..substrait.Expression.Swit" + "chExpression.IfValue\022#\n\004else\030\002 \001(\0132\025.sub" + "strait.Expression\032Y\n\007IfValue\022)\n\002if\030\001 \001(\013" + "2\035.substrait.Expression.Literal\022#\n\004then\030" + "\002 \001(\0132\025.substrait.Expression\032^\n\016Singular" + "OrList\022$\n\005value\030\001 \001(\0132\025.substrait.Expres" + "sion\022&\n\007options\030\002 \003(\0132\025.substrait.Expres" + "sion\032\237\001\n\013MultiOrList\022$\n\005value\030\001 \003(\0132\025.su" + "bstrait.Expression\0229\n\007options\030\002 \003(\0132(.su" + "bstrait.Expression.MultiOrList.Record\032/\n" + "\006Record\022%\n\006fields\030\001 \003(\0132\025.substrait.Expr" + "ession\032\243\003\n\020EmbeddedFunction\022(\n\targuments" + "\030\001 \003(\0132\025.substrait.Expression\022$\n\013output_" + "type\030\002 \001(\0132\017.substrait.Type\022]\n\026python_pi" + "ckle_function\030\003 \001(\0132;.substrait.Expressi" + "on.EmbeddedFunction.PythonPickleFunction" + "H\000\022[\n\025web_assembly_function\030\004 \001(\0132:.subs" + "trait.Expression.EmbeddedFunction.WebAss" + "emblyFunctionH\000\032>\n\024PythonPickleFunction\022" + "\020\n\010function\030\001 \001(\014\022\024\n\014prerequisite\030\002 \003(\t\032" + ";\n\023WebAssemblyFunction\022\016\n\006script\030\001 \001(\014\022\024" + "\n\014prerequisite\030\002 \003(\tB\006\n\004kind\032\232\004\n\020Referen" + "ceSegment\022@\n\007map_key\030\001 \001(\0132-.substrait.E" + "xpression.ReferenceSegment.MapKeyH\000\022J\n\014s" + "truct_field\030\002 \001(\01322.substrait.Expression" + ".ReferenceSegment.StructFieldH\000\022J\n\014list_" + "element\030\003 \001(\01322.substrait.Expression.Ref" + "erenceSegment.ListElementH\000\032o\n\006MapKey\022.\n" + "\007map_key\030\001 \001(\0132\035.substrait.Expression.Li" + "teral\0225\n\005child\030\002 \001(\0132&.substrait.Express" + "ion.ReferenceSegment\032S\n\013StructField\022\r\n\005f" + "ield\030\001 \001(\005\0225\n\005child\030\002 \001(\0132&.substrait.Ex" + "pression.ReferenceSegment\032T\n\013ListElement" + "\022\016\n\006offset\030\001 \001(\005\0225\n\005child\030\002 \001(\0132&.substr" + "ait.Expression.ReferenceSegmentB\020\n\016refer" + "ence_type\032\360\t\n\016MaskExpression\022A\n\006select\030\001" + " \001(\01321.substrait.Expression.MaskExpressi" + "on.StructSelect\022 \n\030maintain_singular_str" + "uct\030\002 \001(\010\032\325\001\n\006Select\022C\n\006struct\030\001 \001(\01321.s" + "ubstrait.Expression.MaskExpression.Struc" + "tSelectH\000\022\?\n\004list\030\002 \001(\0132/.substrait.Expr" + "ession.MaskExpression.ListSelectH\000\022=\n\003ma" + "p\030\003 \001(\0132..substrait.Expression.MaskExpre" + "ssion.MapSelectH\000B\006\n\004type\032U\n\014StructSelec" + "t\022E\n\014struct_items\030\001 \003(\0132/.substrait.Expr" + "ession.MaskExpression.StructItem\032W\n\nStru" + "ctItem\022\r\n\005field\030\001 \001(\005\022:\n\005child\030\002 \001(\0132+.s" + "ubstrait.Expression.MaskExpression.Selec" + "t\032\264\003\n\nListSelect\022Q\n\tselection\030\001 \003(\0132>.su" + "bstrait.Expression.MaskExpression.ListSe" + "lect.ListSelectItem\022:\n\005child\030\002 \001(\0132+.sub" + "strait.Expression.MaskExpression.Select\032" + "\226\002\n\016ListSelectItem\022Z\n\004item\030\001 \001(\0132J.subst" + "rait.Expression.MaskExpression.ListSelec" + "t.ListSelectItem.ListElementH\000\022Y\n\005slice\030" + "\002 \001(\0132H.substrait.Expression.MaskExpress" + "ion.ListSelect.ListSelectItem.ListSliceH" + "\000\032\034\n\013ListElement\022\r\n\005field\030\001 \001(\005\032\'\n\tListS" + "lice\022\r\n\005start\030\001 \001(\005\022\013\n\003end\030\002 \001(\005B\006\n\004type" + "\032\271\002\n\tMapSelect\022D\n\003key\030\001 \001(\01325.substrait." + "Expression.MaskExpression.MapSelect.MapK" + "eyH\000\022U\n\nexpression\030\002 \001(\0132\?.substrait.Exp" + "ression.MaskExpression.MapSelect.MapKeyE" + "xpressionH\000\022:\n\005child\030\003 \001(\0132+.substrait.E" + "xpression.MaskExpression.Select\032\031\n\006MapKe" + "y\022\017\n\007map_key\030\001 \001(\t\032.\n\020MapKeyExpression\022\032" + "\n\022map_key_expression\030\001 \001(\tB\010\n\006select\032\266\003\n" + "\016FieldReference\022B\n\020direct_reference\030\001 \001(" + "\0132&.substrait.Expression.ReferenceSegmen" + "tH\000\022@\n\020masked_reference\030\002 \001(\0132$.substrai" + "t.Expression.MaskExpressionH\000\022+\n\nexpress" + "ion\030\003 \001(\0132\025.substrait.ExpressionH\001\022L\n\016ro" + "ot_reference\030\004 \001(\01322.substrait.Expressio" + "n.FieldReference.RootReferenceH\001\022N\n\017oute" + "r_reference\030\005 \001(\01323.substrait.Expression" + ".FieldReference.OuterReferenceH\001\032\017\n\rRoot" + "Reference\032#\n\016OuterReference\022\021\n\tsteps_out" + "\030\001 \001(\rB\020\n\016reference_typeB\013\n\troot_type\032\214\t" + "\n\010Subquery\0227\n\006scalar\030\001 \001(\0132%.substrait.E" + "xpression.Subquery.ScalarH\000\022B\n\014in_predic" + "ate\030\002 \001(\0132*.substrait.Expression.Subquer" + "y.InPredicateH\000\022D\n\rset_predicate\030\003 \001(\0132+" + ".substrait.Expression.Subquery.SetPredic" + "ateH\000\022F\n\016set_comparison\030\004 \001(\0132,.substrai" + "t.Expression.Subquery.SetComparisonH\000\032\'\n" + "\006Scalar\022\035\n\005input\030\001 \001(\0132\016.substrait.Rel\032W" + "\n\013InPredicate\022&\n\007needles\030\001 \003(\0132\025.substra" + "it.Expression\022 \n\010haystack\030\002 \001(\0132\016.substr" + "ait.Rel\032\334\001\n\014SetPredicate\022M\n\014predicate_op" + "\030\001 \001(\01627.substrait.Expression.Subquery.S" + "etPredicate.PredicateOp\022\036\n\006tuples\030\002 \001(\0132" + "\016.substrait.Rel\"]\n\013PredicateOp\022\034\n\030PREDIC" + "ATE_OP_UNSPECIFIED\020\000\022\027\n\023PREDICATE_OP_EXI" + "STS\020\001\022\027\n\023PREDICATE_OP_UNIQUE\020\002\032\202\004\n\rSetCo" + "mparison\022N\n\014reduction_op\030\001 \001(\01628.substra" + "it.Expression.Subquery.SetComparison.Red" + "uctionOp\022P\n\rcomparison_op\030\002 \001(\01629.substr" + "ait.Expression.Subquery.SetComparison.Co" + "mparisonOp\022#\n\004left\030\003 \001(\0132\025.substrait.Exp" + "ression\022\035\n\005right\030\004 \001(\0132\016.substrait.Rel\"\261" + "\001\n\014ComparisonOp\022\035\n\031COMPARISON_OP_UNSPECI" + "FIED\020\000\022\024\n\020COMPARISON_OP_EQ\020\001\022\024\n\020COMPARIS" + "ON_OP_NE\020\002\022\024\n\020COMPARISON_OP_LT\020\003\022\024\n\020COMP" + "ARISON_OP_GT\020\004\022\024\n\020COMPARISON_OP_LE\020\005\022\024\n\020" + "COMPARISON_OP_GE\020\006\"W\n\013ReductionOp\022\034\n\030RED" + "UCTION_OP_UNSPECIFIED\020\000\022\024\n\020REDUCTION_OP_" + "ANY\020\001\022\024\n\020REDUCTION_OP_ALL\020\002B\017\n\rsubquery_" + "typeB\n\n\010rex_type\"\377\002\n\tSortField\022#\n\004expr\030\001" + " \001(\0132\025.substrait.Expression\0227\n\tdirection" + "\030\002 \001(\0162\".substrait.SortField.SortDirecti" + "onH\000\022\'\n\035comparison_function_reference\030\003 " + "\001(\rH\000\"\335\001\n\rSortDirection\022\036\n\032SORT_DIRECTIO" + "N_UNSPECIFIED\020\000\022\"\n\036SORT_DIRECTION_ASC_NU" + "LLS_FIRST\020\001\022!\n\035SORT_DIRECTION_ASC_NULLS_" + "LAST\020\002\022#\n\037SORT_DIRECTION_DESC_NULLS_FIRS" + "T\020\003\022\"\n\036SORT_DIRECTION_DESC_NULLS_LAST\020\004\022" + "\034\n\030SORT_DIRECTION_CLUSTERED\020\005B\013\n\tsort_ki" + "nd\"\372\003\n\021AggregateFunction\022\032\n\022function_ref" + "erence\030\001 \001(\r\022.\n\targuments\030\007 \003(\0132\033.substr" + "ait.FunctionArgument\022*\n\007options\030\010 \003(\0132\031." + "substrait.FunctionOption\022$\n\013output_type\030" + "\005 \001(\0132\017.substrait.Type\022*\n\005phase\030\004 \001(\0162\033." + "substrait.AggregationPhase\022#\n\005sorts\030\003 \003(" + "\0132\024.substrait.SortField\022F\n\ninvocation\030\006 " + "\001(\01622.substrait.AggregateFunction.Aggreg" + "ationInvocation\022\'\n\004args\030\002 \003(\0132\025.substrai" + "t.ExpressionB\002\030\001\"\204\001\n\025AggregationInvocati" + "on\022&\n\"AGGREGATION_INVOCATION_UNSPECIFIED" + "\020\000\022\036\n\032AGGREGATION_INVOCATION_ALL\020\001\022#\n\037AG" + "GREGATION_INVOCATION_DISTINCT\020\002\"\'\n\014Refer" + "enceRel\022\027\n\017subtree_ordinal\030\001 \001(\005*\357\001\n\020Agg" + "regationPhase\022!\n\035AGGREGATION_PHASE_UNSPE" + "CIFIED\020\000\022-\n)AGGREGATION_PHASE_INITIAL_TO" + "_INTERMEDIATE\020\001\0222\n.AGGREGATION_PHASE_INT" + "ERMEDIATE_TO_INTERMEDIATE\020\002\022\'\n#AGGREGATI" + "ON_PHASE_INITIAL_TO_RESULT\020\003\022,\n(AGGREGAT" + "ION_PHASE_INTERMEDIATE_TO_RESULT\020\004BW\n\022io" + ".substrait.protoP\001Z*github.com/substrait" + "-io/substrait-go/proto\252\002\022Substrait.Proto" + "bufb\006proto3" ; static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_substrait_2falgebra_2eproto_deps[3] = { &::descriptor_table_google_2fprotobuf_2fany_2eproto, @@ -3611,8 +3738,8 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor }; static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_substrait_2falgebra_2eproto_once; const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_substrait_2falgebra_2eproto = { - false, false, 23476, descriptor_table_protodef_substrait_2falgebra_2eproto, "substrait/algebra.proto", - &descriptor_table_substrait_2falgebra_2eproto_once, descriptor_table_substrait_2falgebra_2eproto_deps, 3, 118, + false, false, 25011, descriptor_table_protodef_substrait_2falgebra_2eproto, "substrait/algebra.proto", + &descriptor_table_substrait_2falgebra_2eproto_once, descriptor_table_substrait_2falgebra_2eproto_deps, 3, 121, schemas, file_default_instances, TableStruct_substrait_2falgebra_2eproto::offsets, file_level_metadata_substrait_2falgebra_2eproto, file_level_enum_descriptors_substrait_2falgebra_2eproto, file_level_service_descriptors_substrait_2falgebra_2eproto, }; @@ -3638,6 +3765,10 @@ bool JoinRel_JoinType_IsValid(int value) { case 6: case 7: case 8: + case 9: + case 10: + case 11: + case 12: return true; default: return false; @@ -3650,10 +3781,14 @@ constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_INNER; constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_OUTER; constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_LEFT; constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_RIGHT; -constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_SEMI; -constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_ANTI; -constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_SINGLE; -constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_MARK; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_LEFT_SEMI; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_LEFT_ANTI; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_LEFT_SINGLE; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_RIGHT_SEMI; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_RIGHT_ANTI; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_RIGHT_SINGLE; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_LEFT_MARK; +constexpr JoinRel_JoinType JoinRel::JOIN_TYPE_RIGHT_MARK; constexpr JoinRel_JoinType JoinRel::JoinType_MIN; constexpr JoinRel_JoinType JoinRel::JoinType_MAX; constexpr int JoinRel::JoinType_ARRAYSIZE; @@ -3831,6 +3966,10 @@ bool HashJoinRel_JoinType_IsValid(int value) { case 6: case 7: case 8: + case 9: + case 10: + case 11: + case 12: return true; default: return false; @@ -3847,6 +3986,10 @@ constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_LEFT_SEMI; constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_RIGHT_SEMI; constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_LEFT_ANTI; constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_RIGHT_ANTI; +constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_LEFT_SINGLE; +constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_RIGHT_SINGLE; +constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_LEFT_MARK; +constexpr HashJoinRel_JoinType HashJoinRel::JOIN_TYPE_RIGHT_MARK; constexpr HashJoinRel_JoinType HashJoinRel::JoinType_MIN; constexpr HashJoinRel_JoinType HashJoinRel::JoinType_MAX; constexpr int HashJoinRel::JoinType_ARRAYSIZE; @@ -3866,6 +4009,10 @@ bool MergeJoinRel_JoinType_IsValid(int value) { case 6: case 7: case 8: + case 9: + case 10: + case 11: + case 12: return true; default: return false; @@ -3882,6 +4029,10 @@ constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_LEFT_SEMI; constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_RIGHT_SEMI; constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_LEFT_ANTI; constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_RIGHT_ANTI; +constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_LEFT_SINGLE; +constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_RIGHT_SINGLE; +constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_LEFT_MARK; +constexpr MergeJoinRel_JoinType MergeJoinRel::JOIN_TYPE_RIGHT_MARK; constexpr MergeJoinRel_JoinType MergeJoinRel::JoinType_MIN; constexpr MergeJoinRel_JoinType MergeJoinRel::JoinType_MAX; constexpr int MergeJoinRel::JoinType_ARRAYSIZE; @@ -3901,6 +4052,10 @@ bool NestedLoopJoinRel_JoinType_IsValid(int value) { case 6: case 7: case 8: + case 9: + case 10: + case 11: + case 12: return true; default: return false; @@ -3917,15 +4072,42 @@ constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_LEFT_SEMI; constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_RIGHT_SEMI; constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_LEFT_ANTI; constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_RIGHT_ANTI; +constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_LEFT_SINGLE; +constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_RIGHT_SINGLE; +constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_LEFT_MARK; +constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JOIN_TYPE_RIGHT_MARK; constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JoinType_MIN; constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel::JoinType_MAX; constexpr int NestedLoopJoinRel::JoinType_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) -const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DelimiterJoinRel_JoinType_descriptor() { +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); return file_level_enum_descriptors_substrait_2falgebra_2eproto[10]; } -bool DelimiterJoinRel_JoinType_IsValid(int value) { +bool DuplicateEliminatedJoinRel_DuplicateEliminatedSide_IsValid(int value) { + switch (value) { + case 0: + case 1: + case 2: + return true; + default: + return false; + } +} + +#if (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_UNSPECIFIED; +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_LEFT; +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::DUPLICATE_ELIMINATED_SIDE_RIGHT; +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::DuplicateEliminatedSide_MIN; +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::DuplicateEliminatedSide_MAX; +constexpr int DuplicateEliminatedJoinRel::DuplicateEliminatedSide_ARRAYSIZE; +#endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DuplicateEliminatedJoinRel_JoinType_descriptor() { + ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); + return file_level_enum_descriptors_substrait_2falgebra_2eproto[11]; +} +bool DuplicateEliminatedJoinRel_JoinType_IsValid(int value) { switch (value) { case 0: case 1: @@ -3936,6 +4118,10 @@ bool DelimiterJoinRel_JoinType_IsValid(int value) { case 6: case 7: case 8: + case 9: + case 10: + case 11: + case 12: return true; default: return false; @@ -3943,22 +4129,26 @@ bool DelimiterJoinRel_JoinType_IsValid(int value) { } #if (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_UNSPECIFIED; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_INNER; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_OUTER; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_LEFT; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_RIGHT; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_LEFT_SEMI; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_RIGHT_SEMI; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_LEFT_ANTI; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JOIN_TYPE_RIGHT_ANTI; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JoinType_MIN; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel::JoinType_MAX; -constexpr int DelimiterJoinRel::JoinType_ARRAYSIZE; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_UNSPECIFIED; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_INNER; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_OUTER; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_LEFT; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_RIGHT; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_LEFT_SEMI; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_LEFT_ANTI; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_LEFT_SINGLE; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_RIGHT_SEMI; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_RIGHT_ANTI; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_RIGHT_SINGLE; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_LEFT_MARK; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JOIN_TYPE_RIGHT_MARK; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JoinType_MIN; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::JoinType_MAX; +constexpr int DuplicateEliminatedJoinRel::JoinType_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* Expression_WindowFunction_BoundsType_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[11]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[12]; } bool Expression_WindowFunction_BoundsType_IsValid(int value) { switch (value) { @@ -3981,7 +4171,7 @@ constexpr int Expression_WindowFunction::BoundsType_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* Expression_Cast_FailureBehavior_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[12]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[13]; } bool Expression_Cast_FailureBehavior_IsValid(int value) { switch (value) { @@ -4004,7 +4194,7 @@ constexpr int Expression_Cast::FailureBehavior_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* Expression_Subquery_SetPredicate_PredicateOp_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[13]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[14]; } bool Expression_Subquery_SetPredicate_PredicateOp_IsValid(int value) { switch (value) { @@ -4027,7 +4217,7 @@ constexpr int Expression_Subquery_SetPredicate::PredicateOp_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* Expression_Subquery_SetComparison_ComparisonOp_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[14]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[15]; } bool Expression_Subquery_SetComparison_ComparisonOp_IsValid(int value) { switch (value) { @@ -4058,7 +4248,7 @@ constexpr int Expression_Subquery_SetComparison::ComparisonOp_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* Expression_Subquery_SetComparison_ReductionOp_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[15]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[16]; } bool Expression_Subquery_SetComparison_ReductionOp_IsValid(int value) { switch (value) { @@ -4081,7 +4271,7 @@ constexpr int Expression_Subquery_SetComparison::ReductionOp_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* SortField_SortDirection_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[16]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[17]; } bool SortField_SortDirection_IsValid(int value) { switch (value) { @@ -4110,7 +4300,7 @@ constexpr int SortField::SortDirection_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* AggregateFunction_AggregationInvocation_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[17]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[18]; } bool AggregateFunction_AggregationInvocation_IsValid(int value) { switch (value) { @@ -4133,7 +4323,7 @@ constexpr int AggregateFunction::AggregationInvocation_ARRAYSIZE; #endif // (__cplusplus < 201703) && (!defined(_MSC_VER) || (_MSC_VER >= 1900 && _MSC_VER < 1912)) const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* AggregationPhase_descriptor() { ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors(&descriptor_table_substrait_2falgebra_2eproto); - return file_level_enum_descriptors_substrait_2falgebra_2eproto[18]; + return file_level_enum_descriptors_substrait_2falgebra_2eproto[19]; } bool AggregationPhase_IsValid(int value) { switch (value) { @@ -4903,6 +5093,14 @@ RelCommon_Hint::RelCommon_Hint(::PROTOBUF_NAMESPACE_ID::Arena* arena, RelCommon_Hint::RelCommon_Hint(const RelCommon_Hint& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + alias_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + alias_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (!from._internal_alias().empty()) { + alias_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_alias(), + GetArenaForAllocation()); + } if (from._internal_has_stats()) { stats_ = new ::substrait::RelCommon_Hint_Stats(*from.stats_); } else { @@ -4922,6 +5120,10 @@ RelCommon_Hint::RelCommon_Hint(const RelCommon_Hint& from) } inline void RelCommon_Hint::SharedCtor() { +alias_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + alias_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&stats_) - reinterpret_cast(this)), 0, static_cast(reinterpret_cast(&advanced_extension_) - @@ -4937,6 +5139,7 @@ RelCommon_Hint::~RelCommon_Hint() { inline void RelCommon_Hint::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + alias_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); if (this != internal_default_instance()) delete stats_; if (this != internal_default_instance()) delete constraint_; if (this != internal_default_instance()) delete advanced_extension_; @@ -4958,6 +5161,7 @@ void RelCommon_Hint::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + alias_.ClearToEmpty(); if (GetArenaForAllocation() == nullptr && stats_ != nullptr) { delete stats_; } @@ -4995,6 +5199,16 @@ const char* RelCommon_Hint::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE } else goto handle_unusual; continue; + // string alias = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 26)) { + auto str = _internal_mutable_alias(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(::PROTOBUF_NAMESPACE_ID::internal::VerifyUTF8(str, "substrait.RelCommon.Hint.alias")); + CHK_(ptr); + } else + goto handle_unusual; + continue; // .substrait.extensions.AdvancedExtension advanced_extension = 10; case 10: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 82)) { @@ -5048,6 +5262,16 @@ uint8_t* RelCommon_Hint::_InternalSerialize( 2, _Internal::constraint(this), target, stream); } + // string alias = 3; + if (!this->_internal_alias().empty()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_alias().data(), static_cast(this->_internal_alias().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.RelCommon.Hint.alias"); + target = stream->WriteStringMaybeAliased( + 3, this->_internal_alias(), target); + } + // .substrait.extensions.AdvancedExtension advanced_extension = 10; if (this->_internal_has_advanced_extension()) { target = stream->EnsureSpace(target); @@ -5072,6 +5296,13 @@ size_t RelCommon_Hint::ByteSizeLong() const { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; + // string alias = 3; + if (!this->_internal_alias().empty()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_alias()); + } + // .substrait.RelCommon.Hint.Stats stats = 1; if (this->_internal_has_stats()) { total_size += 1 + @@ -5115,6 +5346,9 @@ void RelCommon_Hint::MergeFrom(const RelCommon_Hint& from) { uint32_t cached_has_bits = 0; (void) cached_has_bits; + if (!from._internal_alias().empty()) { + _internal_set_alias(from._internal_alias()); + } if (from._internal_has_stats()) { _internal_mutable_stats()->::substrait::RelCommon_Hint_Stats::MergeFrom(from._internal_stats()); } @@ -5140,7 +5374,14 @@ bool RelCommon_Hint::IsInitialized() const { void RelCommon_Hint::InternalSwap(RelCommon_Hint* other) { using std::swap; + auto* lhs_arena = GetArenaForAllocation(); + auto* rhs_arena = other->GetArenaForAllocation(); _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + &alias_, lhs_arena, + &other->alias_, rhs_arena + ); ::PROTOBUF_NAMESPACE_ID::internal::memswap< PROTOBUF_FIELD_OFFSET(RelCommon_Hint, advanced_extension_) + sizeof(RelCommon_Hint::advanced_extension_) @@ -6336,6 +6577,429 @@ ::PROTOBUF_NAMESPACE_ID::Metadata ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions // =================================================================== +class ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_Internal { + public: + using HasBits = decltype(std::declval()._has_bits_); + static void set_has_value_treated_as_null(HasBits* has_bits) { + (*has_bits)[0] |= 1u; + } +}; + +ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) +} +ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from) + : ::PROTOBUF_NAMESPACE_ID::Message(), + _has_bits_(from._has_bits_) { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + field_delimiter_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + field_delimiter_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (!from._internal_field_delimiter().empty()) { + field_delimiter_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_field_delimiter(), + GetArenaForAllocation()); + } + quote_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + quote_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (!from._internal_quote().empty()) { + quote_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_quote(), + GetArenaForAllocation()); + } + escape_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + escape_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (!from._internal_escape().empty()) { + escape_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_escape(), + GetArenaForAllocation()); + } + value_treated_as_null_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + value_treated_as_null_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (from._internal_has_value_treated_as_null()) { + value_treated_as_null_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_value_treated_as_null(), + GetArenaForAllocation()); + } + ::memcpy(&max_line_size_, &from.max_line_size_, + static_cast(reinterpret_cast(&header_lines_to_skip_) - + reinterpret_cast(&max_line_size_)) + sizeof(header_lines_to_skip_)); + // @@protoc_insertion_point(copy_constructor:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) +} + +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::SharedCtor() { +field_delimiter_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + field_delimiter_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +quote_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + quote_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +escape_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + escape_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +value_treated_as_null_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + value_treated_as_null_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&max_line_size_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&header_lines_to_skip_) - + reinterpret_cast(&max_line_size_)) + sizeof(header_lines_to_skip_)); +} + +ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::~ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions() { + // @@protoc_insertion_point(destructor:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + field_delimiter_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + quote_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + escape_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + value_treated_as_null_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::ArenaDtor(void* object) { + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* _this = reinterpret_cast< ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* >(object); + (void)_this; +} +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + field_delimiter_.ClearToEmpty(); + quote_.ClearToEmpty(); + escape_.ClearToEmpty(); + cached_has_bits = _has_bits_[0]; + if (cached_has_bits & 0x00000001u) { + value_treated_as_null_.ClearNonDefaultToEmpty(); + } + ::memset(&max_line_size_, 0, static_cast( + reinterpret_cast(&header_lines_to_skip_) - + reinterpret_cast(&max_line_size_)) + sizeof(header_lines_to_skip_)); + _has_bits_.Clear(); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + _Internal::HasBits has_bits{}; + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // string field_delimiter = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + auto str = _internal_mutable_field_delimiter(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(::PROTOBUF_NAMESPACE_ID::internal::VerifyUTF8(str, "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter")); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint64 max_line_size = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + max_line_size_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // string quote = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 26)) { + auto str = _internal_mutable_quote(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(::PROTOBUF_NAMESPACE_ID::internal::VerifyUTF8(str, "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote")); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint64 header_lines_to_skip = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { + header_lines_to_skip_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // string escape = 5; + case 5: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 42)) { + auto str = _internal_mutable_escape(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(::PROTOBUF_NAMESPACE_ID::internal::VerifyUTF8(str, "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape")); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // optional string value_treated_as_null = 6; + case 6: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 50)) { + auto str = _internal_mutable_value_treated_as_null(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(::PROTOBUF_NAMESPACE_ID::internal::VerifyUTF8(str, "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null")); + CHK_(ptr); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + _has_bits_.Or(has_bits); + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // string field_delimiter = 1; + if (!this->_internal_field_delimiter().empty()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_field_delimiter().data(), static_cast(this->_internal_field_delimiter().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter"); + target = stream->WriteStringMaybeAliased( + 1, this->_internal_field_delimiter(), target); + } + + // uint64 max_line_size = 2; + if (this->_internal_max_line_size() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(2, this->_internal_max_line_size(), target); + } + + // string quote = 3; + if (!this->_internal_quote().empty()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_quote().data(), static_cast(this->_internal_quote().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote"); + target = stream->WriteStringMaybeAliased( + 3, this->_internal_quote(), target); + } + + // uint64 header_lines_to_skip = 4; + if (this->_internal_header_lines_to_skip() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(4, this->_internal_header_lines_to_skip(), target); + } + + // string escape = 5; + if (!this->_internal_escape().empty()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_escape().data(), static_cast(this->_internal_escape().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape"); + target = stream->WriteStringMaybeAliased( + 5, this->_internal_escape(), target); + } + + // optional string value_treated_as_null = 6; + if (_internal_has_value_treated_as_null()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_value_treated_as_null().data(), static_cast(this->_internal_value_treated_as_null().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null"); + target = stream->WriteStringMaybeAliased( + 6, this->_internal_value_treated_as_null(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + return target; +} + +size_t ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string field_delimiter = 1; + if (!this->_internal_field_delimiter().empty()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_field_delimiter()); + } + + // string quote = 3; + if (!this->_internal_quote().empty()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_quote()); + } + + // string escape = 5; + if (!this->_internal_escape().empty()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_escape()); + } + + // optional string value_treated_as_null = 6; + cached_has_bits = _has_bits_[0]; + if (cached_has_bits & 0x00000001u) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_value_treated_as_null()); + } + + // uint64 max_line_size = 2; + if (this->_internal_max_line_size() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64SizePlusOne(this->_internal_max_line_size()); + } + + // uint64 header_lines_to_skip = 4; + if (this->_internal_header_lines_to_skip() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64SizePlusOne(this->_internal_header_lines_to_skip()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::GetClassData() const { return &_class_data_; } + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::MergeFrom(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (!from._internal_field_delimiter().empty()) { + _internal_set_field_delimiter(from._internal_field_delimiter()); + } + if (!from._internal_quote().empty()) { + _internal_set_quote(from._internal_quote()); + } + if (!from._internal_escape().empty()) { + _internal_set_escape(from._internal_escape()); + } + if (from._internal_has_value_treated_as_null()) { + _internal_set_value_treated_as_null(from._internal_value_treated_as_null()); + } + if (from._internal_max_line_size() != 0) { + _internal_set_max_line_size(from._internal_max_line_size()); + } + if (from._internal_header_lines_to_skip() != 0) { + _internal_set_header_lines_to_skip(from._internal_header_lines_to_skip()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::CopyFrom(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::IsInitialized() const { + return true; +} + +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::InternalSwap(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* other) { + using std::swap; + auto* lhs_arena = GetArenaForAllocation(); + auto* rhs_arena = other->GetArenaForAllocation(); + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + swap(_has_bits_[0], other->_has_bits_[0]); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + &field_delimiter_, lhs_arena, + &other->field_delimiter_, rhs_arena + ); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + "e_, lhs_arena, + &other->quote_, rhs_arena + ); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + &escape_, lhs_arena, + &other->escape_, rhs_arena + ); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + &value_treated_as_null_, lhs_arena, + &other->value_treated_as_null_, rhs_arena + ); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, header_lines_to_skip_) + + sizeof(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::header_lines_to_skip_) + - PROTOBUF_FIELD_OFFSET(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions, max_line_size_)>( + reinterpret_cast(&max_line_size_), + reinterpret_cast(&other->max_line_size_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, + file_level_metadata_substrait_2falgebra_2eproto[13]); +} + +// =================================================================== + class ReadRel_LocalFiles_FileOrFiles::_Internal { public: static const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& parquet(const ReadRel_LocalFiles_FileOrFiles* msg); @@ -6343,6 +7007,7 @@ class ReadRel_LocalFiles_FileOrFiles::_Internal { static const ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions& orc(const ReadRel_LocalFiles_FileOrFiles* msg); static const ::PROTOBUF_NAMESPACE_ID::Any& extension(const ReadRel_LocalFiles_FileOrFiles* msg); static const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& dwrf(const ReadRel_LocalFiles_FileOrFiles* msg); + static const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& text(const ReadRel_LocalFiles_FileOrFiles* msg); }; const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& @@ -6365,6 +7030,10 @@ const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& ReadRel_LocalFiles_FileOrFiles::_Internal::dwrf(const ReadRel_LocalFiles_FileOrFiles* msg) { return *msg->file_format_.dwrf_; } +const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& +ReadRel_LocalFiles_FileOrFiles::_Internal::text(const ReadRel_LocalFiles_FileOrFiles* msg) { + return *msg->file_format_.text_; +} void ReadRel_LocalFiles_FileOrFiles::set_allocated_parquet(::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_file_format(); @@ -6450,6 +7119,21 @@ void ReadRel_LocalFiles_FileOrFiles::set_allocated_dwrf(::substrait::ReadRel_Loc } // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.dwrf) } +void ReadRel_LocalFiles_FileOrFiles::set_allocated_text(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* text) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_file_format(); + if (text) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions>::GetOwningArena(text); + if (message_arena != submessage_arena) { + text = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, text, submessage_arena); + } + set_has_text(); + file_format_.text_ = text; + } + // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.text) +} ReadRel_LocalFiles_FileOrFiles::ReadRel_LocalFiles_FileOrFiles(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { @@ -6509,6 +7193,10 @@ ReadRel_LocalFiles_FileOrFiles::ReadRel_LocalFiles_FileOrFiles(const ReadRel_Loc _internal_mutable_dwrf()->::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions::MergeFrom(from._internal_dwrf()); break; } + case kText: { + _internal_mutable_text()->::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::MergeFrom(from._internal_text()); + break; + } case FILE_FORMAT_NOT_SET: { break; } @@ -6611,6 +7299,12 @@ void ReadRel_LocalFiles_FileOrFiles::clear_file_format() { } break; } + case kText: { + if (GetArenaForAllocation() == nullptr) { + delete file_format_.text_; + } + break; + } case FILE_FORMAT_NOT_SET: { break; } @@ -6743,6 +7437,14 @@ const char* ReadRel_LocalFiles_FileOrFiles::_InternalParse(const char* ptr, ::PR } else goto handle_unusual; continue; + // .substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions text = 14; + case 14: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 114)) { + ptr = ctx->ParseMessage(_internal_mutable_text(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; default: goto handle_unusual; } // switch @@ -6870,6 +7572,14 @@ uint8_t* ReadRel_LocalFiles_FileOrFiles::_InternalSerialize( 13, _Internal::dwrf(this), target, stream); } + // .substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions text = 14; + if (_internal_has_text()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 14, _Internal::text(this), target, stream); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -6970,6 +7680,13 @@ size_t ReadRel_LocalFiles_FileOrFiles::ByteSizeLong() const { *file_format_.dwrf_); break; } + // .substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions text = 14; + case kText: { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *file_format_.text_); + break; + } case FILE_FORMAT_NOT_SET: { break; } @@ -7047,6 +7764,10 @@ void ReadRel_LocalFiles_FileOrFiles::MergeFrom(const ReadRel_LocalFiles_FileOrFi _internal_mutable_dwrf()->::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions::MergeFrom(from._internal_dwrf()); break; } + case kText: { + _internal_mutable_text()->::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::MergeFrom(from._internal_text()); + break; + } case FILE_FORMAT_NOT_SET: { break; } @@ -7083,7 +7804,7 @@ void ReadRel_LocalFiles_FileOrFiles::InternalSwap(ReadRel_LocalFiles_FileOrFiles ::PROTOBUF_NAMESPACE_ID::Metadata ReadRel_LocalFiles_FileOrFiles::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[13]); + file_level_metadata_substrait_2falgebra_2eproto[14]); } // =================================================================== @@ -7317,7 +8038,7 @@ void ReadRel_LocalFiles::InternalSwap(ReadRel_LocalFiles* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ReadRel_LocalFiles::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[14]); + file_level_metadata_substrait_2falgebra_2eproto[15]); } // =================================================================== @@ -7999,7 +8720,7 @@ void ReadRel::InternalSwap(ReadRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ReadRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[15]); + file_level_metadata_substrait_2falgebra_2eproto[16]); } // =================================================================== @@ -8323,7 +9044,7 @@ void ProjectRel::InternalSwap(ProjectRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ProjectRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[16]); + file_level_metadata_substrait_2falgebra_2eproto[17]); } // =================================================================== @@ -8764,7 +9485,7 @@ void JoinRel::InternalSwap(JoinRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata JoinRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[17]); + file_level_metadata_substrait_2falgebra_2eproto[18]); } // =================================================================== @@ -9096,7 +9817,7 @@ void CrossRel::InternalSwap(CrossRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata CrossRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[18]); + file_level_metadata_substrait_2falgebra_2eproto[19]); } // =================================================================== @@ -9437,7 +10158,7 @@ void FetchRel::InternalSwap(FetchRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata FetchRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[19]); + file_level_metadata_substrait_2falgebra_2eproto[20]); } // =================================================================== @@ -9622,7 +10343,7 @@ void AggregateRel_Grouping::InternalSwap(AggregateRel_Grouping* other) { ::PROTOBUF_NAMESPACE_ID::Metadata AggregateRel_Grouping::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[20]); + file_level_metadata_substrait_2falgebra_2eproto[21]); } // =================================================================== @@ -9866,7 +10587,7 @@ void AggregateRel_Measure::InternalSwap(AggregateRel_Measure* other) { ::PROTOBUF_NAMESPACE_ID::Metadata AggregateRel_Measure::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[21]); + file_level_metadata_substrait_2falgebra_2eproto[22]); } // =================================================================== @@ -10223,7 +10944,7 @@ void AggregateRel::InternalSwap(AggregateRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata AggregateRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[22]); + file_level_metadata_substrait_2falgebra_2eproto[23]); } // =================================================================== @@ -10683,7 +11404,7 @@ void ConsistentPartitionWindowRel_WindowRelFunction::InternalSwap(ConsistentPart ::PROTOBUF_NAMESPACE_ID::Metadata ConsistentPartitionWindowRel_WindowRelFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[23]); + file_level_metadata_substrait_2falgebra_2eproto[24]); } // =================================================================== @@ -11073,7 +11794,7 @@ void ConsistentPartitionWindowRel::InternalSwap(ConsistentPartitionWindowRel* ot ::PROTOBUF_NAMESPACE_ID::Metadata ConsistentPartitionWindowRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[24]); + file_level_metadata_substrait_2falgebra_2eproto[25]); } // =================================================================== @@ -11397,7 +12118,7 @@ void SortRel::InternalSwap(SortRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata SortRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[25]); + file_level_metadata_substrait_2falgebra_2eproto[26]); } // =================================================================== @@ -11729,7 +12450,7 @@ void FilterRel::InternalSwap(FilterRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata FilterRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[26]); + file_level_metadata_substrait_2falgebra_2eproto[27]); } // =================================================================== @@ -12039,7 +12760,7 @@ void SetRel::InternalSwap(SetRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata SetRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[27]); + file_level_metadata_substrait_2falgebra_2eproto[28]); } // =================================================================== @@ -12330,7 +13051,7 @@ void ExtensionSingleRel::InternalSwap(ExtensionSingleRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExtensionSingleRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[28]); + file_level_metadata_substrait_2falgebra_2eproto[29]); } // =================================================================== @@ -12580,7 +13301,7 @@ void ExtensionLeafRel::InternalSwap(ExtensionLeafRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExtensionLeafRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[29]); + file_level_metadata_substrait_2falgebra_2eproto[30]); } // =================================================================== @@ -12863,7 +13584,7 @@ void ExtensionMultiRel::InternalSwap(ExtensionMultiRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExtensionMultiRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[30]); + file_level_metadata_substrait_2falgebra_2eproto[31]); } // =================================================================== @@ -13048,7 +13769,7 @@ void ExchangeRel_ScatterFields::InternalSwap(ExchangeRel_ScatterFields* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_ScatterFields::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[31]); + file_level_metadata_substrait_2falgebra_2eproto[32]); } // =================================================================== @@ -13243,7 +13964,7 @@ void ExchangeRel_SingleBucketExpression::InternalSwap(ExchangeRel_SingleBucketEx ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_SingleBucketExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[32]); + file_level_metadata_substrait_2falgebra_2eproto[33]); } // =================================================================== @@ -13470,7 +14191,7 @@ void ExchangeRel_MultiBucketExpression::InternalSwap(ExchangeRel_MultiBucketExpr ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_MultiBucketExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[33]); + file_level_metadata_substrait_2falgebra_2eproto[34]); } // =================================================================== @@ -13509,7 +14230,7 @@ const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ExchangeRel_Broadcast::GetClas ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_Broadcast::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[34]); + file_level_metadata_substrait_2falgebra_2eproto[35]); } // =================================================================== @@ -13687,7 +14408,7 @@ void ExchangeRel_RoundRobin::InternalSwap(ExchangeRel_RoundRobin* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_RoundRobin::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[35]); + file_level_metadata_substrait_2falgebra_2eproto[36]); } // =================================================================== @@ -14019,7 +14740,7 @@ void ExchangeRel_ExchangeTarget::InternalSwap(ExchangeRel_ExchangeTarget* other) ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel_ExchangeTarget::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[36]); + file_level_metadata_substrait_2falgebra_2eproto[37]); } // =================================================================== @@ -14686,7 +15407,7 @@ void ExchangeRel::InternalSwap(ExchangeRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExchangeRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[37]); + file_level_metadata_substrait_2falgebra_2eproto[38]); } // =================================================================== @@ -14986,7 +15707,7 @@ void ExpandRel_ExpandField::InternalSwap(ExpandRel_ExpandField* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExpandRel_ExpandField::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[38]); + file_level_metadata_substrait_2falgebra_2eproto[39]); } // =================================================================== @@ -15171,7 +15892,7 @@ void ExpandRel_SwitchingField::InternalSwap(ExpandRel_SwitchingField* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExpandRel_SwitchingField::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[39]); + file_level_metadata_substrait_2falgebra_2eproto[40]); } // =================================================================== @@ -15448,7 +16169,7 @@ void ExpandRel::InternalSwap(ExpandRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExpandRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[40]); + file_level_metadata_substrait_2falgebra_2eproto[41]); } // =================================================================== @@ -15681,7 +16402,7 @@ void RelRoot::InternalSwap(RelRoot* other) { ::PROTOBUF_NAMESPACE_ID::Metadata RelRoot::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[41]); + file_level_metadata_substrait_2falgebra_2eproto[42]); } // =================================================================== @@ -15706,8 +16427,8 @@ class Rel::_Internal { static const ::substrait::HashJoinRel& hash_join(const Rel* msg); static const ::substrait::MergeJoinRel& merge_join(const Rel* msg); static const ::substrait::NestedLoopJoinRel& nested_loop_join(const Rel* msg); - static const ::substrait::MarkJoinRel& mark_join(const Rel* msg); - static const ::substrait::DelimiterJoinRel& delimiter_join(const Rel* msg); + static const ::substrait::DuplicateEliminatedGetRel& duplicate_eliminated_get(const Rel* msg); + static const ::substrait::DuplicateEliminatedJoinRel& duplicate_eliminated_join(const Rel* msg); static const ::substrait::ConsistentPartitionWindowRel& window(const Rel* msg); static const ::substrait::ExchangeRel& exchange(const Rel* msg); static const ::substrait::ExpandRel& expand(const Rel* msg); @@ -15785,13 +16506,13 @@ const ::substrait::NestedLoopJoinRel& Rel::_Internal::nested_loop_join(const Rel* msg) { return *msg->rel_type_.nested_loop_join_; } -const ::substrait::MarkJoinRel& -Rel::_Internal::mark_join(const Rel* msg) { - return *msg->rel_type_.mark_join_; +const ::substrait::DuplicateEliminatedGetRel& +Rel::_Internal::duplicate_eliminated_get(const Rel* msg) { + return *msg->rel_type_.duplicate_eliminated_get_; } -const ::substrait::DelimiterJoinRel& -Rel::_Internal::delimiter_join(const Rel* msg) { - return *msg->rel_type_.delimiter_join_; +const ::substrait::DuplicateEliminatedJoinRel& +Rel::_Internal::duplicate_eliminated_join(const Rel* msg) { + return *msg->rel_type_.duplicate_eliminated_join_; } const ::substrait::ConsistentPartitionWindowRel& Rel::_Internal::window(const Rel* msg) { @@ -16075,35 +16796,35 @@ void Rel::set_allocated_nested_loop_join(::substrait::NestedLoopJoinRel* nested_ } // @@protoc_insertion_point(field_set_allocated:substrait.Rel.nested_loop_join) } -void Rel::set_allocated_mark_join(::substrait::MarkJoinRel* mark_join) { +void Rel::set_allocated_duplicate_eliminated_get(::substrait::DuplicateEliminatedGetRel* duplicate_eliminated_get) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_rel_type(); - if (mark_join) { + if (duplicate_eliminated_get) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::MarkJoinRel>::GetOwningArena(mark_join); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DuplicateEliminatedGetRel>::GetOwningArena(duplicate_eliminated_get); if (message_arena != submessage_arena) { - mark_join = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, mark_join, submessage_arena); + duplicate_eliminated_get = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, duplicate_eliminated_get, submessage_arena); } - set_has_mark_join(); - rel_type_.mark_join_ = mark_join; + set_has_duplicate_eliminated_get(); + rel_type_.duplicate_eliminated_get_ = duplicate_eliminated_get; } - // @@protoc_insertion_point(field_set_allocated:substrait.Rel.mark_join) + // @@protoc_insertion_point(field_set_allocated:substrait.Rel.duplicate_eliminated_get) } -void Rel::set_allocated_delimiter_join(::substrait::DelimiterJoinRel* delimiter_join) { +void Rel::set_allocated_duplicate_eliminated_join(::substrait::DuplicateEliminatedJoinRel* duplicate_eliminated_join) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_rel_type(); - if (delimiter_join) { + if (duplicate_eliminated_join) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DelimiterJoinRel>::GetOwningArena(delimiter_join); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DuplicateEliminatedJoinRel>::GetOwningArena(duplicate_eliminated_join); if (message_arena != submessage_arena) { - delimiter_join = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, delimiter_join, submessage_arena); + duplicate_eliminated_join = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, duplicate_eliminated_join, submessage_arena); } - set_has_delimiter_join(); - rel_type_.delimiter_join_ = delimiter_join; + set_has_duplicate_eliminated_join(); + rel_type_.duplicate_eliminated_join_ = duplicate_eliminated_join; } - // @@protoc_insertion_point(field_set_allocated:substrait.Rel.delimiter_join) + // @@protoc_insertion_point(field_set_allocated:substrait.Rel.duplicate_eliminated_join) } void Rel::set_allocated_window(::substrait::ConsistentPartitionWindowRel* window) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); @@ -16236,12 +16957,12 @@ Rel::Rel(const Rel& from) _internal_mutable_nested_loop_join()->::substrait::NestedLoopJoinRel::MergeFrom(from._internal_nested_loop_join()); break; } - case kMarkJoin: { - _internal_mutable_mark_join()->::substrait::MarkJoinRel::MergeFrom(from._internal_mark_join()); + case kDuplicateEliminatedGet: { + _internal_mutable_duplicate_eliminated_get()->::substrait::DuplicateEliminatedGetRel::MergeFrom(from._internal_duplicate_eliminated_get()); break; } - case kDelimiterJoin: { - _internal_mutable_delimiter_join()->::substrait::DelimiterJoinRel::MergeFrom(from._internal_delimiter_join()); + case kDuplicateEliminatedJoin: { + _internal_mutable_duplicate_eliminated_join()->::substrait::DuplicateEliminatedJoinRel::MergeFrom(from._internal_duplicate_eliminated_join()); break; } case kWindow: { @@ -16402,15 +17123,15 @@ void Rel::clear_rel_type() { } break; } - case kMarkJoin: { + case kDuplicateEliminatedGet: { if (GetArenaForAllocation() == nullptr) { - delete rel_type_.mark_join_; + delete rel_type_.duplicate_eliminated_get_; } break; } - case kDelimiterJoin: { + case kDuplicateEliminatedJoin: { if (GetArenaForAllocation() == nullptr) { - delete rel_type_.delimiter_join_; + delete rel_type_.duplicate_eliminated_join_; } break; } @@ -16624,18 +17345,18 @@ const char* Rel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::intern } else goto handle_unusual; continue; - // .substrait.MarkJoinRel mark_join = 23; + // .substrait.DuplicateEliminatedGetRel duplicate_eliminated_get = 23; case 23: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 186)) { - ptr = ctx->ParseMessage(_internal_mutable_mark_join(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_duplicate_eliminated_get(), ptr); CHK_(ptr); } else goto handle_unusual; continue; - // .substrait.DelimiterJoinRel delimiter_join = 24; + // .substrait.DuplicateEliminatedJoinRel duplicate_eliminated_join = 24; case 24: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 194)) { - ptr = ctx->ParseMessage(_internal_mutable_delimiter_join(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_duplicate_eliminated_join(), ptr); CHK_(ptr); } else goto handle_unusual; @@ -16837,20 +17558,20 @@ uint8_t* Rel::_InternalSerialize( 21, _Internal::reference(this), target, stream); } - // .substrait.MarkJoinRel mark_join = 23; - if (_internal_has_mark_join()) { + // .substrait.DuplicateEliminatedGetRel duplicate_eliminated_get = 23; + if (_internal_has_duplicate_eliminated_get()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 23, _Internal::mark_join(this), target, stream); + 23, _Internal::duplicate_eliminated_get(this), target, stream); } - // .substrait.DelimiterJoinRel delimiter_join = 24; - if (_internal_has_delimiter_join()) { + // .substrait.DuplicateEliminatedJoinRel duplicate_eliminated_join = 24; + if (_internal_has_duplicate_eliminated_join()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 24, _Internal::delimiter_join(this), target, stream); + 24, _Internal::duplicate_eliminated_join(this), target, stream); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { @@ -16996,18 +17717,18 @@ size_t Rel::ByteSizeLong() const { *rel_type_.nested_loop_join_); break; } - // .substrait.MarkJoinRel mark_join = 23; - case kMarkJoin: { + // .substrait.DuplicateEliminatedGetRel duplicate_eliminated_get = 23; + case kDuplicateEliminatedGet: { total_size += 2 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *rel_type_.mark_join_); + *rel_type_.duplicate_eliminated_get_); break; } - // .substrait.DelimiterJoinRel delimiter_join = 24; - case kDelimiterJoin: { + // .substrait.DuplicateEliminatedJoinRel duplicate_eliminated_join = 24; + case kDuplicateEliminatedJoin: { total_size += 2 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *rel_type_.delimiter_join_); + *rel_type_.duplicate_eliminated_join_); break; } // .substrait.ConsistentPartitionWindowRel window = 17; @@ -17130,12 +17851,12 @@ void Rel::MergeFrom(const Rel& from) { _internal_mutable_nested_loop_join()->::substrait::NestedLoopJoinRel::MergeFrom(from._internal_nested_loop_join()); break; } - case kMarkJoin: { - _internal_mutable_mark_join()->::substrait::MarkJoinRel::MergeFrom(from._internal_mark_join()); + case kDuplicateEliminatedGet: { + _internal_mutable_duplicate_eliminated_get()->::substrait::DuplicateEliminatedGetRel::MergeFrom(from._internal_duplicate_eliminated_get()); break; } - case kDelimiterJoin: { - _internal_mutable_delimiter_join()->::substrait::DelimiterJoinRel::MergeFrom(from._internal_delimiter_join()); + case kDuplicateEliminatedJoin: { + _internal_mutable_duplicate_eliminated_join()->::substrait::DuplicateEliminatedJoinRel::MergeFrom(from._internal_duplicate_eliminated_join()); break; } case kWindow: { @@ -17178,7 +17899,7 @@ void Rel::InternalSwap(Rel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Rel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[42]); + file_level_metadata_substrait_2falgebra_2eproto[43]); } // =================================================================== @@ -17417,7 +18138,7 @@ void NamedObjectWrite::InternalSwap(NamedObjectWrite* other) { ::PROTOBUF_NAMESPACE_ID::Metadata NamedObjectWrite::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[43]); + file_level_metadata_substrait_2falgebra_2eproto[44]); } // =================================================================== @@ -17618,7 +18339,7 @@ void ExtensionObject::InternalSwap(ExtensionObject* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ExtensionObject::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[44]); + file_level_metadata_substrait_2falgebra_2eproto[45]); } // =================================================================== @@ -18154,7 +18875,7 @@ void DdlRel::InternalSwap(DdlRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata DdlRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[45]); + file_level_metadata_substrait_2falgebra_2eproto[46]); } // =================================================================== @@ -18649,7 +19370,7 @@ void WriteRel::InternalSwap(WriteRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata WriteRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[46]); + file_level_metadata_substrait_2falgebra_2eproto[47]); } // =================================================================== @@ -18900,7 +19621,7 @@ void ComparisonJoinKey_ComparisonType::InternalSwap(ComparisonJoinKey_Comparison ::PROTOBUF_NAMESPACE_ID::Metadata ComparisonJoinKey_ComparisonType::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[47]); + file_level_metadata_substrait_2falgebra_2eproto[48]); } // =================================================================== @@ -19185,7 +19906,7 @@ void ComparisonJoinKey::InternalSwap(ComparisonJoinKey* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ComparisonJoinKey::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[48]); + file_level_metadata_substrait_2falgebra_2eproto[49]); } // =================================================================== @@ -19684,7 +20405,7 @@ void HashJoinRel::InternalSwap(HashJoinRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata HashJoinRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[49]); + file_level_metadata_substrait_2falgebra_2eproto[50]); } // =================================================================== @@ -20183,7 +20904,7 @@ void MergeJoinRel::InternalSwap(MergeJoinRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata MergeJoinRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[50]); + file_level_metadata_substrait_2falgebra_2eproto[51]); } // =================================================================== @@ -20583,56 +21304,35 @@ void NestedLoopJoinRel::InternalSwap(NestedLoopJoinRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata NestedLoopJoinRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[51]); + file_level_metadata_substrait_2falgebra_2eproto[52]); } // =================================================================== -class MarkJoinRel::_Internal { +class DuplicateEliminatedGetRel::_Internal { public: - static const ::substrait::RelCommon& common(const MarkJoinRel* msg); - static const ::substrait::Rel& left(const MarkJoinRel* msg); - static const ::substrait::Rel& right(const MarkJoinRel* msg); - static const ::substrait::Expression& expression(const MarkJoinRel* msg); - static const ::substrait::extensions::AdvancedExtension& advanced_extension(const MarkJoinRel* msg); + static const ::substrait::RelCommon& common(const DuplicateEliminatedGetRel* msg); + static const ::substrait::ReferenceRel& input(const DuplicateEliminatedGetRel* msg); }; const ::substrait::RelCommon& -MarkJoinRel::_Internal::common(const MarkJoinRel* msg) { +DuplicateEliminatedGetRel::_Internal::common(const DuplicateEliminatedGetRel* msg) { return *msg->common_; } -const ::substrait::Rel& -MarkJoinRel::_Internal::left(const MarkJoinRel* msg) { - return *msg->left_; -} -const ::substrait::Rel& -MarkJoinRel::_Internal::right(const MarkJoinRel* msg) { - return *msg->right_; -} -const ::substrait::Expression& -MarkJoinRel::_Internal::expression(const MarkJoinRel* msg) { - return *msg->expression_; -} -const ::substrait::extensions::AdvancedExtension& -MarkJoinRel::_Internal::advanced_extension(const MarkJoinRel* msg) { - return *msg->advanced_extension_; -} -void MarkJoinRel::clear_advanced_extension() { - if (GetArenaForAllocation() == nullptr && advanced_extension_ != nullptr) { - delete advanced_extension_; - } - advanced_extension_ = nullptr; +const ::substrait::ReferenceRel& +DuplicateEliminatedGetRel::_Internal::input(const DuplicateEliminatedGetRel* msg) { + return *msg->input_; } -MarkJoinRel::MarkJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, +DuplicateEliminatedGetRel::DuplicateEliminatedGetRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.MarkJoinRel) + // @@protoc_insertion_point(arena_constructor:substrait.DuplicateEliminatedGetRel) } -MarkJoinRel::MarkJoinRel(const MarkJoinRel& from) +DuplicateEliminatedGetRel::DuplicateEliminatedGetRel(const DuplicateEliminatedGetRel& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); if (from._internal_has_common()) { @@ -20640,64 +21340,46 @@ MarkJoinRel::MarkJoinRel(const MarkJoinRel& from) } else { common_ = nullptr; } - if (from._internal_has_left()) { - left_ = new ::substrait::Rel(*from.left_); - } else { - left_ = nullptr; - } - if (from._internal_has_right()) { - right_ = new ::substrait::Rel(*from.right_); - } else { - right_ = nullptr; - } - if (from._internal_has_expression()) { - expression_ = new ::substrait::Expression(*from.expression_); - } else { - expression_ = nullptr; - } - if (from._internal_has_advanced_extension()) { - advanced_extension_ = new ::substrait::extensions::AdvancedExtension(*from.advanced_extension_); + if (from._internal_has_input()) { + input_ = new ::substrait::ReferenceRel(*from.input_); } else { - advanced_extension_ = nullptr; + input_ = nullptr; } - // @@protoc_insertion_point(copy_constructor:substrait.MarkJoinRel) + // @@protoc_insertion_point(copy_constructor:substrait.DuplicateEliminatedGetRel) } -inline void MarkJoinRel::SharedCtor() { +inline void DuplicateEliminatedGetRel::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&common_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(&advanced_extension_) - - reinterpret_cast(&common_)) + sizeof(advanced_extension_)); + 0, static_cast(reinterpret_cast(&input_) - + reinterpret_cast(&common_)) + sizeof(input_)); } -MarkJoinRel::~MarkJoinRel() { - // @@protoc_insertion_point(destructor:substrait.MarkJoinRel) +DuplicateEliminatedGetRel::~DuplicateEliminatedGetRel() { + // @@protoc_insertion_point(destructor:substrait.DuplicateEliminatedGetRel) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void MarkJoinRel::SharedDtor() { +inline void DuplicateEliminatedGetRel::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); if (this != internal_default_instance()) delete common_; - if (this != internal_default_instance()) delete left_; - if (this != internal_default_instance()) delete right_; - if (this != internal_default_instance()) delete expression_; - if (this != internal_default_instance()) delete advanced_extension_; + if (this != internal_default_instance()) delete input_; } -void MarkJoinRel::ArenaDtor(void* object) { - MarkJoinRel* _this = reinterpret_cast< MarkJoinRel* >(object); +void DuplicateEliminatedGetRel::ArenaDtor(void* object) { + DuplicateEliminatedGetRel* _this = reinterpret_cast< DuplicateEliminatedGetRel* >(object); (void)_this; } -void MarkJoinRel::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void DuplicateEliminatedGetRel::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void MarkJoinRel::SetCachedSize(int size) const { +void DuplicateEliminatedGetRel::SetCachedSize(int size) const { _cached_size_.Set(size); } -void MarkJoinRel::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.MarkJoinRel) +void DuplicateEliminatedGetRel::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.DuplicateEliminatedGetRel) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; @@ -20706,26 +21388,14 @@ void MarkJoinRel::Clear() { delete common_; } common_ = nullptr; - if (GetArenaForAllocation() == nullptr && left_ != nullptr) { - delete left_; - } - left_ = nullptr; - if (GetArenaForAllocation() == nullptr && right_ != nullptr) { - delete right_; - } - right_ = nullptr; - if (GetArenaForAllocation() == nullptr && expression_ != nullptr) { - delete expression_; - } - expression_ = nullptr; - if (GetArenaForAllocation() == nullptr && advanced_extension_ != nullptr) { - delete advanced_extension_; + if (GetArenaForAllocation() == nullptr && input_ != nullptr) { + delete input_; } - advanced_extension_ = nullptr; + input_ = nullptr; _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* MarkJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* DuplicateEliminatedGetRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; @@ -20739,34 +21409,10 @@ const char* MarkJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID } else goto handle_unusual; continue; - // .substrait.Rel left = 2; + // .substrait.ReferenceRel input = 2; case 2: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { - ptr = ctx->ParseMessage(_internal_mutable_left(), ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // .substrait.Rel right = 3; - case 3: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 26)) { - ptr = ctx->ParseMessage(_internal_mutable_right(), ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // .substrait.Expression expression = 4; - case 4: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 34)) { - ptr = ctx->ParseMessage(_internal_mutable_expression(), ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // .substrait.extensions.AdvancedExtension advanced_extension = 10; - case 10: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 82)) { - ptr = ctx->ParseMessage(_internal_mutable_advanced_extension(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_input(), ptr); CHK_(ptr); } else goto handle_unusual; @@ -20794,9 +21440,9 @@ const char* MarkJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID #undef CHK_ } -uint8_t* MarkJoinRel::_InternalSerialize( +uint8_t* DuplicateEliminatedGetRel::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.MarkJoinRel) + // @@protoc_insertion_point(serialize_to_array_start:substrait.DuplicateEliminatedGetRel) uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -20808,48 +21454,24 @@ uint8_t* MarkJoinRel::_InternalSerialize( 1, _Internal::common(this), target, stream); } - // .substrait.Rel left = 2; - if (this->_internal_has_left()) { - target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 2, _Internal::left(this), target, stream); - } - - // .substrait.Rel right = 3; - if (this->_internal_has_right()) { - target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 3, _Internal::right(this), target, stream); - } - - // .substrait.Expression expression = 4; - if (this->_internal_has_expression()) { - target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 4, _Internal::expression(this), target, stream); - } - - // .substrait.extensions.AdvancedExtension advanced_extension = 10; - if (this->_internal_has_advanced_extension()) { + // .substrait.ReferenceRel input = 2; + if (this->_internal_has_input()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 10, _Internal::advanced_extension(this), target, stream); + 2, _Internal::input(this), target, stream); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.MarkJoinRel) + // @@protoc_insertion_point(serialize_to_array_end:substrait.DuplicateEliminatedGetRel) return target; } -size_t MarkJoinRel::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.MarkJoinRel) +size_t DuplicateEliminatedGetRel::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DuplicateEliminatedGetRel) size_t total_size = 0; uint32_t cached_has_bits = 0; @@ -20863,52 +21485,31 @@ size_t MarkJoinRel::ByteSizeLong() const { *common_); } - // .substrait.Rel left = 2; - if (this->_internal_has_left()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *left_); - } - - // .substrait.Rel right = 3; - if (this->_internal_has_right()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *right_); - } - - // .substrait.Expression expression = 4; - if (this->_internal_has_expression()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *expression_); - } - - // .substrait.extensions.AdvancedExtension advanced_extension = 10; - if (this->_internal_has_advanced_extension()) { + // .substrait.ReferenceRel input = 2; + if (this->_internal_has_input()) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *advanced_extension_); + *input_); } return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData MarkJoinRel::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DuplicateEliminatedGetRel::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - MarkJoinRel::MergeImpl + DuplicateEliminatedGetRel::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*MarkJoinRel::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DuplicateEliminatedGetRel::GetClassData() const { return &_class_data_; } -void MarkJoinRel::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void DuplicateEliminatedGetRel::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void MarkJoinRel::MergeFrom(const MarkJoinRel& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.MarkJoinRel) +void DuplicateEliminatedGetRel::MergeFrom(const DuplicateEliminatedGetRel& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DuplicateEliminatedGetRel) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -20916,101 +21517,95 @@ void MarkJoinRel::MergeFrom(const MarkJoinRel& from) { if (from._internal_has_common()) { _internal_mutable_common()->::substrait::RelCommon::MergeFrom(from._internal_common()); } - if (from._internal_has_left()) { - _internal_mutable_left()->::substrait::Rel::MergeFrom(from._internal_left()); - } - if (from._internal_has_right()) { - _internal_mutable_right()->::substrait::Rel::MergeFrom(from._internal_right()); - } - if (from._internal_has_expression()) { - _internal_mutable_expression()->::substrait::Expression::MergeFrom(from._internal_expression()); - } - if (from._internal_has_advanced_extension()) { - _internal_mutable_advanced_extension()->::substrait::extensions::AdvancedExtension::MergeFrom(from._internal_advanced_extension()); + if (from._internal_has_input()) { + _internal_mutable_input()->::substrait::ReferenceRel::MergeFrom(from._internal_input()); } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void MarkJoinRel::CopyFrom(const MarkJoinRel& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.MarkJoinRel) +void DuplicateEliminatedGetRel::CopyFrom(const DuplicateEliminatedGetRel& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DuplicateEliminatedGetRel) if (&from == this) return; Clear(); MergeFrom(from); } -bool MarkJoinRel::IsInitialized() const { +bool DuplicateEliminatedGetRel::IsInitialized() const { return true; } -void MarkJoinRel::InternalSwap(MarkJoinRel* other) { +void DuplicateEliminatedGetRel::InternalSwap(DuplicateEliminatedGetRel* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(MarkJoinRel, advanced_extension_) - + sizeof(MarkJoinRel::advanced_extension_) - - PROTOBUF_FIELD_OFFSET(MarkJoinRel, common_)>( + PROTOBUF_FIELD_OFFSET(DuplicateEliminatedGetRel, input_) + + sizeof(DuplicateEliminatedGetRel::input_) + - PROTOBUF_FIELD_OFFSET(DuplicateEliminatedGetRel, common_)>( reinterpret_cast(&common_), reinterpret_cast(&other->common_)); } -::PROTOBUF_NAMESPACE_ID::Metadata MarkJoinRel::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata DuplicateEliminatedGetRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[52]); + file_level_metadata_substrait_2falgebra_2eproto[53]); } // =================================================================== -class DelimiterJoinRel::_Internal { +class DuplicateEliminatedJoinRel::_Internal { public: - static const ::substrait::RelCommon& common(const DelimiterJoinRel* msg); - static const ::substrait::Rel& left(const DelimiterJoinRel* msg); - static const ::substrait::Rel& right(const DelimiterJoinRel* msg); - static const ::substrait::Expression_FieldReference& delimiter_field(const DelimiterJoinRel* msg); - static const ::substrait::extensions::AdvancedExtension& advanced_extension(const DelimiterJoinRel* msg); + static const ::substrait::RelCommon& common(const DuplicateEliminatedJoinRel* msg); + static const ::substrait::Rel& left(const DuplicateEliminatedJoinRel* msg); + static const ::substrait::Rel& right(const DuplicateEliminatedJoinRel* msg); + static const ::substrait::Expression& expression(const DuplicateEliminatedJoinRel* msg); + static const ::substrait::Expression& post_join_filter(const DuplicateEliminatedJoinRel* msg); + static const ::substrait::extensions::AdvancedExtension& advanced_extension(const DuplicateEliminatedJoinRel* msg); }; const ::substrait::RelCommon& -DelimiterJoinRel::_Internal::common(const DelimiterJoinRel* msg) { +DuplicateEliminatedJoinRel::_Internal::common(const DuplicateEliminatedJoinRel* msg) { return *msg->common_; } const ::substrait::Rel& -DelimiterJoinRel::_Internal::left(const DelimiterJoinRel* msg) { +DuplicateEliminatedJoinRel::_Internal::left(const DuplicateEliminatedJoinRel* msg) { return *msg->left_; } const ::substrait::Rel& -DelimiterJoinRel::_Internal::right(const DelimiterJoinRel* msg) { +DuplicateEliminatedJoinRel::_Internal::right(const DuplicateEliminatedJoinRel* msg) { return *msg->right_; } -const ::substrait::Expression_FieldReference& -DelimiterJoinRel::_Internal::delimiter_field(const DelimiterJoinRel* msg) { - return *msg->delimiter_field_; +const ::substrait::Expression& +DuplicateEliminatedJoinRel::_Internal::expression(const DuplicateEliminatedJoinRel* msg) { + return *msg->expression_; +} +const ::substrait::Expression& +DuplicateEliminatedJoinRel::_Internal::post_join_filter(const DuplicateEliminatedJoinRel* msg) { + return *msg->post_join_filter_; } const ::substrait::extensions::AdvancedExtension& -DelimiterJoinRel::_Internal::advanced_extension(const DelimiterJoinRel* msg) { +DuplicateEliminatedJoinRel::_Internal::advanced_extension(const DuplicateEliminatedJoinRel* msg) { return *msg->advanced_extension_; } -void DelimiterJoinRel::clear_advanced_extension() { +void DuplicateEliminatedJoinRel::clear_advanced_extension() { if (GetArenaForAllocation() == nullptr && advanced_extension_ != nullptr) { delete advanced_extension_; } advanced_extension_ = nullptr; } -DelimiterJoinRel::DelimiterJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, +DuplicateEliminatedJoinRel::DuplicateEliminatedJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned), - left_keys_(arena), - right_keys_(arena) { + duplicate_eliminated_columns_(arena) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.DelimiterJoinRel) + // @@protoc_insertion_point(arena_constructor:substrait.DuplicateEliminatedJoinRel) } -DelimiterJoinRel::DelimiterJoinRel(const DelimiterJoinRel& from) +DuplicateEliminatedJoinRel::DuplicateEliminatedJoinRel(const DuplicateEliminatedJoinRel& from) : ::PROTOBUF_NAMESPACE_ID::Message(), - left_keys_(from.left_keys_), - right_keys_(from.right_keys_) { + duplicate_eliminated_columns_(from.duplicate_eliminated_columns_) { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); if (from._internal_has_common()) { common_ = new ::substrait::RelCommon(*from.common_); @@ -21027,61 +21622,68 @@ DelimiterJoinRel::DelimiterJoinRel(const DelimiterJoinRel& from) } else { right_ = nullptr; } - if (from._internal_has_delimiter_field()) { - delimiter_field_ = new ::substrait::Expression_FieldReference(*from.delimiter_field_); + if (from._internal_has_expression()) { + expression_ = new ::substrait::Expression(*from.expression_); + } else { + expression_ = nullptr; + } + if (from._internal_has_post_join_filter()) { + post_join_filter_ = new ::substrait::Expression(*from.post_join_filter_); } else { - delimiter_field_ = nullptr; + post_join_filter_ = nullptr; } if (from._internal_has_advanced_extension()) { advanced_extension_ = new ::substrait::extensions::AdvancedExtension(*from.advanced_extension_); } else { advanced_extension_ = nullptr; } - type_ = from.type_; - // @@protoc_insertion_point(copy_constructor:substrait.DelimiterJoinRel) + ::memcpy(&type_, &from.type_, + static_cast(reinterpret_cast(&duplicate_eliminated_side_) - + reinterpret_cast(&type_)) + sizeof(duplicate_eliminated_side_)); + // @@protoc_insertion_point(copy_constructor:substrait.DuplicateEliminatedJoinRel) } -inline void DelimiterJoinRel::SharedCtor() { +inline void DuplicateEliminatedJoinRel::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&common_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(&type_) - - reinterpret_cast(&common_)) + sizeof(type_)); + 0, static_cast(reinterpret_cast(&duplicate_eliminated_side_) - + reinterpret_cast(&common_)) + sizeof(duplicate_eliminated_side_)); } -DelimiterJoinRel::~DelimiterJoinRel() { - // @@protoc_insertion_point(destructor:substrait.DelimiterJoinRel) +DuplicateEliminatedJoinRel::~DuplicateEliminatedJoinRel() { + // @@protoc_insertion_point(destructor:substrait.DuplicateEliminatedJoinRel) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void DelimiterJoinRel::SharedDtor() { +inline void DuplicateEliminatedJoinRel::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); if (this != internal_default_instance()) delete common_; if (this != internal_default_instance()) delete left_; if (this != internal_default_instance()) delete right_; - if (this != internal_default_instance()) delete delimiter_field_; + if (this != internal_default_instance()) delete expression_; + if (this != internal_default_instance()) delete post_join_filter_; if (this != internal_default_instance()) delete advanced_extension_; } -void DelimiterJoinRel::ArenaDtor(void* object) { - DelimiterJoinRel* _this = reinterpret_cast< DelimiterJoinRel* >(object); +void DuplicateEliminatedJoinRel::ArenaDtor(void* object) { + DuplicateEliminatedJoinRel* _this = reinterpret_cast< DuplicateEliminatedJoinRel* >(object); (void)_this; } -void DelimiterJoinRel::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void DuplicateEliminatedJoinRel::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void DelimiterJoinRel::SetCachedSize(int size) const { +void DuplicateEliminatedJoinRel::SetCachedSize(int size) const { _cached_size_.Set(size); } -void DelimiterJoinRel::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.DelimiterJoinRel) +void DuplicateEliminatedJoinRel::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.DuplicateEliminatedJoinRel) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - left_keys_.Clear(); - right_keys_.Clear(); + duplicate_eliminated_columns_.Clear(); if (GetArenaForAllocation() == nullptr && common_ != nullptr) { delete common_; } @@ -21094,19 +21696,25 @@ void DelimiterJoinRel::Clear() { delete right_; } right_ = nullptr; - if (GetArenaForAllocation() == nullptr && delimiter_field_ != nullptr) { - delete delimiter_field_; + if (GetArenaForAllocation() == nullptr && expression_ != nullptr) { + delete expression_; } - delimiter_field_ = nullptr; + expression_ = nullptr; + if (GetArenaForAllocation() == nullptr && post_join_filter_ != nullptr) { + delete post_join_filter_; + } + post_join_filter_ = nullptr; if (GetArenaForAllocation() == nullptr && advanced_extension_ != nullptr) { delete advanced_extension_; } advanced_extension_ = nullptr; - type_ = 0; + ::memset(&type_, 0, static_cast( + reinterpret_cast(&duplicate_eliminated_side_) - + reinterpret_cast(&type_)) + sizeof(duplicate_eliminated_side_)); _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* DelimiterJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* DuplicateEliminatedJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; @@ -21136,46 +21744,50 @@ const char* DelimiterJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPA } else goto handle_unusual; continue; - // repeated .substrait.Expression.FieldReference left_keys = 4; + // .substrait.Expression expression = 4; case 4: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 34)) { - ptr -= 1; - do { - ptr += 1; - ptr = ctx->ParseMessage(_internal_add_left_keys(), ptr); - CHK_(ptr); - if (!ctx->DataAvailable(ptr)) break; - } while (::PROTOBUF_NAMESPACE_ID::internal::ExpectTag<34>(ptr)); + ptr = ctx->ParseMessage(_internal_mutable_expression(), ptr); + CHK_(ptr); } else goto handle_unusual; continue; - // repeated .substrait.Expression.FieldReference right_keys = 5; + // .substrait.Expression post_join_filter = 5; case 5: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 42)) { - ptr -= 1; - do { - ptr += 1; - ptr = ctx->ParseMessage(_internal_add_right_keys(), ptr); - CHK_(ptr); - if (!ctx->DataAvailable(ptr)) break; - } while (::PROTOBUF_NAMESPACE_ID::internal::ExpectTag<42>(ptr)); + ptr = ctx->ParseMessage(_internal_mutable_post_join_filter(), ptr); + CHK_(ptr); } else goto handle_unusual; continue; - // .substrait.DelimiterJoinRel.JoinType type = 6; + // .substrait.DuplicateEliminatedJoinRel.JoinType type = 6; case 6: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 48)) { uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); - _internal_set_type(static_cast<::substrait::DelimiterJoinRel_JoinType>(val)); + _internal_set_type(static_cast<::substrait::DuplicateEliminatedJoinRel_JoinType>(val)); } else goto handle_unusual; continue; - // .substrait.Expression.FieldReference delimiter_field = 7; + // repeated .substrait.Expression.FieldReference duplicate_eliminated_columns = 7; case 7: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 58)) { - ptr = ctx->ParseMessage(_internal_mutable_delimiter_field(), ptr); + ptr -= 1; + do { + ptr += 1; + ptr = ctx->ParseMessage(_internal_add_duplicate_eliminated_columns(), ptr); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::ExpectTag<58>(ptr)); + } else + goto handle_unusual; + continue; + // .substrait.DuplicateEliminatedJoinRel.DuplicateEliminatedSide duplicate_eliminated_side = 8; + case 8: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 64)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); + _internal_set_duplicate_eliminated_side(static_cast<::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide>(val)); } else goto handle_unusual; continue; @@ -21210,9 +21822,9 @@ const char* DelimiterJoinRel::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPA #undef CHK_ } -uint8_t* DelimiterJoinRel::_InternalSerialize( +uint8_t* DuplicateEliminatedJoinRel::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.DelimiterJoinRel) + // @@protoc_insertion_point(serialize_to_array_start:substrait.DuplicateEliminatedJoinRel) uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -21240,35 +21852,42 @@ uint8_t* DelimiterJoinRel::_InternalSerialize( 3, _Internal::right(this), target, stream); } - // repeated .substrait.Expression.FieldReference left_keys = 4; - for (unsigned int i = 0, - n = static_cast(this->_internal_left_keys_size()); i < n; i++) { + // .substrait.Expression expression = 4; + if (this->_internal_has_expression()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage(4, this->_internal_left_keys(i), target, stream); + InternalWriteMessage( + 4, _Internal::expression(this), target, stream); } - // repeated .substrait.Expression.FieldReference right_keys = 5; - for (unsigned int i = 0, - n = static_cast(this->_internal_right_keys_size()); i < n; i++) { + // .substrait.Expression post_join_filter = 5; + if (this->_internal_has_post_join_filter()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage(5, this->_internal_right_keys(i), target, stream); + InternalWriteMessage( + 5, _Internal::post_join_filter(this), target, stream); } - // .substrait.DelimiterJoinRel.JoinType type = 6; + // .substrait.DuplicateEliminatedJoinRel.JoinType type = 6; if (this->_internal_type() != 0) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( 6, this->_internal_type(), target); } - // .substrait.Expression.FieldReference delimiter_field = 7; - if (this->_internal_has_delimiter_field()) { + // repeated .substrait.Expression.FieldReference duplicate_eliminated_columns = 7; + for (unsigned int i = 0, + n = static_cast(this->_internal_duplicate_eliminated_columns_size()); i < n; i++) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 7, _Internal::delimiter_field(this), target, stream); + InternalWriteMessage(7, this->_internal_duplicate_eliminated_columns(i), target, stream); + } + + // .substrait.DuplicateEliminatedJoinRel.DuplicateEliminatedSide duplicate_eliminated_side = 8; + if (this->_internal_duplicate_eliminated_side() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 8, this->_internal_duplicate_eliminated_side(), target); } // .substrait.extensions.AdvancedExtension advanced_extension = 10; @@ -21283,28 +21902,21 @@ uint8_t* DelimiterJoinRel::_InternalSerialize( target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.DelimiterJoinRel) + // @@protoc_insertion_point(serialize_to_array_end:substrait.DuplicateEliminatedJoinRel) return target; } -size_t DelimiterJoinRel::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.DelimiterJoinRel) +size_t DuplicateEliminatedJoinRel::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DuplicateEliminatedJoinRel) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // repeated .substrait.Expression.FieldReference left_keys = 4; - total_size += 1UL * this->_internal_left_keys_size(); - for (const auto& msg : this->left_keys_) { - total_size += - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize(msg); - } - - // repeated .substrait.Expression.FieldReference right_keys = 5; - total_size += 1UL * this->_internal_right_keys_size(); - for (const auto& msg : this->right_keys_) { + // repeated .substrait.Expression.FieldReference duplicate_eliminated_columns = 7; + total_size += 1UL * this->_internal_duplicate_eliminated_columns_size(); + for (const auto& msg : this->duplicate_eliminated_columns_) { total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize(msg); } @@ -21330,11 +21942,18 @@ size_t DelimiterJoinRel::ByteSizeLong() const { *right_); } - // .substrait.Expression.FieldReference delimiter_field = 7; - if (this->_internal_has_delimiter_field()) { + // .substrait.Expression expression = 4; + if (this->_internal_has_expression()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *expression_); + } + + // .substrait.Expression post_join_filter = 5; + if (this->_internal_has_post_join_filter()) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *delimiter_field_); + *post_join_filter_); } // .substrait.extensions.AdvancedExtension advanced_extension = 10; @@ -21344,36 +21963,41 @@ size_t DelimiterJoinRel::ByteSizeLong() const { *advanced_extension_); } - // .substrait.DelimiterJoinRel.JoinType type = 6; + // .substrait.DuplicateEliminatedJoinRel.JoinType type = 6; if (this->_internal_type() != 0) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_type()); } + // .substrait.DuplicateEliminatedJoinRel.DuplicateEliminatedSide duplicate_eliminated_side = 8; + if (this->_internal_duplicate_eliminated_side() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_duplicate_eliminated_side()); + } + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DelimiterJoinRel::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DuplicateEliminatedJoinRel::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - DelimiterJoinRel::MergeImpl + DuplicateEliminatedJoinRel::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DelimiterJoinRel::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DuplicateEliminatedJoinRel::GetClassData() const { return &_class_data_; } -void DelimiterJoinRel::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void DuplicateEliminatedJoinRel::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void DelimiterJoinRel::MergeFrom(const DelimiterJoinRel& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DelimiterJoinRel) +void DuplicateEliminatedJoinRel::MergeFrom(const DuplicateEliminatedJoinRel& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DuplicateEliminatedJoinRel) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; - left_keys_.MergeFrom(from.left_keys_); - right_keys_.MergeFrom(from.right_keys_); + duplicate_eliminated_columns_.MergeFrom(from.duplicate_eliminated_columns_); if (from._internal_has_common()) { _internal_mutable_common()->::substrait::RelCommon::MergeFrom(from._internal_common()); } @@ -21383,8 +22007,11 @@ void DelimiterJoinRel::MergeFrom(const DelimiterJoinRel& from) { if (from._internal_has_right()) { _internal_mutable_right()->::substrait::Rel::MergeFrom(from._internal_right()); } - if (from._internal_has_delimiter_field()) { - _internal_mutable_delimiter_field()->::substrait::Expression_FieldReference::MergeFrom(from._internal_delimiter_field()); + if (from._internal_has_expression()) { + _internal_mutable_expression()->::substrait::Expression::MergeFrom(from._internal_expression()); + } + if (from._internal_has_post_join_filter()) { + _internal_mutable_post_join_filter()->::substrait::Expression::MergeFrom(from._internal_post_join_filter()); } if (from._internal_has_advanced_extension()) { _internal_mutable_advanced_extension()->::substrait::extensions::AdvancedExtension::MergeFrom(from._internal_advanced_extension()); @@ -21392,37 +22019,39 @@ void DelimiterJoinRel::MergeFrom(const DelimiterJoinRel& from) { if (from._internal_type() != 0) { _internal_set_type(from._internal_type()); } + if (from._internal_duplicate_eliminated_side() != 0) { + _internal_set_duplicate_eliminated_side(from._internal_duplicate_eliminated_side()); + } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void DelimiterJoinRel::CopyFrom(const DelimiterJoinRel& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DelimiterJoinRel) +void DuplicateEliminatedJoinRel::CopyFrom(const DuplicateEliminatedJoinRel& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DuplicateEliminatedJoinRel) if (&from == this) return; Clear(); MergeFrom(from); } -bool DelimiterJoinRel::IsInitialized() const { +bool DuplicateEliminatedJoinRel::IsInitialized() const { return true; } -void DelimiterJoinRel::InternalSwap(DelimiterJoinRel* other) { +void DuplicateEliminatedJoinRel::InternalSwap(DuplicateEliminatedJoinRel* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); - left_keys_.InternalSwap(&other->left_keys_); - right_keys_.InternalSwap(&other->right_keys_); + duplicate_eliminated_columns_.InternalSwap(&other->duplicate_eliminated_columns_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(DelimiterJoinRel, type_) - + sizeof(DelimiterJoinRel::type_) - - PROTOBUF_FIELD_OFFSET(DelimiterJoinRel, common_)>( + PROTOBUF_FIELD_OFFSET(DuplicateEliminatedJoinRel, duplicate_eliminated_side_) + + sizeof(DuplicateEliminatedJoinRel::duplicate_eliminated_side_) + - PROTOBUF_FIELD_OFFSET(DuplicateEliminatedJoinRel, common_)>( reinterpret_cast(&common_), reinterpret_cast(&other->common_)); } -::PROTOBUF_NAMESPACE_ID::Metadata DelimiterJoinRel::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata DuplicateEliminatedJoinRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[53]); + file_level_metadata_substrait_2falgebra_2eproto[54]); } // =================================================================== @@ -21771,7 +22400,7 @@ void FunctionArgument::InternalSwap(FunctionArgument* other) { ::PROTOBUF_NAMESPACE_ID::Metadata FunctionArgument::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[54]); + file_level_metadata_substrait_2falgebra_2eproto[55]); } // =================================================================== @@ -22012,7 +22641,7 @@ void FunctionOption::InternalSwap(FunctionOption* other) { ::PROTOBUF_NAMESPACE_ID::Metadata FunctionOption::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[55]); + file_level_metadata_substrait_2falgebra_2eproto[56]); } // =================================================================== @@ -22051,7 +22680,7 @@ const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Enum_Empty::GetClas ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Enum_Empty::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[56]); + file_level_metadata_substrait_2falgebra_2eproto[57]); } // =================================================================== @@ -22333,7 +22962,7 @@ void Expression_Enum::InternalSwap(Expression_Enum* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Enum::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[57]); + file_level_metadata_substrait_2falgebra_2eproto[58]); } // =================================================================== @@ -22454,74 +23083,324 @@ const char* Expression_Literal_VarChar::_InternalParse(const char* ptr, ::PROTOB #undef CHK_ } -uint8_t* Expression_Literal_VarChar::_InternalSerialize( +uint8_t* Expression_Literal_VarChar::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.VarChar) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // string value = 1; + if (!this->_internal_value().empty()) { + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( + this->_internal_value().data(), static_cast(this->_internal_value().length()), + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, + "substrait.Expression.Literal.VarChar.value"); + target = stream->WriteStringMaybeAliased( + 1, this->_internal_value(), target); + } + + // uint32 length = 2; + if (this->_internal_length() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_length(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.VarChar) + return target; +} + +size_t Expression_Literal_VarChar::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.VarChar) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // string value = 1; + if (!this->_internal_value().empty()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + this->_internal_value()); + } + + // uint32 length = 2; + if (this->_internal_length() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_length()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_VarChar::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + Expression_Literal_VarChar::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_VarChar::GetClassData() const { return &_class_data_; } + +void Expression_Literal_VarChar::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void Expression_Literal_VarChar::MergeFrom(const Expression_Literal_VarChar& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.VarChar) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (!from._internal_value().empty()) { + _internal_set_value(from._internal_value()); + } + if (from._internal_length() != 0) { + _internal_set_length(from._internal_length()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void Expression_Literal_VarChar::CopyFrom(const Expression_Literal_VarChar& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.VarChar) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Expression_Literal_VarChar::IsInitialized() const { + return true; +} + +void Expression_Literal_VarChar::InternalSwap(Expression_Literal_VarChar* other) { + using std::swap; + auto* lhs_arena = GetArenaForAllocation(); + auto* rhs_arena = other->GetArenaForAllocation(); + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( + &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), + &value_, lhs_arena, + &other->value_, rhs_arena + ); + swap(length_, other->length_); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_VarChar::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, + file_level_metadata_substrait_2falgebra_2eproto[59]); +} + +// =================================================================== + +class Expression_Literal_Decimal::_Internal { + public: +}; + +Expression_Literal_Decimal::Expression_Literal_Decimal(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.Expression.Literal.Decimal) +} +Expression_Literal_Decimal::Expression_Literal_Decimal(const Expression_Literal_Decimal& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); + #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + value_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (!from._internal_value().empty()) { + value_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_value(), + GetArenaForAllocation()); + } + ::memcpy(&precision_, &from.precision_, + static_cast(reinterpret_cast(&scale_) - + reinterpret_cast(&precision_)) + sizeof(scale_)); + // @@protoc_insertion_point(copy_constructor:substrait.Expression.Literal.Decimal) +} + +inline void Expression_Literal_Decimal::SharedCtor() { +value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + value_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&precision_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&scale_) - + reinterpret_cast(&precision_)) + sizeof(scale_)); +} + +Expression_Literal_Decimal::~Expression_Literal_Decimal() { + // @@protoc_insertion_point(destructor:substrait.Expression.Literal.Decimal) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void Expression_Literal_Decimal::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + value_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); +} + +void Expression_Literal_Decimal::ArenaDtor(void* object) { + Expression_Literal_Decimal* _this = reinterpret_cast< Expression_Literal_Decimal* >(object); + (void)_this; +} +void Expression_Literal_Decimal::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void Expression_Literal_Decimal::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void Expression_Literal_Decimal::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.Expression.Literal.Decimal) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + value_.ClearToEmpty(); + ::memset(&precision_, 0, static_cast( + reinterpret_cast(&scale_) - + reinterpret_cast(&precision_)) + sizeof(scale_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* Expression_Literal_Decimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // bytes value = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + auto str = _internal_mutable_value(); + ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // int32 precision = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + precision_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // int32 scale = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + scale_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* Expression_Literal_Decimal::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.VarChar) + // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.Decimal) uint32_t cached_has_bits = 0; (void) cached_has_bits; - // string value = 1; + // bytes value = 1; if (!this->_internal_value().empty()) { - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::VerifyUtf8String( - this->_internal_value().data(), static_cast(this->_internal_value().length()), - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::SERIALIZE, - "substrait.Expression.Literal.VarChar.value"); - target = stream->WriteStringMaybeAliased( + target = stream->WriteBytesMaybeAliased( 1, this->_internal_value(), target); } - // uint32 length = 2; - if (this->_internal_length() != 0) { + // int32 precision = 2; + if (this->_internal_precision() != 0) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_length(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(2, this->_internal_precision(), target); + } + + // int32 scale = 3; + if (this->_internal_scale() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(3, this->_internal_scale(), target); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.VarChar) + // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.Decimal) return target; } -size_t Expression_Literal_VarChar::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.VarChar) +size_t Expression_Literal_Decimal::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.Decimal) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // string value = 1; + // bytes value = 1; if (!this->_internal_value().empty()) { total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::StringSize( + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::BytesSize( this->_internal_value()); } - // uint32 length = 2; - if (this->_internal_length() != 0) { - total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_length()); + // int32 precision = 2; + if (this->_internal_precision() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_precision()); + } + + // int32 scale = 3; + if (this->_internal_scale() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_scale()); } return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_VarChar::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_Decimal::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - Expression_Literal_VarChar::MergeImpl + Expression_Literal_Decimal::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_VarChar::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_Decimal::GetClassData() const { return &_class_data_; } -void Expression_Literal_VarChar::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void Expression_Literal_Decimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void Expression_Literal_VarChar::MergeFrom(const Expression_Literal_VarChar& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.VarChar) +void Expression_Literal_Decimal::MergeFrom(const Expression_Literal_Decimal& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.Decimal) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -22529,24 +23408,27 @@ void Expression_Literal_VarChar::MergeFrom(const Expression_Literal_VarChar& fro if (!from._internal_value().empty()) { _internal_set_value(from._internal_value()); } - if (from._internal_length() != 0) { - _internal_set_length(from._internal_length()); + if (from._internal_precision() != 0) { + _internal_set_precision(from._internal_precision()); + } + if (from._internal_scale() != 0) { + _internal_set_scale(from._internal_scale()); } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void Expression_Literal_VarChar::CopyFrom(const Expression_Literal_VarChar& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.VarChar) +void Expression_Literal_Decimal::CopyFrom(const Expression_Literal_Decimal& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.Decimal) if (&from == this) return; Clear(); MergeFrom(from); } -bool Expression_Literal_VarChar::IsInitialized() const { +bool Expression_Literal_Decimal::IsInitialized() const { return true; } -void Expression_Literal_VarChar::InternalSwap(Expression_Literal_VarChar* other) { +void Expression_Literal_Decimal::InternalSwap(Expression_Literal_Decimal* other) { using std::swap; auto* lhs_arena = GetArenaForAllocation(); auto* rhs_arena = other->GetArenaForAllocation(); @@ -22556,120 +23438,102 @@ void Expression_Literal_VarChar::InternalSwap(Expression_Literal_VarChar* other) &value_, lhs_arena, &other->value_, rhs_arena ); - swap(length_, other->length_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(Expression_Literal_Decimal, scale_) + + sizeof(Expression_Literal_Decimal::scale_) + - PROTOBUF_FIELD_OFFSET(Expression_Literal_Decimal, precision_)>( + reinterpret_cast(&precision_), + reinterpret_cast(&other->precision_)); } -::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_VarChar::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_Decimal::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[58]); + file_level_metadata_substrait_2falgebra_2eproto[60]); } // =================================================================== -class Expression_Literal_Decimal::_Internal { +class Expression_Literal_PrecisionTimestamp::_Internal { public: }; -Expression_Literal_Decimal::Expression_Literal_Decimal(::PROTOBUF_NAMESPACE_ID::Arena* arena, +Expression_Literal_PrecisionTimestamp::Expression_Literal_PrecisionTimestamp(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.Expression.Literal.Decimal) + // @@protoc_insertion_point(arena_constructor:substrait.Expression.Literal.PrecisionTimestamp) } -Expression_Literal_Decimal::Expression_Literal_Decimal(const Expression_Literal_Decimal& from) +Expression_Literal_PrecisionTimestamp::Expression_Literal_PrecisionTimestamp(const Expression_Literal_PrecisionTimestamp& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); - value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); - #ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING - value_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); - #endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING - if (!from._internal_value().empty()) { - value_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, from._internal_value(), - GetArenaForAllocation()); - } - ::memcpy(&precision_, &from.precision_, - static_cast(reinterpret_cast(&scale_) - - reinterpret_cast(&precision_)) + sizeof(scale_)); - // @@protoc_insertion_point(copy_constructor:substrait.Expression.Literal.Decimal) + ::memcpy(&value_, &from.value_, + static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&value_)) + sizeof(precision_)); + // @@protoc_insertion_point(copy_constructor:substrait.Expression.Literal.PrecisionTimestamp) } -inline void Expression_Literal_Decimal::SharedCtor() { -value_.UnsafeSetDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); -#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING - value_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); -#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING +inline void Expression_Literal_PrecisionTimestamp::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( - reinterpret_cast(&precision_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(&scale_) - - reinterpret_cast(&precision_)) + sizeof(scale_)); + reinterpret_cast(&value_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&value_)) + sizeof(precision_)); } -Expression_Literal_Decimal::~Expression_Literal_Decimal() { - // @@protoc_insertion_point(destructor:substrait.Expression.Literal.Decimal) +Expression_Literal_PrecisionTimestamp::~Expression_Literal_PrecisionTimestamp() { + // @@protoc_insertion_point(destructor:substrait.Expression.Literal.PrecisionTimestamp) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void Expression_Literal_Decimal::SharedDtor() { +inline void Expression_Literal_PrecisionTimestamp::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); - value_.DestroyNoArena(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited()); } -void Expression_Literal_Decimal::ArenaDtor(void* object) { - Expression_Literal_Decimal* _this = reinterpret_cast< Expression_Literal_Decimal* >(object); +void Expression_Literal_PrecisionTimestamp::ArenaDtor(void* object) { + Expression_Literal_PrecisionTimestamp* _this = reinterpret_cast< Expression_Literal_PrecisionTimestamp* >(object); (void)_this; } -void Expression_Literal_Decimal::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void Expression_Literal_PrecisionTimestamp::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void Expression_Literal_Decimal::SetCachedSize(int size) const { +void Expression_Literal_PrecisionTimestamp::SetCachedSize(int size) const { _cached_size_.Set(size); } -void Expression_Literal_Decimal::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.Expression.Literal.Decimal) +void Expression_Literal_PrecisionTimestamp::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.Expression.Literal.PrecisionTimestamp) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - value_.ClearToEmpty(); - ::memset(&precision_, 0, static_cast( - reinterpret_cast(&scale_) - - reinterpret_cast(&precision_)) + sizeof(scale_)); + ::memset(&value_, 0, static_cast( + reinterpret_cast(&precision_) - + reinterpret_cast(&value_)) + sizeof(precision_)); _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* Expression_Literal_Decimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* Expression_Literal_PrecisionTimestamp::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); switch (tag >> 3) { - // bytes value = 1; + // int32 precision = 1; case 1: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { - auto str = _internal_mutable_value(); - ptr = ::PROTOBUF_NAMESPACE_ID::internal::InlineGreedyStringParser(str, ptr, ctx); + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 8)) { + precision_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); CHK_(ptr); } else goto handle_unusual; continue; - // int32 precision = 2; + // int64 value = 2; case 2: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { - precision_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // int32 scale = 3; - case 3: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { - scale_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + value_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); } else goto handle_unusual; @@ -22697,130 +23561,107 @@ const char* Expression_Literal_Decimal::_InternalParse(const char* ptr, ::PROTOB #undef CHK_ } -uint8_t* Expression_Literal_Decimal::_InternalSerialize( +uint8_t* Expression_Literal_PrecisionTimestamp::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.Decimal) + // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.PrecisionTimestamp) uint32_t cached_has_bits = 0; (void) cached_has_bits; - // bytes value = 1; - if (!this->_internal_value().empty()) { - target = stream->WriteBytesMaybeAliased( - 1, this->_internal_value(), target); - } - - // int32 precision = 2; + // int32 precision = 1; if (this->_internal_precision() != 0) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(2, this->_internal_precision(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(1, this->_internal_precision(), target); } - // int32 scale = 3; - if (this->_internal_scale() != 0) { + // int64 value = 2; + if (this->_internal_value() != 0) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(3, this->_internal_scale(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64ToArray(2, this->_internal_value(), target); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.Decimal) + // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.PrecisionTimestamp) return target; } -size_t Expression_Literal_Decimal::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.Decimal) +size_t Expression_Literal_PrecisionTimestamp::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.PrecisionTimestamp) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // bytes value = 1; - if (!this->_internal_value().empty()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::BytesSize( - this->_internal_value()); + // int64 value = 2; + if (this->_internal_value() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int64SizePlusOne(this->_internal_value()); } - // int32 precision = 2; + // int32 precision = 1; if (this->_internal_precision() != 0) { total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_precision()); } - // int32 scale = 3; - if (this->_internal_scale() != 0) { - total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_scale()); - } - return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_Decimal::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_PrecisionTimestamp::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - Expression_Literal_Decimal::MergeImpl + Expression_Literal_PrecisionTimestamp::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_Decimal::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_PrecisionTimestamp::GetClassData() const { return &_class_data_; } -void Expression_Literal_Decimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void Expression_Literal_PrecisionTimestamp::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void Expression_Literal_Decimal::MergeFrom(const Expression_Literal_Decimal& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.Decimal) +void Expression_Literal_PrecisionTimestamp::MergeFrom(const Expression_Literal_PrecisionTimestamp& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.PrecisionTimestamp) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; - if (!from._internal_value().empty()) { + if (from._internal_value() != 0) { _internal_set_value(from._internal_value()); } if (from._internal_precision() != 0) { _internal_set_precision(from._internal_precision()); } - if (from._internal_scale() != 0) { - _internal_set_scale(from._internal_scale()); - } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void Expression_Literal_Decimal::CopyFrom(const Expression_Literal_Decimal& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.Decimal) +void Expression_Literal_PrecisionTimestamp::CopyFrom(const Expression_Literal_PrecisionTimestamp& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.PrecisionTimestamp) if (&from == this) return; Clear(); MergeFrom(from); } -bool Expression_Literal_Decimal::IsInitialized() const { +bool Expression_Literal_PrecisionTimestamp::IsInitialized() const { return true; } -void Expression_Literal_Decimal::InternalSwap(Expression_Literal_Decimal* other) { +void Expression_Literal_PrecisionTimestamp::InternalSwap(Expression_Literal_PrecisionTimestamp* other) { using std::swap; - auto* lhs_arena = GetArenaForAllocation(); - auto* rhs_arena = other->GetArenaForAllocation(); _internal_metadata_.InternalSwap(&other->_internal_metadata_); - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::InternalSwap( - &::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), - &value_, lhs_arena, - &other->value_, rhs_arena - ); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(Expression_Literal_Decimal, scale_) - + sizeof(Expression_Literal_Decimal::scale_) - - PROTOBUF_FIELD_OFFSET(Expression_Literal_Decimal, precision_)>( - reinterpret_cast(&precision_), - reinterpret_cast(&other->precision_)); + PROTOBUF_FIELD_OFFSET(Expression_Literal_PrecisionTimestamp, precision_) + + sizeof(Expression_Literal_PrecisionTimestamp::precision_) + - PROTOBUF_FIELD_OFFSET(Expression_Literal_PrecisionTimestamp, value_)>( + reinterpret_cast(&value_), + reinterpret_cast(&other->value_)); } -::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_Decimal::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_PrecisionTimestamp::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[59]); + file_level_metadata_substrait_2falgebra_2eproto[61]); } // =================================================================== @@ -23064,7 +23905,7 @@ void Expression_Literal_Map_KeyValue::InternalSwap(Expression_Literal_Map_KeyVal ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_Map_KeyValue::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[60]); + file_level_metadata_substrait_2falgebra_2eproto[62]); } // =================================================================== @@ -23249,7 +24090,7 @@ void Expression_Literal_Map::InternalSwap(Expression_Literal_Map* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_Map::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[61]); + file_level_metadata_substrait_2falgebra_2eproto[63]); } // =================================================================== @@ -23461,7 +24302,7 @@ void Expression_Literal_IntervalYearToMonth::InternalSwap(Expression_Literal_Int ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_IntervalYearToMonth::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[62]); + file_level_metadata_substrait_2falgebra_2eproto[64]); } // =================================================================== @@ -23483,16 +24324,31 @@ Expression_Literal_IntervalDayToSecond::Expression_Literal_IntervalDayToSecond(c : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); ::memcpy(&days_, &from.days_, - static_cast(reinterpret_cast(µseconds_) - - reinterpret_cast(&days_)) + sizeof(microseconds_)); + static_cast(reinterpret_cast(&subseconds_) - + reinterpret_cast(&days_)) + sizeof(subseconds_)); + clear_has_precision_mode(); + switch (from.precision_mode_case()) { + case kMicroseconds: { + _internal_set_microseconds(from._internal_microseconds()); + break; + } + case kPrecision: { + _internal_set_precision(from._internal_precision()); + break; + } + case PRECISION_MODE_NOT_SET: { + break; + } + } // @@protoc_insertion_point(copy_constructor:substrait.Expression.Literal.IntervalDayToSecond) } inline void Expression_Literal_IntervalDayToSecond::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&days_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(µseconds_) - - reinterpret_cast(&days_)) + sizeof(microseconds_)); + 0, static_cast(reinterpret_cast(&subseconds_) - + reinterpret_cast(&days_)) + sizeof(subseconds_)); +clear_has_precision_mode(); } Expression_Literal_IntervalDayToSecond::~Expression_Literal_IntervalDayToSecond() { @@ -23504,6 +24360,9 @@ Expression_Literal_IntervalDayToSecond::~Expression_Literal_IntervalDayToSecond( inline void Expression_Literal_IntervalDayToSecond::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (has_precision_mode()) { + clear_precision_mode(); + } } void Expression_Literal_IntervalDayToSecond::ArenaDtor(void* object) { @@ -23516,6 +24375,25 @@ void Expression_Literal_IntervalDayToSecond::SetCachedSize(int size) const { _cached_size_.Set(size); } +void Expression_Literal_IntervalDayToSecond::clear_precision_mode() { +// @@protoc_insertion_point(one_of_clear_start:substrait.Expression.Literal.IntervalDayToSecond) + switch (precision_mode_case()) { + case kMicroseconds: { + // No need to clear + break; + } + case kPrecision: { + // No need to clear + break; + } + case PRECISION_MODE_NOT_SET: { + break; + } + } + _oneof_case_[0] = PRECISION_MODE_NOT_SET; +} + + void Expression_Literal_IntervalDayToSecond::Clear() { // @@protoc_insertion_point(message_clear_start:substrait.Expression.Literal.IntervalDayToSecond) uint32_t cached_has_bits = 0; @@ -23523,8 +24401,9 @@ void Expression_Literal_IntervalDayToSecond::Clear() { (void) cached_has_bits; ::memset(&days_, 0, static_cast( - reinterpret_cast(µseconds_) - - reinterpret_cast(&days_)) + sizeof(microseconds_)); + reinterpret_cast(&subseconds_) - + reinterpret_cast(&days_)) + sizeof(subseconds_)); + clear_precision_mode(); _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } @@ -23550,10 +24429,26 @@ const char* Expression_Literal_IntervalDayToSecond::_InternalParse(const char* p } else goto handle_unusual; continue; - // int32 microseconds = 3; + // int32 microseconds = 3 [deprecated = true]; case 3: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { - microseconds_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + _internal_set_microseconds(::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr)); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // int32 precision = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { + _internal_set_precision(::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr)); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // int64 subseconds = 5; + case 5: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 40)) { + subseconds_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); } else goto handle_unusual; @@ -23599,12 +24494,24 @@ uint8_t* Expression_Literal_IntervalDayToSecond::_InternalSerialize( target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(2, this->_internal_seconds(), target); } - // int32 microseconds = 3; - if (this->_internal_microseconds() != 0) { + // int32 microseconds = 3 [deprecated = true]; + if (_internal_has_microseconds()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(3, this->_internal_microseconds(), target); } + // int32 precision = 4; + if (_internal_has_precision()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(4, this->_internal_precision(), target); + } + + // int64 subseconds = 5; + if (this->_internal_subseconds() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt64ToArray(5, this->_internal_subseconds(), target); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -23631,11 +24538,26 @@ size_t Expression_Literal_IntervalDayToSecond::ByteSizeLong() const { total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_seconds()); } - // int32 microseconds = 3; - if (this->_internal_microseconds() != 0) { - total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_microseconds()); + // int64 subseconds = 5; + if (this->_internal_subseconds() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int64SizePlusOne(this->_internal_subseconds()); } + switch (precision_mode_case()) { + // int32 microseconds = 3 [deprecated = true]; + case kMicroseconds: { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_microseconds()); + break; + } + // int32 precision = 4; + case kPrecision: { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_precision()); + break; + } + case PRECISION_MODE_NOT_SET: { + break; + } + } return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } @@ -23664,8 +24586,21 @@ void Expression_Literal_IntervalDayToSecond::MergeFrom(const Expression_Literal_ if (from._internal_seconds() != 0) { _internal_set_seconds(from._internal_seconds()); } - if (from._internal_microseconds() != 0) { - _internal_set_microseconds(from._internal_microseconds()); + if (from._internal_subseconds() != 0) { + _internal_set_subseconds(from._internal_subseconds()); + } + switch (from.precision_mode_case()) { + case kMicroseconds: { + _internal_set_microseconds(from._internal_microseconds()); + break; + } + case kPrecision: { + _internal_set_precision(from._internal_precision()); + break; + } + case PRECISION_MODE_NOT_SET: { + break; + } } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } @@ -23685,17 +24620,263 @@ void Expression_Literal_IntervalDayToSecond::InternalSwap(Expression_Literal_Int using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(Expression_Literal_IntervalDayToSecond, microseconds_) - + sizeof(Expression_Literal_IntervalDayToSecond::microseconds_) + PROTOBUF_FIELD_OFFSET(Expression_Literal_IntervalDayToSecond, subseconds_) + + sizeof(Expression_Literal_IntervalDayToSecond::subseconds_) - PROTOBUF_FIELD_OFFSET(Expression_Literal_IntervalDayToSecond, days_)>( reinterpret_cast(&days_), reinterpret_cast(&other->days_)); + swap(precision_mode_, other->precision_mode_); + swap(_oneof_case_[0], other->_oneof_case_[0]); } ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_IntervalDayToSecond::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[63]); + file_level_metadata_substrait_2falgebra_2eproto[65]); +} + +// =================================================================== + +class Expression_Literal_IntervalCompound::_Internal { + public: + static const ::substrait::Expression_Literal_IntervalYearToMonth& interval_year_to_month(const Expression_Literal_IntervalCompound* msg); + static const ::substrait::Expression_Literal_IntervalDayToSecond& interval_day_to_second(const Expression_Literal_IntervalCompound* msg); +}; + +const ::substrait::Expression_Literal_IntervalYearToMonth& +Expression_Literal_IntervalCompound::_Internal::interval_year_to_month(const Expression_Literal_IntervalCompound* msg) { + return *msg->interval_year_to_month_; +} +const ::substrait::Expression_Literal_IntervalDayToSecond& +Expression_Literal_IntervalCompound::_Internal::interval_day_to_second(const Expression_Literal_IntervalCompound* msg) { + return *msg->interval_day_to_second_; +} +Expression_Literal_IntervalCompound::Expression_Literal_IntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.Expression.Literal.IntervalCompound) +} +Expression_Literal_IntervalCompound::Expression_Literal_IntervalCompound(const Expression_Literal_IntervalCompound& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + if (from._internal_has_interval_year_to_month()) { + interval_year_to_month_ = new ::substrait::Expression_Literal_IntervalYearToMonth(*from.interval_year_to_month_); + } else { + interval_year_to_month_ = nullptr; + } + if (from._internal_has_interval_day_to_second()) { + interval_day_to_second_ = new ::substrait::Expression_Literal_IntervalDayToSecond(*from.interval_day_to_second_); + } else { + interval_day_to_second_ = nullptr; + } + // @@protoc_insertion_point(copy_constructor:substrait.Expression.Literal.IntervalCompound) +} + +inline void Expression_Literal_IntervalCompound::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&interval_year_to_month_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&interval_day_to_second_) - + reinterpret_cast(&interval_year_to_month_)) + sizeof(interval_day_to_second_)); +} + +Expression_Literal_IntervalCompound::~Expression_Literal_IntervalCompound() { + // @@protoc_insertion_point(destructor:substrait.Expression.Literal.IntervalCompound) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void Expression_Literal_IntervalCompound::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (this != internal_default_instance()) delete interval_year_to_month_; + if (this != internal_default_instance()) delete interval_day_to_second_; +} + +void Expression_Literal_IntervalCompound::ArenaDtor(void* object) { + Expression_Literal_IntervalCompound* _this = reinterpret_cast< Expression_Literal_IntervalCompound* >(object); + (void)_this; +} +void Expression_Literal_IntervalCompound::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void Expression_Literal_IntervalCompound::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void Expression_Literal_IntervalCompound::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.Expression.Literal.IntervalCompound) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaForAllocation() == nullptr && interval_year_to_month_ != nullptr) { + delete interval_year_to_month_; + } + interval_year_to_month_ = nullptr; + if (GetArenaForAllocation() == nullptr && interval_day_to_second_ != nullptr) { + delete interval_day_to_second_; + } + interval_day_to_second_ = nullptr; + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* Expression_Literal_IntervalCompound::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // .substrait.Expression.Literal.IntervalYearToMonth interval_year_to_month = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_year_to_month(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Expression.Literal.IntervalDayToSecond interval_day_to_second = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_day_to_second(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* Expression_Literal_IntervalCompound::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.Expression.Literal.IntervalCompound) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // .substrait.Expression.Literal.IntervalYearToMonth interval_year_to_month = 1; + if (this->_internal_has_interval_year_to_month()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 1, _Internal::interval_year_to_month(this), target, stream); + } + + // .substrait.Expression.Literal.IntervalDayToSecond interval_day_to_second = 2; + if (this->_internal_has_interval_day_to_second()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 2, _Internal::interval_day_to_second(this), target, stream); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.Expression.Literal.IntervalCompound) + return target; +} + +size_t Expression_Literal_IntervalCompound::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.Expression.Literal.IntervalCompound) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .substrait.Expression.Literal.IntervalYearToMonth interval_year_to_month = 1; + if (this->_internal_has_interval_year_to_month()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *interval_year_to_month_); + } + + // .substrait.Expression.Literal.IntervalDayToSecond interval_day_to_second = 2; + if (this->_internal_has_interval_day_to_second()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *interval_day_to_second_); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Expression_Literal_IntervalCompound::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + Expression_Literal_IntervalCompound::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_Literal_IntervalCompound::GetClassData() const { return &_class_data_; } + +void Expression_Literal_IntervalCompound::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void Expression_Literal_IntervalCompound::MergeFrom(const Expression_Literal_IntervalCompound& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Expression.Literal.IntervalCompound) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_has_interval_year_to_month()) { + _internal_mutable_interval_year_to_month()->::substrait::Expression_Literal_IntervalYearToMonth::MergeFrom(from._internal_interval_year_to_month()); + } + if (from._internal_has_interval_day_to_second()) { + _internal_mutable_interval_day_to_second()->::substrait::Expression_Literal_IntervalDayToSecond::MergeFrom(from._internal_interval_day_to_second()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void Expression_Literal_IntervalCompound::CopyFrom(const Expression_Literal_IntervalCompound& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Expression.Literal.IntervalCompound) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Expression_Literal_IntervalCompound::IsInitialized() const { + return true; +} + +void Expression_Literal_IntervalCompound::InternalSwap(Expression_Literal_IntervalCompound* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(Expression_Literal_IntervalCompound, interval_day_to_second_) + + sizeof(Expression_Literal_IntervalCompound::interval_day_to_second_) + - PROTOBUF_FIELD_OFFSET(Expression_Literal_IntervalCompound, interval_year_to_month_)>( + reinterpret_cast(&interval_year_to_month_), + reinterpret_cast(&other->interval_year_to_month_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_IntervalCompound::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, + file_level_metadata_substrait_2falgebra_2eproto[66]); } // =================================================================== @@ -23880,7 +25061,7 @@ void Expression_Literal_Struct::InternalSwap(Expression_Literal_Struct* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_Struct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[64]); + file_level_metadata_substrait_2falgebra_2eproto[67]); } // =================================================================== @@ -24065,7 +25246,7 @@ void Expression_Literal_List::InternalSwap(Expression_Literal_List* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_List::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[65]); + file_level_metadata_substrait_2falgebra_2eproto[68]); } // =================================================================== @@ -24437,7 +25618,7 @@ void Expression_Literal_UserDefined::InternalSwap(Expression_Literal_UserDefined ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal_UserDefined::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[66]); + file_level_metadata_substrait_2falgebra_2eproto[69]); } // =================================================================== @@ -24446,8 +25627,11 @@ class Expression_Literal::_Internal { public: static const ::substrait::Expression_Literal_IntervalYearToMonth& interval_year_to_month(const Expression_Literal* msg); static const ::substrait::Expression_Literal_IntervalDayToSecond& interval_day_to_second(const Expression_Literal* msg); + static const ::substrait::Expression_Literal_IntervalCompound& interval_compound(const Expression_Literal* msg); static const ::substrait::Expression_Literal_VarChar& var_char(const Expression_Literal* msg); static const ::substrait::Expression_Literal_Decimal& decimal(const Expression_Literal* msg); + static const ::substrait::Expression_Literal_PrecisionTimestamp& precision_timestamp(const Expression_Literal* msg); + static const ::substrait::Expression_Literal_PrecisionTimestamp& precision_timestamp_tz(const Expression_Literal* msg); static const ::substrait::Expression_Literal_Struct& struct_(const Expression_Literal* msg); static const ::substrait::Expression_Literal_Map& map(const Expression_Literal* msg); static const ::substrait::Type& null(const Expression_Literal* msg); @@ -24465,6 +25649,10 @@ const ::substrait::Expression_Literal_IntervalDayToSecond& Expression_Literal::_Internal::interval_day_to_second(const Expression_Literal* msg) { return *msg->literal_type_.interval_day_to_second_; } +const ::substrait::Expression_Literal_IntervalCompound& +Expression_Literal::_Internal::interval_compound(const Expression_Literal* msg) { + return *msg->literal_type_.interval_compound_; +} const ::substrait::Expression_Literal_VarChar& Expression_Literal::_Internal::var_char(const Expression_Literal* msg) { return *msg->literal_type_.var_char_; @@ -24473,6 +25661,14 @@ const ::substrait::Expression_Literal_Decimal& Expression_Literal::_Internal::decimal(const Expression_Literal* msg) { return *msg->literal_type_.decimal_; } +const ::substrait::Expression_Literal_PrecisionTimestamp& +Expression_Literal::_Internal::precision_timestamp(const Expression_Literal* msg) { + return *msg->literal_type_.precision_timestamp_; +} +const ::substrait::Expression_Literal_PrecisionTimestamp& +Expression_Literal::_Internal::precision_timestamp_tz(const Expression_Literal* msg) { + return *msg->literal_type_.precision_timestamp_tz_; +} const ::substrait::Expression_Literal_Struct& Expression_Literal::_Internal::struct_(const Expression_Literal* msg) { return *msg->literal_type_.struct__; @@ -24531,6 +25727,21 @@ void Expression_Literal::set_allocated_interval_day_to_second(::substrait::Expre } // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.interval_day_to_second) } +void Expression_Literal::set_allocated_interval_compound(::substrait::Expression_Literal_IntervalCompound* interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_literal_type(); + if (interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_Literal_IntervalCompound>::GetOwningArena(interval_compound); + if (message_arena != submessage_arena) { + interval_compound = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_compound, submessage_arena); + } + set_has_interval_compound(); + literal_type_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.interval_compound) +} void Expression_Literal::set_allocated_var_char(::substrait::Expression_Literal_VarChar* var_char) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_literal_type(); @@ -24561,6 +25772,36 @@ void Expression_Literal::set_allocated_decimal(::substrait::Expression_Literal_D } // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.decimal) } +void Expression_Literal::set_allocated_precision_timestamp(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_literal_type(); + if (precision_timestamp) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_Literal_PrecisionTimestamp>::GetOwningArena(precision_timestamp); + if (message_arena != submessage_arena) { + precision_timestamp = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision_timestamp, submessage_arena); + } + set_has_precision_timestamp(); + literal_type_.precision_timestamp_ = precision_timestamp; + } + // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.precision_timestamp) +} +void Expression_Literal::set_allocated_precision_timestamp_tz(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_tz) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_literal_type(); + if (precision_timestamp_tz) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_Literal_PrecisionTimestamp>::GetOwningArena(precision_timestamp_tz); + if (message_arena != submessage_arena) { + precision_timestamp_tz = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision_timestamp_tz, submessage_arena); + } + set_has_precision_timestamp_tz(); + literal_type_.precision_timestamp_tz_ = precision_timestamp_tz; + } + // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.precision_timestamp_tz) +} void Expression_Literal::set_allocated_struct_(::substrait::Expression_Literal_Struct* struct_) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_literal_type(); @@ -24769,6 +26010,10 @@ Expression_Literal::Expression_Literal(const Expression_Literal& from) _internal_mutable_interval_day_to_second()->::substrait::Expression_Literal_IntervalDayToSecond::MergeFrom(from._internal_interval_day_to_second()); break; } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::Expression_Literal_IntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kFixedChar: { _internal_set_fixed_char(from._internal_fixed_char()); break; @@ -24786,11 +26031,11 @@ Expression_Literal::Expression_Literal(const Expression_Literal& from) break; } case kPrecisionTimestamp: { - _internal_set_precision_timestamp(from._internal_precision_timestamp()); + _internal_mutable_precision_timestamp()->::substrait::Expression_Literal_PrecisionTimestamp::MergeFrom(from._internal_precision_timestamp()); break; } case kPrecisionTimestampTz: { - _internal_set_precision_timestamp_tz(from._internal_precision_timestamp_tz()); + _internal_mutable_precision_timestamp_tz()->::substrait::Expression_Literal_PrecisionTimestamp::MergeFrom(from._internal_precision_timestamp_tz()); break; } case kStruct: { @@ -24931,6 +26176,12 @@ void Expression_Literal::clear_literal_type() { } break; } + case kIntervalCompound: { + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.interval_compound_; + } + break; + } case kFixedChar: { literal_type_.fixed_char_.Destroy(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); break; @@ -24952,11 +26203,15 @@ void Expression_Literal::clear_literal_type() { break; } case kPrecisionTimestamp: { - // No need to clear + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.precision_timestamp_; + } break; } case kPrecisionTimestampTz: { - // No need to clear + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.precision_timestamp_tz_; + } break; } case kStruct: { @@ -25259,18 +26514,26 @@ const char* Expression_Literal::_InternalParse(const char* ptr, ::PROTOBUF_NAMES } else goto handle_unusual; continue; - // uint64 precision_timestamp = 34; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp = 34; case 34: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { - _internal_set_precision_timestamp(::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr)); + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { + ptr = ctx->ParseMessage(_internal_mutable_precision_timestamp(), ptr); CHK_(ptr); } else goto handle_unusual; continue; - // uint64 precision_timestamp_tz = 35; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp_tz = 35; case 35: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { - _internal_set_precision_timestamp_tz(::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr)); + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 26)) { + ptr = ctx->ParseMessage(_internal_mutable_precision_timestamp_tz(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Expression.Literal.IntervalCompound interval_compound = 36; + case 36: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 34)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_compound(), ptr); CHK_(ptr); } else goto handle_unusual; @@ -25512,16 +26775,28 @@ uint8_t* Expression_Literal::_InternalSerialize( 33, _Internal::user_defined(this), target, stream); } - // uint64 precision_timestamp = 34; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp = 34; if (_internal_has_precision_timestamp()) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(34, this->_internal_precision_timestamp(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 34, _Internal::precision_timestamp(this), target, stream); } - // uint64 precision_timestamp_tz = 35; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp_tz = 35; if (_internal_has_precision_timestamp_tz()) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt64ToArray(35, this->_internal_precision_timestamp_tz(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 35, _Internal::precision_timestamp_tz(this), target, stream); + } + + // .substrait.Expression.Literal.IntervalCompound interval_compound = 36; + if (_internal_has_interval_compound()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 36, _Internal::interval_compound(this), target, stream); } // bool nullable = 50; @@ -25647,6 +26922,13 @@ size_t Expression_Literal::ByteSizeLong() const { *literal_type_.interval_day_to_second_); break; } + // .substrait.Expression.Literal.IntervalCompound interval_compound = 36; + case kIntervalCompound: { + total_size += 2 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *literal_type_.interval_compound_); + break; + } // string fixed_char = 21; case kFixedChar: { total_size += 2 + @@ -25675,18 +26957,18 @@ size_t Expression_Literal::ByteSizeLong() const { *literal_type_.decimal_); break; } - // uint64 precision_timestamp = 34; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp = 34; case kPrecisionTimestamp: { total_size += 2 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( - this->_internal_precision_timestamp()); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *literal_type_.precision_timestamp_); break; } - // uint64 precision_timestamp_tz = 35; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp_tz = 35; case kPrecisionTimestampTz: { total_size += 2 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt64Size( - this->_internal_precision_timestamp_tz()); + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *literal_type_.precision_timestamp_tz_); break; } // .substrait.Expression.Literal.Struct struct = 25; @@ -25841,6 +27123,10 @@ void Expression_Literal::MergeFrom(const Expression_Literal& from) { _internal_mutable_interval_day_to_second()->::substrait::Expression_Literal_IntervalDayToSecond::MergeFrom(from._internal_interval_day_to_second()); break; } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::Expression_Literal_IntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kFixedChar: { _internal_set_fixed_char(from._internal_fixed_char()); break; @@ -25858,11 +27144,11 @@ void Expression_Literal::MergeFrom(const Expression_Literal& from) { break; } case kPrecisionTimestamp: { - _internal_set_precision_timestamp(from._internal_precision_timestamp()); + _internal_mutable_precision_timestamp()->::substrait::Expression_Literal_PrecisionTimestamp::MergeFrom(from._internal_precision_timestamp()); break; } case kPrecisionTimestampTz: { - _internal_set_precision_timestamp_tz(from._internal_precision_timestamp_tz()); + _internal_mutable_precision_timestamp_tz()->::substrait::Expression_Literal_PrecisionTimestamp::MergeFrom(from._internal_precision_timestamp_tz()); break; } case kStruct: { @@ -25935,7 +27221,7 @@ void Expression_Literal::InternalSwap(Expression_Literal* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Literal::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[67]); + file_level_metadata_substrait_2falgebra_2eproto[70]); } // =================================================================== @@ -26179,7 +27465,7 @@ void Expression_Nested_Map_KeyValue::InternalSwap(Expression_Nested_Map_KeyValue ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Nested_Map_KeyValue::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[68]); + file_level_metadata_substrait_2falgebra_2eproto[71]); } // =================================================================== @@ -26364,7 +27650,7 @@ void Expression_Nested_Map::InternalSwap(Expression_Nested_Map* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Nested_Map::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[69]); + file_level_metadata_substrait_2falgebra_2eproto[72]); } // =================================================================== @@ -26549,7 +27835,7 @@ void Expression_Nested_Struct::InternalSwap(Expression_Nested_Struct* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Nested_Struct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[70]); + file_level_metadata_substrait_2falgebra_2eproto[73]); } // =================================================================== @@ -26734,7 +28020,7 @@ void Expression_Nested_List::InternalSwap(Expression_Nested_List* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Nested_List::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[71]); + file_level_metadata_substrait_2falgebra_2eproto[74]); } // =================================================================== @@ -27151,7 +28437,7 @@ void Expression_Nested::InternalSwap(Expression_Nested* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Nested::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[72]); + file_level_metadata_substrait_2falgebra_2eproto[75]); } // =================================================================== @@ -27483,7 +28769,7 @@ void Expression_ScalarFunction::InternalSwap(Expression_ScalarFunction* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_ScalarFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[73]); + file_level_metadata_substrait_2falgebra_2eproto[76]); } // =================================================================== @@ -27661,7 +28947,7 @@ void Expression_WindowFunction_Bound_Preceding::InternalSwap(Expression_WindowFu ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction_Bound_Preceding::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[74]); + file_level_metadata_substrait_2falgebra_2eproto[77]); } // =================================================================== @@ -27839,7 +29125,7 @@ void Expression_WindowFunction_Bound_Following::InternalSwap(Expression_WindowFu ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction_Bound_Following::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[75]); + file_level_metadata_substrait_2falgebra_2eproto[78]); } // =================================================================== @@ -27878,7 +29164,7 @@ const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_WindowFunction_Boun ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction_Bound_CurrentRow::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[76]); + file_level_metadata_substrait_2falgebra_2eproto[79]); } // =================================================================== @@ -27917,7 +29203,7 @@ const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_WindowFunction_Boun ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction_Bound_Unbounded::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[77]); + file_level_metadata_substrait_2falgebra_2eproto[80]); } // =================================================================== @@ -28331,7 +29617,7 @@ void Expression_WindowFunction_Bound::InternalSwap(Expression_WindowFunction_Bou ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction_Bound::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[78]); + file_level_metadata_substrait_2falgebra_2eproto[81]); } // =================================================================== @@ -28890,7 +30176,7 @@ void Expression_WindowFunction::InternalSwap(Expression_WindowFunction* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_WindowFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[79]); + file_level_metadata_substrait_2falgebra_2eproto[82]); } // =================================================================== @@ -29134,7 +30420,7 @@ void Expression_IfThen_IfClause::InternalSwap(Expression_IfThen_IfClause* other) ::PROTOBUF_NAMESPACE_ID::Metadata Expression_IfThen_IfClause::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[80]); + file_level_metadata_substrait_2falgebra_2eproto[83]); } // =================================================================== @@ -29362,7 +30648,7 @@ void Expression_IfThen::InternalSwap(Expression_IfThen* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_IfThen::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[81]); + file_level_metadata_substrait_2falgebra_2eproto[84]); } // =================================================================== @@ -29639,7 +30925,7 @@ void Expression_Cast::InternalSwap(Expression_Cast* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Cast::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[82]); + file_level_metadata_substrait_2falgebra_2eproto[85]); } // =================================================================== @@ -29883,7 +31169,7 @@ void Expression_SwitchExpression_IfValue::InternalSwap(Expression_SwitchExpressi ::PROTOBUF_NAMESPACE_ID::Metadata Expression_SwitchExpression_IfValue::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[83]); + file_level_metadata_substrait_2falgebra_2eproto[86]); } // =================================================================== @@ -30160,7 +31446,7 @@ void Expression_SwitchExpression::InternalSwap(Expression_SwitchExpression* othe ::PROTOBUF_NAMESPACE_ID::Metadata Expression_SwitchExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[84]); + file_level_metadata_substrait_2falgebra_2eproto[87]); } // =================================================================== @@ -30388,7 +31674,7 @@ void Expression_SingularOrList::InternalSwap(Expression_SingularOrList* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_SingularOrList::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[85]); + file_level_metadata_substrait_2falgebra_2eproto[88]); } // =================================================================== @@ -30573,7 +31859,7 @@ void Expression_MultiOrList_Record::InternalSwap(Expression_MultiOrList_Record* ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MultiOrList_Record::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[86]); + file_level_metadata_substrait_2falgebra_2eproto[89]); } // =================================================================== @@ -30791,7 +32077,7 @@ void Expression_MultiOrList::InternalSwap(Expression_MultiOrList* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MultiOrList::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[87]); + file_level_metadata_substrait_2falgebra_2eproto[90]); } // =================================================================== @@ -31027,7 +32313,7 @@ void Expression_EmbeddedFunction_PythonPickleFunction::InternalSwap(Expression_E ::PROTOBUF_NAMESPACE_ID::Metadata Expression_EmbeddedFunction_PythonPickleFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[88]); + file_level_metadata_substrait_2falgebra_2eproto[91]); } // =================================================================== @@ -31263,7 +32549,7 @@ void Expression_EmbeddedFunction_WebAssemblyFunction::InternalSwap(Expression_Em ::PROTOBUF_NAMESPACE_ID::Metadata Expression_EmbeddedFunction_WebAssemblyFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[89]); + file_level_metadata_substrait_2falgebra_2eproto[92]); } // =================================================================== @@ -31645,7 +32931,7 @@ void Expression_EmbeddedFunction::InternalSwap(Expression_EmbeddedFunction* othe ::PROTOBUF_NAMESPACE_ID::Metadata Expression_EmbeddedFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[90]); + file_level_metadata_substrait_2falgebra_2eproto[93]); } // =================================================================== @@ -31889,7 +33175,7 @@ void Expression_ReferenceSegment_MapKey::InternalSwap(Expression_ReferenceSegmen ::PROTOBUF_NAMESPACE_ID::Metadata Expression_ReferenceSegment_MapKey::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[91]); + file_level_metadata_substrait_2falgebra_2eproto[94]); } // =================================================================== @@ -32116,7 +33402,7 @@ void Expression_ReferenceSegment_StructField::InternalSwap(Expression_ReferenceS ::PROTOBUF_NAMESPACE_ID::Metadata Expression_ReferenceSegment_StructField::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[92]); + file_level_metadata_substrait_2falgebra_2eproto[95]); } // =================================================================== @@ -32343,7 +33629,7 @@ void Expression_ReferenceSegment_ListElement::InternalSwap(Expression_ReferenceS ::PROTOBUF_NAMESPACE_ID::Metadata Expression_ReferenceSegment_ListElement::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[93]); + file_level_metadata_substrait_2falgebra_2eproto[96]); } // =================================================================== @@ -32700,7 +33986,7 @@ void Expression_ReferenceSegment::InternalSwap(Expression_ReferenceSegment* othe ::PROTOBUF_NAMESPACE_ID::Metadata Expression_ReferenceSegment::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[94]); + file_level_metadata_substrait_2falgebra_2eproto[97]); } // =================================================================== @@ -33057,7 +34343,7 @@ void Expression_MaskExpression_Select::InternalSwap(Expression_MaskExpression_Se ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_Select::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[95]); + file_level_metadata_substrait_2falgebra_2eproto[98]); } // =================================================================== @@ -33242,7 +34528,7 @@ void Expression_MaskExpression_StructSelect::InternalSwap(Expression_MaskExpress ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_StructSelect::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[96]); + file_level_metadata_substrait_2falgebra_2eproto[99]); } // =================================================================== @@ -33469,7 +34755,7 @@ void Expression_MaskExpression_StructItem::InternalSwap(Expression_MaskExpressio ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_StructItem::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[97]); + file_level_metadata_substrait_2falgebra_2eproto[100]); } // =================================================================== @@ -33647,7 +34933,7 @@ void Expression_MaskExpression_ListSelect_ListSelectItem_ListElement::InternalSw ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_ListSelect_ListSelectItem_ListElement::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[98]); + file_level_metadata_substrait_2falgebra_2eproto[101]); } // =================================================================== @@ -33859,7 +35145,7 @@ void Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice::InternalSwap ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[99]); + file_level_metadata_substrait_2falgebra_2eproto[102]); } // =================================================================== @@ -34159,7 +35445,7 @@ void Expression_MaskExpression_ListSelect_ListSelectItem::InternalSwap(Expressio ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_ListSelect_ListSelectItem::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[100]); + file_level_metadata_substrait_2falgebra_2eproto[103]); } // =================================================================== @@ -34387,7 +35673,7 @@ void Expression_MaskExpression_ListSelect::InternalSwap(Expression_MaskExpressio ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_ListSelect::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[101]); + file_level_metadata_substrait_2falgebra_2eproto[104]); } // =================================================================== @@ -34590,7 +35876,7 @@ void Expression_MaskExpression_MapSelect_MapKey::InternalSwap(Expression_MaskExp ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_MapSelect_MapKey::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[102]); + file_level_metadata_substrait_2falgebra_2eproto[105]); } // =================================================================== @@ -34793,7 +36079,7 @@ void Expression_MaskExpression_MapSelect_MapKeyExpression::InternalSwap(Expressi ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_MapSelect_MapKeyExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[103]); + file_level_metadata_substrait_2falgebra_2eproto[106]); } // =================================================================== @@ -35136,7 +36422,7 @@ void Expression_MaskExpression_MapSelect::InternalSwap(Expression_MaskExpression ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression_MapSelect::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[104]); + file_level_metadata_substrait_2falgebra_2eproto[107]); } // =================================================================== @@ -35363,7 +36649,7 @@ void Expression_MaskExpression::InternalSwap(Expression_MaskExpression* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_MaskExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[105]); + file_level_metadata_substrait_2falgebra_2eproto[108]); } // =================================================================== @@ -35402,7 +36688,7 @@ const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Expression_FieldReference_Root ::PROTOBUF_NAMESPACE_ID::Metadata Expression_FieldReference_RootReference::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[106]); + file_level_metadata_substrait_2falgebra_2eproto[109]); } // =================================================================== @@ -35580,7 +36866,7 @@ void Expression_FieldReference_OuterReference::InternalSwap(Expression_FieldRefe ::PROTOBUF_NAMESPACE_ID::Metadata Expression_FieldReference_OuterReference::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[107]); + file_level_metadata_substrait_2falgebra_2eproto[110]); } // =================================================================== @@ -36084,7 +37370,7 @@ void Expression_FieldReference::InternalSwap(Expression_FieldReference* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_FieldReference::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[108]); + file_level_metadata_substrait_2falgebra_2eproto[111]); } // =================================================================== @@ -36279,7 +37565,7 @@ void Expression_Subquery_Scalar::InternalSwap(Expression_Subquery_Scalar* other) ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Subquery_Scalar::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[109]); + file_level_metadata_substrait_2falgebra_2eproto[112]); } // =================================================================== @@ -36507,7 +37793,7 @@ void Expression_Subquery_InPredicate::InternalSwap(Expression_Subquery_InPredica ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Subquery_InPredicate::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[110]); + file_level_metadata_substrait_2falgebra_2eproto[113]); } // =================================================================== @@ -36737,7 +38023,7 @@ void Expression_Subquery_SetPredicate::InternalSwap(Expression_Subquery_SetPredi ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Subquery_SetPredicate::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[111]); + file_level_metadata_substrait_2falgebra_2eproto[114]); } // =================================================================== @@ -37037,7 +38323,7 @@ void Expression_Subquery_SetComparison::InternalSwap(Expression_Subquery_SetComp ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Subquery_SetComparison::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[112]); + file_level_metadata_substrait_2falgebra_2eproto[115]); } // =================================================================== @@ -37451,7 +38737,7 @@ void Expression_Subquery::InternalSwap(Expression_Subquery* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression_Subquery::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[113]); + file_level_metadata_substrait_2falgebra_2eproto[116]); } // =================================================================== @@ -38321,7 +39607,7 @@ void Expression::InternalSwap(Expression* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Expression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[114]); + file_level_metadata_substrait_2falgebra_2eproto[117]); } // =================================================================== @@ -38615,7 +39901,7 @@ void SortField::InternalSwap(SortField* other) { ::PROTOBUF_NAMESPACE_ID::Metadata SortField::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[115]); + file_level_metadata_substrait_2falgebra_2eproto[118]); } // =================================================================== @@ -39034,7 +40320,7 @@ void AggregateFunction::InternalSwap(AggregateFunction* other) { ::PROTOBUF_NAMESPACE_ID::Metadata AggregateFunction::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[116]); + file_level_metadata_substrait_2falgebra_2eproto[119]); } // =================================================================== @@ -39212,7 +40498,7 @@ void ReferenceRel::InternalSwap(ReferenceRel* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ReferenceRel::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2falgebra_2eproto_getter, &descriptor_table_substrait_2falgebra_2eproto_once, - file_level_metadata_substrait_2falgebra_2eproto[117]); + file_level_metadata_substrait_2falgebra_2eproto[120]); } // @@protoc_insertion_point(namespace_scope) @@ -39257,6 +40543,9 @@ template<> PROTOBUF_NOINLINE ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcRead template<> PROTOBUF_NOINLINE ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* Arena::CreateMaybeMessage< ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* Arena::CreateMaybeMessage< ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::ReadRel_LocalFiles_FileOrFiles* Arena::CreateMaybeMessage< ::substrait::ReadRel_LocalFiles_FileOrFiles >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::ReadRel_LocalFiles_FileOrFiles >(arena); } @@ -39374,11 +40663,11 @@ template<> PROTOBUF_NOINLINE ::substrait::MergeJoinRel* Arena::CreateMaybeMessag template<> PROTOBUF_NOINLINE ::substrait::NestedLoopJoinRel* Arena::CreateMaybeMessage< ::substrait::NestedLoopJoinRel >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::NestedLoopJoinRel >(arena); } -template<> PROTOBUF_NOINLINE ::substrait::MarkJoinRel* Arena::CreateMaybeMessage< ::substrait::MarkJoinRel >(Arena* arena) { - return Arena::CreateMessageInternal< ::substrait::MarkJoinRel >(arena); +template<> PROTOBUF_NOINLINE ::substrait::DuplicateEliminatedGetRel* Arena::CreateMaybeMessage< ::substrait::DuplicateEliminatedGetRel >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::DuplicateEliminatedGetRel >(arena); } -template<> PROTOBUF_NOINLINE ::substrait::DelimiterJoinRel* Arena::CreateMaybeMessage< ::substrait::DelimiterJoinRel >(Arena* arena) { - return Arena::CreateMessageInternal< ::substrait::DelimiterJoinRel >(arena); +template<> PROTOBUF_NOINLINE ::substrait::DuplicateEliminatedJoinRel* Arena::CreateMaybeMessage< ::substrait::DuplicateEliminatedJoinRel >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::DuplicateEliminatedJoinRel >(arena); } template<> PROTOBUF_NOINLINE ::substrait::FunctionArgument* Arena::CreateMaybeMessage< ::substrait::FunctionArgument >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::FunctionArgument >(arena); @@ -39398,6 +40687,9 @@ template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_VarChar* Arena::Cre template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_Decimal* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_Decimal >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Expression_Literal_Decimal >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_PrecisionTimestamp* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_PrecisionTimestamp >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::Expression_Literal_PrecisionTimestamp >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_Map_KeyValue* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_Map_KeyValue >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Expression_Literal_Map_KeyValue >(arena); } @@ -39410,6 +40702,9 @@ template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_IntervalYearToMonth template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_IntervalDayToSecond* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_IntervalDayToSecond >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Expression_Literal_IntervalDayToSecond >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_IntervalCompound* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_IntervalCompound >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::Expression_Literal_IntervalCompound >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::Expression_Literal_Struct* Arena::CreateMaybeMessage< ::substrait::Expression_Literal_Struct >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Expression_Literal_Struct >(arena); } diff --git a/third_party/substrait/substrait/algebra.pb.h b/third_party/substrait/substrait/algebra.pb.h index eefa158..b4be474 100644 --- a/third_party/substrait/substrait/algebra.pb.h +++ b/third_party/substrait/substrait/algebra.pb.h @@ -51,7 +51,7 @@ struct TableStruct_substrait_2falgebra_2eproto { PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::AuxiliaryParseTableField aux[] PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[118] + static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[121] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[]; static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[]; @@ -89,9 +89,12 @@ extern CrossRelDefaultTypeInternal _CrossRel_default_instance_; class DdlRel; struct DdlRelDefaultTypeInternal; extern DdlRelDefaultTypeInternal _DdlRel_default_instance_; -class DelimiterJoinRel; -struct DelimiterJoinRelDefaultTypeInternal; -extern DelimiterJoinRelDefaultTypeInternal _DelimiterJoinRel_default_instance_; +class DuplicateEliminatedGetRel; +struct DuplicateEliminatedGetRelDefaultTypeInternal; +extern DuplicateEliminatedGetRelDefaultTypeInternal _DuplicateEliminatedGetRel_default_instance_; +class DuplicateEliminatedJoinRel; +struct DuplicateEliminatedJoinRelDefaultTypeInternal; +extern DuplicateEliminatedJoinRelDefaultTypeInternal _DuplicateEliminatedJoinRel_default_instance_; class ExchangeRel; struct ExchangeRelDefaultTypeInternal; extern ExchangeRelDefaultTypeInternal _ExchangeRel_default_instance_; @@ -164,6 +167,9 @@ extern Expression_LiteralDefaultTypeInternal _Expression_Literal_default_instanc class Expression_Literal_Decimal; struct Expression_Literal_DecimalDefaultTypeInternal; extern Expression_Literal_DecimalDefaultTypeInternal _Expression_Literal_Decimal_default_instance_; +class Expression_Literal_IntervalCompound; +struct Expression_Literal_IntervalCompoundDefaultTypeInternal; +extern Expression_Literal_IntervalCompoundDefaultTypeInternal _Expression_Literal_IntervalCompound_default_instance_; class Expression_Literal_IntervalDayToSecond; struct Expression_Literal_IntervalDayToSecondDefaultTypeInternal; extern Expression_Literal_IntervalDayToSecondDefaultTypeInternal _Expression_Literal_IntervalDayToSecond_default_instance_; @@ -179,6 +185,9 @@ extern Expression_Literal_MapDefaultTypeInternal _Expression_Literal_Map_default class Expression_Literal_Map_KeyValue; struct Expression_Literal_Map_KeyValueDefaultTypeInternal; extern Expression_Literal_Map_KeyValueDefaultTypeInternal _Expression_Literal_Map_KeyValue_default_instance_; +class Expression_Literal_PrecisionTimestamp; +struct Expression_Literal_PrecisionTimestampDefaultTypeInternal; +extern Expression_Literal_PrecisionTimestampDefaultTypeInternal _Expression_Literal_PrecisionTimestamp_default_instance_; class Expression_Literal_Struct; struct Expression_Literal_StructDefaultTypeInternal; extern Expression_Literal_StructDefaultTypeInternal _Expression_Literal_Struct_default_instance_; @@ -329,9 +338,6 @@ extern HashJoinRelDefaultTypeInternal _HashJoinRel_default_instance_; class JoinRel; struct JoinRelDefaultTypeInternal; extern JoinRelDefaultTypeInternal _JoinRel_default_instance_; -class MarkJoinRel; -struct MarkJoinRelDefaultTypeInternal; -extern MarkJoinRelDefaultTypeInternal _MarkJoinRel_default_instance_; class MergeJoinRel; struct MergeJoinRelDefaultTypeInternal; extern MergeJoinRelDefaultTypeInternal _MergeJoinRel_default_instance_; @@ -359,6 +365,9 @@ extern ReadRel_LocalFiles_FileOrFilesDefaultTypeInternal _ReadRel_LocalFiles_Fil class ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions; struct ReadRel_LocalFiles_FileOrFiles_ArrowReadOptionsDefaultTypeInternal; extern ReadRel_LocalFiles_FileOrFiles_ArrowReadOptionsDefaultTypeInternal _ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions_default_instance_; +class ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions; +struct ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal; +extern ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptionsDefaultTypeInternal _ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions_default_instance_; class ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions; struct ReadRel_LocalFiles_FileOrFiles_DwrfReadOptionsDefaultTypeInternal; extern ReadRel_LocalFiles_FileOrFiles_DwrfReadOptionsDefaultTypeInternal _ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions_default_instance_; @@ -425,7 +434,8 @@ template<> ::substrait::ConsistentPartitionWindowRel* Arena::CreateMaybeMessage< template<> ::substrait::ConsistentPartitionWindowRel_WindowRelFunction* Arena::CreateMaybeMessage<::substrait::ConsistentPartitionWindowRel_WindowRelFunction>(Arena*); template<> ::substrait::CrossRel* Arena::CreateMaybeMessage<::substrait::CrossRel>(Arena*); template<> ::substrait::DdlRel* Arena::CreateMaybeMessage<::substrait::DdlRel>(Arena*); -template<> ::substrait::DelimiterJoinRel* Arena::CreateMaybeMessage<::substrait::DelimiterJoinRel>(Arena*); +template<> ::substrait::DuplicateEliminatedGetRel* Arena::CreateMaybeMessage<::substrait::DuplicateEliminatedGetRel>(Arena*); +template<> ::substrait::DuplicateEliminatedJoinRel* Arena::CreateMaybeMessage<::substrait::DuplicateEliminatedJoinRel>(Arena*); template<> ::substrait::ExchangeRel* Arena::CreateMaybeMessage<::substrait::ExchangeRel>(Arena*); template<> ::substrait::ExchangeRel_Broadcast* Arena::CreateMaybeMessage<::substrait::ExchangeRel_Broadcast>(Arena*); template<> ::substrait::ExchangeRel_ExchangeTarget* Arena::CreateMaybeMessage<::substrait::ExchangeRel_ExchangeTarget>(Arena*); @@ -450,11 +460,13 @@ template<> ::substrait::Expression_IfThen* Arena::CreateMaybeMessage<::substrait template<> ::substrait::Expression_IfThen_IfClause* Arena::CreateMaybeMessage<::substrait::Expression_IfThen_IfClause>(Arena*); template<> ::substrait::Expression_Literal* Arena::CreateMaybeMessage<::substrait::Expression_Literal>(Arena*); template<> ::substrait::Expression_Literal_Decimal* Arena::CreateMaybeMessage<::substrait::Expression_Literal_Decimal>(Arena*); +template<> ::substrait::Expression_Literal_IntervalCompound* Arena::CreateMaybeMessage<::substrait::Expression_Literal_IntervalCompound>(Arena*); template<> ::substrait::Expression_Literal_IntervalDayToSecond* Arena::CreateMaybeMessage<::substrait::Expression_Literal_IntervalDayToSecond>(Arena*); template<> ::substrait::Expression_Literal_IntervalYearToMonth* Arena::CreateMaybeMessage<::substrait::Expression_Literal_IntervalYearToMonth>(Arena*); template<> ::substrait::Expression_Literal_List* Arena::CreateMaybeMessage<::substrait::Expression_Literal_List>(Arena*); template<> ::substrait::Expression_Literal_Map* Arena::CreateMaybeMessage<::substrait::Expression_Literal_Map>(Arena*); template<> ::substrait::Expression_Literal_Map_KeyValue* Arena::CreateMaybeMessage<::substrait::Expression_Literal_Map_KeyValue>(Arena*); +template<> ::substrait::Expression_Literal_PrecisionTimestamp* Arena::CreateMaybeMessage<::substrait::Expression_Literal_PrecisionTimestamp>(Arena*); template<> ::substrait::Expression_Literal_Struct* Arena::CreateMaybeMessage<::substrait::Expression_Literal_Struct>(Arena*); template<> ::substrait::Expression_Literal_UserDefined* Arena::CreateMaybeMessage<::substrait::Expression_Literal_UserDefined>(Arena*); template<> ::substrait::Expression_Literal_VarChar* Arena::CreateMaybeMessage<::substrait::Expression_Literal_VarChar>(Arena*); @@ -505,7 +517,6 @@ template<> ::substrait::FunctionArgument* Arena::CreateMaybeMessage<::substrait: template<> ::substrait::FunctionOption* Arena::CreateMaybeMessage<::substrait::FunctionOption>(Arena*); template<> ::substrait::HashJoinRel* Arena::CreateMaybeMessage<::substrait::HashJoinRel>(Arena*); template<> ::substrait::JoinRel* Arena::CreateMaybeMessage<::substrait::JoinRel>(Arena*); -template<> ::substrait::MarkJoinRel* Arena::CreateMaybeMessage<::substrait::MarkJoinRel>(Arena*); template<> ::substrait::MergeJoinRel* Arena::CreateMaybeMessage<::substrait::MergeJoinRel>(Arena*); template<> ::substrait::NamedObjectWrite* Arena::CreateMaybeMessage<::substrait::NamedObjectWrite>(Arena*); template<> ::substrait::NestedLoopJoinRel* Arena::CreateMaybeMessage<::substrait::NestedLoopJoinRel>(Arena*); @@ -515,6 +526,7 @@ template<> ::substrait::ReadRel_ExtensionTable* Arena::CreateMaybeMessage<::subs template<> ::substrait::ReadRel_LocalFiles* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles>(Arena*); template<> ::substrait::ReadRel_LocalFiles_FileOrFiles* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles>(Arena*); template<> ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions>(Arena*); +template<> ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions>(Arena*); template<> ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions>(Arena*); template<> ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions>(Arena*); template<> ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* Arena::CreateMaybeMessage<::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions>(Arena*); @@ -542,16 +554,20 @@ enum JoinRel_JoinType : int { JoinRel_JoinType_JOIN_TYPE_OUTER = 2, JoinRel_JoinType_JOIN_TYPE_LEFT = 3, JoinRel_JoinType_JOIN_TYPE_RIGHT = 4, - JoinRel_JoinType_JOIN_TYPE_SEMI = 5, - JoinRel_JoinType_JOIN_TYPE_ANTI = 6, - JoinRel_JoinType_JOIN_TYPE_SINGLE = 7, - JoinRel_JoinType_JOIN_TYPE_MARK = 8, + JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI = 5, + JoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 6, + JoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE = 7, + JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 8, + JoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 9, + JoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE = 10, + JoinRel_JoinType_JOIN_TYPE_LEFT_MARK = 11, + JoinRel_JoinType_JOIN_TYPE_RIGHT_MARK = 12, JoinRel_JoinType_JoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), JoinRel_JoinType_JoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() }; bool JoinRel_JoinType_IsValid(int value); constexpr JoinRel_JoinType JoinRel_JoinType_JoinType_MIN = JoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; -constexpr JoinRel_JoinType JoinRel_JoinType_JoinType_MAX = JoinRel_JoinType_JOIN_TYPE_MARK; +constexpr JoinRel_JoinType JoinRel_JoinType_JoinType_MAX = JoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; constexpr int JoinRel_JoinType_JoinType_ARRAYSIZE = JoinRel_JoinType_JoinType_MAX + 1; const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* JoinRel_JoinType_descriptor(); @@ -744,12 +760,16 @@ enum HashJoinRel_JoinType : int { HashJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 6, HashJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 7, HashJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 8, + HashJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE = 9, + HashJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE = 10, + HashJoinRel_JoinType_JOIN_TYPE_LEFT_MARK = 11, + HashJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK = 12, HashJoinRel_JoinType_HashJoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), HashJoinRel_JoinType_HashJoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() }; bool HashJoinRel_JoinType_IsValid(int value); constexpr HashJoinRel_JoinType HashJoinRel_JoinType_JoinType_MIN = HashJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; -constexpr HashJoinRel_JoinType HashJoinRel_JoinType_JoinType_MAX = HashJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; +constexpr HashJoinRel_JoinType HashJoinRel_JoinType_JoinType_MAX = HashJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; constexpr int HashJoinRel_JoinType_JoinType_ARRAYSIZE = HashJoinRel_JoinType_JoinType_MAX + 1; const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* HashJoinRel_JoinType_descriptor(); @@ -776,12 +796,16 @@ enum MergeJoinRel_JoinType : int { MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 6, MergeJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 7, MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 8, + MergeJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE = 9, + MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE = 10, + MergeJoinRel_JoinType_JOIN_TYPE_LEFT_MARK = 11, + MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK = 12, MergeJoinRel_JoinType_MergeJoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), MergeJoinRel_JoinType_MergeJoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() }; bool MergeJoinRel_JoinType_IsValid(int value); constexpr MergeJoinRel_JoinType MergeJoinRel_JoinType_JoinType_MIN = MergeJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; -constexpr MergeJoinRel_JoinType MergeJoinRel_JoinType_JoinType_MAX = MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; +constexpr MergeJoinRel_JoinType MergeJoinRel_JoinType_JoinType_MAX = MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; constexpr int MergeJoinRel_JoinType_JoinType_ARRAYSIZE = MergeJoinRel_JoinType_JoinType_MAX + 1; const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* MergeJoinRel_JoinType_descriptor(); @@ -808,12 +832,16 @@ enum NestedLoopJoinRel_JoinType : int { NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 6, NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 7, NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 8, + NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE = 9, + NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE = 10, + NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_MARK = 11, + NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK = 12, NestedLoopJoinRel_JoinType_NestedLoopJoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), NestedLoopJoinRel_JoinType_NestedLoopJoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() }; bool NestedLoopJoinRel_JoinType_IsValid(int value); constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel_JoinType_JoinType_MIN = NestedLoopJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; -constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel_JoinType_JoinType_MAX = NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; +constexpr NestedLoopJoinRel_JoinType NestedLoopJoinRel_JoinType_JoinType_MAX = NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; constexpr int NestedLoopJoinRel_JoinType_JoinType_ARRAYSIZE = NestedLoopJoinRel_JoinType_JoinType_MAX + 1; const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* NestedLoopJoinRel_JoinType_descriptor(); @@ -830,37 +858,67 @@ inline bool NestedLoopJoinRel_JoinType_Parse( return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum( NestedLoopJoinRel_JoinType_descriptor(), name, value); } -enum DelimiterJoinRel_JoinType : int { - DelimiterJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED = 0, - DelimiterJoinRel_JoinType_JOIN_TYPE_INNER = 1, - DelimiterJoinRel_JoinType_JOIN_TYPE_OUTER = 2, - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT = 3, - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT = 4, - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT_SEMI = 5, - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 6, - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 7, - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 8, - DelimiterJoinRel_JoinType_DelimiterJoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), - DelimiterJoinRel_JoinType_DelimiterJoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() +enum DuplicateEliminatedJoinRel_DuplicateEliminatedSide : int { + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_UNSPECIFIED = 0, + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_LEFT = 1, + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_RIGHT = 2, + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedJoinRel_DuplicateEliminatedSide_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedJoinRel_DuplicateEliminatedSide_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() }; -bool DelimiterJoinRel_JoinType_IsValid(int value); -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel_JoinType_JoinType_MIN = DelimiterJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; -constexpr DelimiterJoinRel_JoinType DelimiterJoinRel_JoinType_JoinType_MAX = DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; -constexpr int DelimiterJoinRel_JoinType_JoinType_ARRAYSIZE = DelimiterJoinRel_JoinType_JoinType_MAX + 1; +bool DuplicateEliminatedJoinRel_DuplicateEliminatedSide_IsValid(int value); +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_MIN = DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_UNSPECIFIED; +constexpr DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_MAX = DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_RIGHT; +constexpr int DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_ARRAYSIZE = DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_MAX + 1; -const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DelimiterJoinRel_JoinType_descriptor(); +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor(); template -inline const std::string& DelimiterJoinRel_JoinType_Name(T enum_t_value) { - static_assert(::std::is_same::value || +inline const std::string& DuplicateEliminatedJoinRel_DuplicateEliminatedSide_Name(T enum_t_value) { + static_assert(::std::is_same::value || ::std::is_integral::value, - "Incorrect type passed to function DelimiterJoinRel_JoinType_Name."); + "Incorrect type passed to function DuplicateEliminatedJoinRel_DuplicateEliminatedSide_Name."); return ::PROTOBUF_NAMESPACE_ID::internal::NameOfEnum( - DelimiterJoinRel_JoinType_descriptor(), enum_t_value); + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor(), enum_t_value); +} +inline bool DuplicateEliminatedJoinRel_DuplicateEliminatedSide_Parse( + ::PROTOBUF_NAMESPACE_ID::ConstStringParam name, DuplicateEliminatedJoinRel_DuplicateEliminatedSide* value) { + return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum( + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor(), name, value); +} +enum DuplicateEliminatedJoinRel_JoinType : int { + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED = 0, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_INNER = 1, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_OUTER = 2, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT = 3, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT = 4, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SEMI = 5, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI = 6, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE = 7, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI = 8, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI = 9, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE = 10, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_MARK = 11, + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK = 12, + DuplicateEliminatedJoinRel_JoinType_DuplicateEliminatedJoinRel_JoinType_INT_MIN_SENTINEL_DO_NOT_USE_ = std::numeric_limits::min(), + DuplicateEliminatedJoinRel_JoinType_DuplicateEliminatedJoinRel_JoinType_INT_MAX_SENTINEL_DO_NOT_USE_ = std::numeric_limits::max() +}; +bool DuplicateEliminatedJoinRel_JoinType_IsValid(int value); +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel_JoinType_JoinType_MIN = DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; +constexpr DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel_JoinType_JoinType_MAX = DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; +constexpr int DuplicateEliminatedJoinRel_JoinType_JoinType_ARRAYSIZE = DuplicateEliminatedJoinRel_JoinType_JoinType_MAX + 1; + +const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* DuplicateEliminatedJoinRel_JoinType_descriptor(); +template +inline const std::string& DuplicateEliminatedJoinRel_JoinType_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function DuplicateEliminatedJoinRel_JoinType_Name."); + return ::PROTOBUF_NAMESPACE_ID::internal::NameOfEnum( + DuplicateEliminatedJoinRel_JoinType_descriptor(), enum_t_value); } -inline bool DelimiterJoinRel_JoinType_Parse( - ::PROTOBUF_NAMESPACE_ID::ConstStringParam name, DelimiterJoinRel_JoinType* value) { - return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum( - DelimiterJoinRel_JoinType_descriptor(), name, value); +inline bool DuplicateEliminatedJoinRel_JoinType_Parse( + ::PROTOBUF_NAMESPACE_ID::ConstStringParam name, DuplicateEliminatedJoinRel_JoinType* value) { + return ::PROTOBUF_NAMESPACE_ID::internal::ParseNamedEnum( + DuplicateEliminatedJoinRel_JoinType_descriptor(), name, value); } enum Expression_WindowFunction_BoundsType : int { Expression_WindowFunction_BoundsType_BOUNDS_TYPE_UNSPECIFIED = 0, @@ -1816,10 +1874,25 @@ class RelCommon_Hint final : // accessors ------------------------------------------------------- enum : int { + kAliasFieldNumber = 3, kStatsFieldNumber = 1, kConstraintFieldNumber = 2, kAdvancedExtensionFieldNumber = 10, }; + // string alias = 3; + void clear_alias(); + const std::string& alias() const; + template + void set_alias(ArgT0&& arg0, ArgT... args); + std::string* mutable_alias(); + PROTOBUF_NODISCARD std::string* release_alias(); + void set_allocated_alias(std::string* alias); + private: + const std::string& _internal_alias() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_alias(const std::string& value); + std::string* _internal_mutable_alias(); + public: + // .substrait.RelCommon.Hint.Stats stats = 1; bool has_stats() const; private: @@ -1881,6 +1954,7 @@ class RelCommon_Hint final : template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr alias_; ::substrait::RelCommon_Hint_Stats* stats_; ::substrait::RelCommon_Hint_RuntimeConstraint* constraint_; ::substrait::extensions::AdvancedExtension* advanced_extension_; @@ -3090,24 +3164,24 @@ class ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions final : }; // ------------------------------------------------------------------- -class ReadRel_LocalFiles_FileOrFiles final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ReadRel.LocalFiles.FileOrFiles) */ { +class ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) */ { public: - inline ReadRel_LocalFiles_FileOrFiles() : ReadRel_LocalFiles_FileOrFiles(nullptr) {} - ~ReadRel_LocalFiles_FileOrFiles() override; - explicit constexpr ReadRel_LocalFiles_FileOrFiles(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions() : ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(nullptr) {} + ~ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions() override; + explicit constexpr ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - ReadRel_LocalFiles_FileOrFiles(const ReadRel_LocalFiles_FileOrFiles& from); - ReadRel_LocalFiles_FileOrFiles(ReadRel_LocalFiles_FileOrFiles&& from) noexcept - : ReadRel_LocalFiles_FileOrFiles() { + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from); + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions&& from) noexcept + : ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions() { *this = ::std::move(from); } - inline ReadRel_LocalFiles_FileOrFiles& operator=(const ReadRel_LocalFiles_FileOrFiles& from) { + inline ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& operator=(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from) { CopyFrom(from); return *this; } - inline ReadRel_LocalFiles_FileOrFiles& operator=(ReadRel_LocalFiles_FileOrFiles&& from) noexcept { + inline ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& operator=(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -3130,37 +3204,20 @@ class ReadRel_LocalFiles_FileOrFiles final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const ReadRel_LocalFiles_FileOrFiles& default_instance() { + static const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& default_instance() { return *internal_default_instance(); } - enum PathTypeCase { - kUriPath = 1, - kUriPathGlob = 2, - kUriFile = 3, - kUriFolder = 4, - PATH_TYPE_NOT_SET = 0, - }; - - enum FileFormatCase { - kParquet = 9, - kArrow = 10, - kOrc = 11, - kExtension = 12, - kDwrf = 13, - FILE_FORMAT_NOT_SET = 0, - }; - - static inline const ReadRel_LocalFiles_FileOrFiles* internal_default_instance() { - return reinterpret_cast( - &_ReadRel_LocalFiles_FileOrFiles_default_instance_); + static inline const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* internal_default_instance() { + return reinterpret_cast( + &_ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions_default_instance_); } static constexpr int kIndexInFileMessages = 13; - friend void swap(ReadRel_LocalFiles_FileOrFiles& a, ReadRel_LocalFiles_FileOrFiles& b) { + friend void swap(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& a, ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& b) { a.Swap(&b); } - inline void Swap(ReadRel_LocalFiles_FileOrFiles* other) { + inline void Swap(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -3173,7 +3230,7 @@ class ReadRel_LocalFiles_FileOrFiles final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(ReadRel_LocalFiles_FileOrFiles* other) { + void UnsafeArenaSwap(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -3181,13 +3238,13 @@ class ReadRel_LocalFiles_FileOrFiles final : // implements Message ---------------------------------------------- - ReadRel_LocalFiles_FileOrFiles* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const ReadRel_LocalFiles_FileOrFiles& from); + void CopyFrom(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const ReadRel_LocalFiles_FileOrFiles& from); + void MergeFrom(const ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -3204,15 +3261,15 @@ class ReadRel_LocalFiles_FileOrFiles final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(ReadRel_LocalFiles_FileOrFiles* other); + void InternalSwap(ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.ReadRel.LocalFiles.FileOrFiles"; + return "substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions"; } protected: - explicit ReadRel_LocalFiles_FileOrFiles(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -3226,287 +3283,553 @@ class ReadRel_LocalFiles_FileOrFiles final : // nested types ---------------------------------------------------- - typedef ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions ParquetReadOptions; - typedef ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions ArrowReadOptions; - typedef ReadRel_LocalFiles_FileOrFiles_OrcReadOptions OrcReadOptions; - typedef ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions DwrfReadOptions; - // accessors ------------------------------------------------------- enum : int { - kPartitionIndexFieldNumber = 6, - kStartFieldNumber = 7, - kLengthFieldNumber = 8, - kUriPathFieldNumber = 1, - kUriPathGlobFieldNumber = 2, - kUriFileFieldNumber = 3, - kUriFolderFieldNumber = 4, - kParquetFieldNumber = 9, - kArrowFieldNumber = 10, - kOrcFieldNumber = 11, - kExtensionFieldNumber = 12, - kDwrfFieldNumber = 13, + kFieldDelimiterFieldNumber = 1, + kQuoteFieldNumber = 3, + kEscapeFieldNumber = 5, + kValueTreatedAsNullFieldNumber = 6, + kMaxLineSizeFieldNumber = 2, + kHeaderLinesToSkipFieldNumber = 4, }; - // uint64 partition_index = 6; - void clear_partition_index(); - uint64_t partition_index() const; - void set_partition_index(uint64_t value); - private: - uint64_t _internal_partition_index() const; - void _internal_set_partition_index(uint64_t value); - public: - - // uint64 start = 7; - void clear_start(); - uint64_t start() const; - void set_start(uint64_t value); - private: - uint64_t _internal_start() const; - void _internal_set_start(uint64_t value); - public: - - // uint64 length = 8; - void clear_length(); - uint64_t length() const; - void set_length(uint64_t value); - private: - uint64_t _internal_length() const; - void _internal_set_length(uint64_t value); - public: - - // string uri_path = 1; - bool has_uri_path() const; - private: - bool _internal_has_uri_path() const; - public: - void clear_uri_path(); - const std::string& uri_path() const; + // string field_delimiter = 1; + void clear_field_delimiter(); + const std::string& field_delimiter() const; template - void set_uri_path(ArgT0&& arg0, ArgT... args); - std::string* mutable_uri_path(); - PROTOBUF_NODISCARD std::string* release_uri_path(); - void set_allocated_uri_path(std::string* uri_path); + void set_field_delimiter(ArgT0&& arg0, ArgT... args); + std::string* mutable_field_delimiter(); + PROTOBUF_NODISCARD std::string* release_field_delimiter(); + void set_allocated_field_delimiter(std::string* field_delimiter); private: - const std::string& _internal_uri_path() const; - inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_path(const std::string& value); - std::string* _internal_mutable_uri_path(); + const std::string& _internal_field_delimiter() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_field_delimiter(const std::string& value); + std::string* _internal_mutable_field_delimiter(); public: - // string uri_path_glob = 2; - bool has_uri_path_glob() const; - private: - bool _internal_has_uri_path_glob() const; - public: - void clear_uri_path_glob(); - const std::string& uri_path_glob() const; + // string quote = 3; + void clear_quote(); + const std::string& quote() const; template - void set_uri_path_glob(ArgT0&& arg0, ArgT... args); - std::string* mutable_uri_path_glob(); - PROTOBUF_NODISCARD std::string* release_uri_path_glob(); - void set_allocated_uri_path_glob(std::string* uri_path_glob); + void set_quote(ArgT0&& arg0, ArgT... args); + std::string* mutable_quote(); + PROTOBUF_NODISCARD std::string* release_quote(); + void set_allocated_quote(std::string* quote); private: - const std::string& _internal_uri_path_glob() const; - inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_path_glob(const std::string& value); - std::string* _internal_mutable_uri_path_glob(); + const std::string& _internal_quote() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_quote(const std::string& value); + std::string* _internal_mutable_quote(); public: - // string uri_file = 3; - bool has_uri_file() const; - private: - bool _internal_has_uri_file() const; - public: - void clear_uri_file(); - const std::string& uri_file() const; + // string escape = 5; + void clear_escape(); + const std::string& escape() const; template - void set_uri_file(ArgT0&& arg0, ArgT... args); - std::string* mutable_uri_file(); - PROTOBUF_NODISCARD std::string* release_uri_file(); - void set_allocated_uri_file(std::string* uri_file); + void set_escape(ArgT0&& arg0, ArgT... args); + std::string* mutable_escape(); + PROTOBUF_NODISCARD std::string* release_escape(); + void set_allocated_escape(std::string* escape); private: - const std::string& _internal_uri_file() const; - inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_file(const std::string& value); - std::string* _internal_mutable_uri_file(); + const std::string& _internal_escape() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_escape(const std::string& value); + std::string* _internal_mutable_escape(); public: - // string uri_folder = 4; - bool has_uri_folder() const; + // optional string value_treated_as_null = 6; + bool has_value_treated_as_null() const; private: - bool _internal_has_uri_folder() const; + bool _internal_has_value_treated_as_null() const; public: - void clear_uri_folder(); - const std::string& uri_folder() const; + void clear_value_treated_as_null(); + const std::string& value_treated_as_null() const; template - void set_uri_folder(ArgT0&& arg0, ArgT... args); - std::string* mutable_uri_folder(); - PROTOBUF_NODISCARD std::string* release_uri_folder(); - void set_allocated_uri_folder(std::string* uri_folder); - private: - const std::string& _internal_uri_folder() const; - inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_folder(const std::string& value); - std::string* _internal_mutable_uri_folder(); - public: - - // .substrait.ReadRel.LocalFiles.FileOrFiles.ParquetReadOptions parquet = 9; - bool has_parquet() const; + void set_value_treated_as_null(ArgT0&& arg0, ArgT... args); + std::string* mutable_value_treated_as_null(); + PROTOBUF_NODISCARD std::string* release_value_treated_as_null(); + void set_allocated_value_treated_as_null(std::string* value_treated_as_null); private: - bool _internal_has_parquet() const; + const std::string& _internal_value_treated_as_null() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_value_treated_as_null(const std::string& value); + std::string* _internal_mutable_value_treated_as_null(); public: - void clear_parquet(); - const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& parquet() const; - PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* release_parquet(); - ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* mutable_parquet(); - void set_allocated_parquet(::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet); - private: - const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& _internal_parquet() const; - ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* _internal_mutable_parquet(); - public: - void unsafe_arena_set_allocated_parquet( - ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet); - ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* unsafe_arena_release_parquet(); - - // .substrait.ReadRel.LocalFiles.FileOrFiles.ArrowReadOptions arrow = 10; - bool has_arrow() const; - private: - bool _internal_has_arrow() const; - public: - void clear_arrow(); - const ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions& arrow() const; - PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* release_arrow(); - ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* mutable_arrow(); - void set_allocated_arrow(::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow); - private: - const ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions& _internal_arrow() const; - ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* _internal_mutable_arrow(); - public: - void unsafe_arena_set_allocated_arrow( - ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow); - ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* unsafe_arena_release_arrow(); - // .substrait.ReadRel.LocalFiles.FileOrFiles.OrcReadOptions orc = 11; - bool has_orc() const; + // uint64 max_line_size = 2; + void clear_max_line_size(); + uint64_t max_line_size() const; + void set_max_line_size(uint64_t value); private: - bool _internal_has_orc() const; + uint64_t _internal_max_line_size() const; + void _internal_set_max_line_size(uint64_t value); public: - void clear_orc(); - const ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions& orc() const; - PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* release_orc(); - ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* mutable_orc(); - void set_allocated_orc(::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc); - private: - const ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions& _internal_orc() const; - ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* _internal_mutable_orc(); - public: - void unsafe_arena_set_allocated_orc( - ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc); - ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* unsafe_arena_release_orc(); - // .google.protobuf.Any extension = 12; - bool has_extension() const; + // uint64 header_lines_to_skip = 4; + void clear_header_lines_to_skip(); + uint64_t header_lines_to_skip() const; + void set_header_lines_to_skip(uint64_t value); private: - bool _internal_has_extension() const; + uint64_t _internal_header_lines_to_skip() const; + void _internal_set_header_lines_to_skip(uint64_t value); public: - void clear_extension(); - const ::PROTOBUF_NAMESPACE_ID::Any& extension() const; - PROTOBUF_NODISCARD ::PROTOBUF_NAMESPACE_ID::Any* release_extension(); - ::PROTOBUF_NAMESPACE_ID::Any* mutable_extension(); - void set_allocated_extension(::PROTOBUF_NAMESPACE_ID::Any* extension); - private: - const ::PROTOBUF_NAMESPACE_ID::Any& _internal_extension() const; - ::PROTOBUF_NAMESPACE_ID::Any* _internal_mutable_extension(); - public: - void unsafe_arena_set_allocated_extension( - ::PROTOBUF_NAMESPACE_ID::Any* extension); - ::PROTOBUF_NAMESPACE_ID::Any* unsafe_arena_release_extension(); - // .substrait.ReadRel.LocalFiles.FileOrFiles.DwrfReadOptions dwrf = 13; - bool has_dwrf() const; - private: - bool _internal_has_dwrf() const; - public: - void clear_dwrf(); - const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& dwrf() const; - PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* release_dwrf(); - ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* mutable_dwrf(); - void set_allocated_dwrf(::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf); - private: - const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& _internal_dwrf() const; - ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* _internal_mutable_dwrf(); - public: - void unsafe_arena_set_allocated_dwrf( - ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf); - ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* unsafe_arena_release_dwrf(); - - void clear_path_type(); - PathTypeCase path_type_case() const; - void clear_file_format(); - FileFormatCase file_format_case() const; - // @@protoc_insertion_point(class_scope:substrait.ReadRel.LocalFiles.FileOrFiles) + // @@protoc_insertion_point(class_scope:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions) private: class _Internal; - void set_has_uri_path(); - void set_has_uri_path_glob(); - void set_has_uri_file(); - void set_has_uri_folder(); - void set_has_parquet(); - void set_has_arrow(); - void set_has_orc(); - void set_has_extension(); - void set_has_dwrf(); - - inline bool has_path_type() const; - inline void clear_has_path_type(); - - inline bool has_file_format() const; - inline void clear_has_file_format(); template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - uint64_t partition_index_; - uint64_t start_; - uint64_t length_; - union PathTypeUnion { - constexpr PathTypeUnion() : _constinit_{} {} - ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized _constinit_; - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_path_; - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_path_glob_; - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_file_; - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_folder_; - } path_type_; - union FileFormatUnion { - constexpr FileFormatUnion() : _constinit_{} {} - ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized _constinit_; - ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet_; - ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow_; - ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc_; - ::PROTOBUF_NAMESPACE_ID::Any* extension_; - ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf_; - } file_format_; + ::PROTOBUF_NAMESPACE_ID::internal::HasBits<1> _has_bits_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; - uint32_t _oneof_case_[2]; - + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr field_delimiter_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr quote_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr escape_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr value_treated_as_null_; + uint64_t max_line_size_; + uint64_t header_lines_to_skip_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; // ------------------------------------------------------------------- -class ReadRel_LocalFiles final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ReadRel.LocalFiles) */ { +class ReadRel_LocalFiles_FileOrFiles final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ReadRel.LocalFiles.FileOrFiles) */ { public: - inline ReadRel_LocalFiles() : ReadRel_LocalFiles(nullptr) {} - ~ReadRel_LocalFiles() override; - explicit constexpr ReadRel_LocalFiles(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline ReadRel_LocalFiles_FileOrFiles() : ReadRel_LocalFiles_FileOrFiles(nullptr) {} + ~ReadRel_LocalFiles_FileOrFiles() override; + explicit constexpr ReadRel_LocalFiles_FileOrFiles(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - ReadRel_LocalFiles(const ReadRel_LocalFiles& from); - ReadRel_LocalFiles(ReadRel_LocalFiles&& from) noexcept - : ReadRel_LocalFiles() { + ReadRel_LocalFiles_FileOrFiles(const ReadRel_LocalFiles_FileOrFiles& from); + ReadRel_LocalFiles_FileOrFiles(ReadRel_LocalFiles_FileOrFiles&& from) noexcept + : ReadRel_LocalFiles_FileOrFiles() { *this = ::std::move(from); } - inline ReadRel_LocalFiles& operator=(const ReadRel_LocalFiles& from) { + inline ReadRel_LocalFiles_FileOrFiles& operator=(const ReadRel_LocalFiles_FileOrFiles& from) { CopyFrom(from); return *this; } - inline ReadRel_LocalFiles& operator=(ReadRel_LocalFiles&& from) noexcept { + inline ReadRel_LocalFiles_FileOrFiles& operator=(ReadRel_LocalFiles_FileOrFiles&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const ReadRel_LocalFiles_FileOrFiles& default_instance() { + return *internal_default_instance(); + } + enum PathTypeCase { + kUriPath = 1, + kUriPathGlob = 2, + kUriFile = 3, + kUriFolder = 4, + PATH_TYPE_NOT_SET = 0, + }; + + enum FileFormatCase { + kParquet = 9, + kArrow = 10, + kOrc = 11, + kExtension = 12, + kDwrf = 13, + kText = 14, + FILE_FORMAT_NOT_SET = 0, + }; + + static inline const ReadRel_LocalFiles_FileOrFiles* internal_default_instance() { + return reinterpret_cast( + &_ReadRel_LocalFiles_FileOrFiles_default_instance_); + } + static constexpr int kIndexInFileMessages = + 14; + + friend void swap(ReadRel_LocalFiles_FileOrFiles& a, ReadRel_LocalFiles_FileOrFiles& b) { + a.Swap(&b); + } + inline void Swap(ReadRel_LocalFiles_FileOrFiles* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(ReadRel_LocalFiles_FileOrFiles* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + ReadRel_LocalFiles_FileOrFiles* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const ReadRel_LocalFiles_FileOrFiles& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const ReadRel_LocalFiles_FileOrFiles& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(ReadRel_LocalFiles_FileOrFiles* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.ReadRel.LocalFiles.FileOrFiles"; + } + protected: + explicit ReadRel_LocalFiles_FileOrFiles(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + typedef ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions ParquetReadOptions; + typedef ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions ArrowReadOptions; + typedef ReadRel_LocalFiles_FileOrFiles_OrcReadOptions OrcReadOptions; + typedef ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions DwrfReadOptions; + typedef ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions DelimiterSeparatedTextReadOptions; + + // accessors ------------------------------------------------------- + + enum : int { + kPartitionIndexFieldNumber = 6, + kStartFieldNumber = 7, + kLengthFieldNumber = 8, + kUriPathFieldNumber = 1, + kUriPathGlobFieldNumber = 2, + kUriFileFieldNumber = 3, + kUriFolderFieldNumber = 4, + kParquetFieldNumber = 9, + kArrowFieldNumber = 10, + kOrcFieldNumber = 11, + kExtensionFieldNumber = 12, + kDwrfFieldNumber = 13, + kTextFieldNumber = 14, + }; + // uint64 partition_index = 6; + void clear_partition_index(); + uint64_t partition_index() const; + void set_partition_index(uint64_t value); + private: + uint64_t _internal_partition_index() const; + void _internal_set_partition_index(uint64_t value); + public: + + // uint64 start = 7; + void clear_start(); + uint64_t start() const; + void set_start(uint64_t value); + private: + uint64_t _internal_start() const; + void _internal_set_start(uint64_t value); + public: + + // uint64 length = 8; + void clear_length(); + uint64_t length() const; + void set_length(uint64_t value); + private: + uint64_t _internal_length() const; + void _internal_set_length(uint64_t value); + public: + + // string uri_path = 1; + bool has_uri_path() const; + private: + bool _internal_has_uri_path() const; + public: + void clear_uri_path(); + const std::string& uri_path() const; + template + void set_uri_path(ArgT0&& arg0, ArgT... args); + std::string* mutable_uri_path(); + PROTOBUF_NODISCARD std::string* release_uri_path(); + void set_allocated_uri_path(std::string* uri_path); + private: + const std::string& _internal_uri_path() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_path(const std::string& value); + std::string* _internal_mutable_uri_path(); + public: + + // string uri_path_glob = 2; + bool has_uri_path_glob() const; + private: + bool _internal_has_uri_path_glob() const; + public: + void clear_uri_path_glob(); + const std::string& uri_path_glob() const; + template + void set_uri_path_glob(ArgT0&& arg0, ArgT... args); + std::string* mutable_uri_path_glob(); + PROTOBUF_NODISCARD std::string* release_uri_path_glob(); + void set_allocated_uri_path_glob(std::string* uri_path_glob); + private: + const std::string& _internal_uri_path_glob() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_path_glob(const std::string& value); + std::string* _internal_mutable_uri_path_glob(); + public: + + // string uri_file = 3; + bool has_uri_file() const; + private: + bool _internal_has_uri_file() const; + public: + void clear_uri_file(); + const std::string& uri_file() const; + template + void set_uri_file(ArgT0&& arg0, ArgT... args); + std::string* mutable_uri_file(); + PROTOBUF_NODISCARD std::string* release_uri_file(); + void set_allocated_uri_file(std::string* uri_file); + private: + const std::string& _internal_uri_file() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_file(const std::string& value); + std::string* _internal_mutable_uri_file(); + public: + + // string uri_folder = 4; + bool has_uri_folder() const; + private: + bool _internal_has_uri_folder() const; + public: + void clear_uri_folder(); + const std::string& uri_folder() const; + template + void set_uri_folder(ArgT0&& arg0, ArgT... args); + std::string* mutable_uri_folder(); + PROTOBUF_NODISCARD std::string* release_uri_folder(); + void set_allocated_uri_folder(std::string* uri_folder); + private: + const std::string& _internal_uri_folder() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_uri_folder(const std::string& value); + std::string* _internal_mutable_uri_folder(); + public: + + // .substrait.ReadRel.LocalFiles.FileOrFiles.ParquetReadOptions parquet = 9; + bool has_parquet() const; + private: + bool _internal_has_parquet() const; + public: + void clear_parquet(); + const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& parquet() const; + PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* release_parquet(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* mutable_parquet(); + void set_allocated_parquet(::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet); + private: + const ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions& _internal_parquet() const; + ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* _internal_mutable_parquet(); + public: + void unsafe_arena_set_allocated_parquet( + ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet); + ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* unsafe_arena_release_parquet(); + + // .substrait.ReadRel.LocalFiles.FileOrFiles.ArrowReadOptions arrow = 10; + bool has_arrow() const; + private: + bool _internal_has_arrow() const; + public: + void clear_arrow(); + const ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions& arrow() const; + PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* release_arrow(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* mutable_arrow(); + void set_allocated_arrow(::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow); + private: + const ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions& _internal_arrow() const; + ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* _internal_mutable_arrow(); + public: + void unsafe_arena_set_allocated_arrow( + ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow); + ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* unsafe_arena_release_arrow(); + + // .substrait.ReadRel.LocalFiles.FileOrFiles.OrcReadOptions orc = 11; + bool has_orc() const; + private: + bool _internal_has_orc() const; + public: + void clear_orc(); + const ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions& orc() const; + PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* release_orc(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* mutable_orc(); + void set_allocated_orc(::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc); + private: + const ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions& _internal_orc() const; + ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* _internal_mutable_orc(); + public: + void unsafe_arena_set_allocated_orc( + ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc); + ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* unsafe_arena_release_orc(); + + // .google.protobuf.Any extension = 12; + bool has_extension() const; + private: + bool _internal_has_extension() const; + public: + void clear_extension(); + const ::PROTOBUF_NAMESPACE_ID::Any& extension() const; + PROTOBUF_NODISCARD ::PROTOBUF_NAMESPACE_ID::Any* release_extension(); + ::PROTOBUF_NAMESPACE_ID::Any* mutable_extension(); + void set_allocated_extension(::PROTOBUF_NAMESPACE_ID::Any* extension); + private: + const ::PROTOBUF_NAMESPACE_ID::Any& _internal_extension() const; + ::PROTOBUF_NAMESPACE_ID::Any* _internal_mutable_extension(); + public: + void unsafe_arena_set_allocated_extension( + ::PROTOBUF_NAMESPACE_ID::Any* extension); + ::PROTOBUF_NAMESPACE_ID::Any* unsafe_arena_release_extension(); + + // .substrait.ReadRel.LocalFiles.FileOrFiles.DwrfReadOptions dwrf = 13; + bool has_dwrf() const; + private: + bool _internal_has_dwrf() const; + public: + void clear_dwrf(); + const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& dwrf() const; + PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* release_dwrf(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* mutable_dwrf(); + void set_allocated_dwrf(::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf); + private: + const ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions& _internal_dwrf() const; + ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* _internal_mutable_dwrf(); + public: + void unsafe_arena_set_allocated_dwrf( + ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* unsafe_arena_release_dwrf(); + + // .substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions text = 14; + bool has_text() const; + private: + bool _internal_has_text() const; + public: + void clear_text(); + const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& text() const; + PROTOBUF_NODISCARD ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* release_text(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* mutable_text(); + void set_allocated_text(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* text); + private: + const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& _internal_text() const; + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* _internal_mutable_text(); + public: + void unsafe_arena_set_allocated_text( + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* text); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* unsafe_arena_release_text(); + + void clear_path_type(); + PathTypeCase path_type_case() const; + void clear_file_format(); + FileFormatCase file_format_case() const; + // @@protoc_insertion_point(class_scope:substrait.ReadRel.LocalFiles.FileOrFiles) + private: + class _Internal; + void set_has_uri_path(); + void set_has_uri_path_glob(); + void set_has_uri_file(); + void set_has_uri_folder(); + void set_has_parquet(); + void set_has_arrow(); + void set_has_orc(); + void set_has_extension(); + void set_has_dwrf(); + void set_has_text(); + + inline bool has_path_type() const; + inline void clear_has_path_type(); + + inline bool has_file_format() const; + inline void clear_has_file_format(); + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + uint64_t partition_index_; + uint64_t start_; + uint64_t length_; + union PathTypeUnion { + constexpr PathTypeUnion() : _constinit_{} {} + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized _constinit_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_path_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_path_glob_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_file_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr uri_folder_; + } path_type_; + union FileFormatUnion { + constexpr FileFormatUnion() : _constinit_{} {} + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized _constinit_; + ::substrait::ReadRel_LocalFiles_FileOrFiles_ParquetReadOptions* parquet_; + ::substrait::ReadRel_LocalFiles_FileOrFiles_ArrowReadOptions* arrow_; + ::substrait::ReadRel_LocalFiles_FileOrFiles_OrcReadOptions* orc_; + ::PROTOBUF_NAMESPACE_ID::Any* extension_; + ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* dwrf_; + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* text_; + } file_format_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + uint32_t _oneof_case_[2]; + + friend struct ::TableStruct_substrait_2falgebra_2eproto; +}; +// ------------------------------------------------------------------- + +class ReadRel_LocalFiles final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ReadRel.LocalFiles) */ { + public: + inline ReadRel_LocalFiles() : ReadRel_LocalFiles(nullptr) {} + ~ReadRel_LocalFiles() override; + explicit constexpr ReadRel_LocalFiles(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + ReadRel_LocalFiles(const ReadRel_LocalFiles& from); + ReadRel_LocalFiles(ReadRel_LocalFiles&& from) noexcept + : ReadRel_LocalFiles() { + *this = ::std::move(from); + } + + inline ReadRel_LocalFiles& operator=(const ReadRel_LocalFiles& from) { + CopyFrom(from); + return *this; + } + inline ReadRel_LocalFiles& operator=(ReadRel_LocalFiles&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -3537,7 +3860,7 @@ class ReadRel_LocalFiles final : &_ReadRel_LocalFiles_default_instance_); } static constexpr int kIndexInFileMessages = - 14; + 15; friend void swap(ReadRel_LocalFiles& a, ReadRel_LocalFiles& b) { a.Swap(&b); @@ -3722,7 +4045,7 @@ class ReadRel final : &_ReadRel_default_instance_); } static constexpr int kIndexInFileMessages = - 15; + 16; friend void swap(ReadRel& a, ReadRel& b) { a.Swap(&b); @@ -4077,7 +4400,7 @@ class ProjectRel final : &_ProjectRel_default_instance_); } static constexpr int kIndexInFileMessages = - 16; + 17; friend void swap(ProjectRel& a, ProjectRel& b) { a.Swap(&b); @@ -4292,7 +4615,7 @@ class JoinRel final : &_JoinRel_default_instance_); } static constexpr int kIndexInFileMessages = - 17; + 18; friend void swap(JoinRel& a, JoinRel& b) { a.Swap(&b); @@ -4374,14 +4697,22 @@ class JoinRel final : JoinRel_JoinType_JOIN_TYPE_LEFT; static constexpr JoinType JOIN_TYPE_RIGHT = JoinRel_JoinType_JOIN_TYPE_RIGHT; - static constexpr JoinType JOIN_TYPE_SEMI = - JoinRel_JoinType_JOIN_TYPE_SEMI; - static constexpr JoinType JOIN_TYPE_ANTI = - JoinRel_JoinType_JOIN_TYPE_ANTI; - static constexpr JoinType JOIN_TYPE_SINGLE = - JoinRel_JoinType_JOIN_TYPE_SINGLE; - static constexpr JoinType JOIN_TYPE_MARK = - JoinRel_JoinType_JOIN_TYPE_MARK; + static constexpr JoinType JOIN_TYPE_LEFT_SEMI = + JoinRel_JoinType_JOIN_TYPE_LEFT_SEMI; + static constexpr JoinType JOIN_TYPE_LEFT_ANTI = + JoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; + static constexpr JoinType JOIN_TYPE_LEFT_SINGLE = + JoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE; + static constexpr JoinType JOIN_TYPE_RIGHT_SEMI = + JoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI; + static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = + JoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; + static constexpr JoinType JOIN_TYPE_RIGHT_SINGLE = + JoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE; + static constexpr JoinType JOIN_TYPE_LEFT_MARK = + JoinRel_JoinType_JOIN_TYPE_LEFT_MARK; + static constexpr JoinType JOIN_TYPE_RIGHT_MARK = + JoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; static inline bool JoinType_IsValid(int value) { return JoinRel_JoinType_IsValid(value); } @@ -4602,7 +4933,7 @@ class CrossRel final : &_CrossRel_default_instance_); } static constexpr int kIndexInFileMessages = - 18; + 19; friend void swap(CrossRel& a, CrossRel& b) { a.Swap(&b); @@ -4817,7 +5148,7 @@ class FetchRel final : &_FetchRel_default_instance_); } static constexpr int kIndexInFileMessages = - 19; + 20; friend void swap(FetchRel& a, FetchRel& b) { a.Swap(&b); @@ -5034,7 +5365,7 @@ class AggregateRel_Grouping final : &_AggregateRel_Grouping_default_instance_); } static constexpr int kIndexInFileMessages = - 20; + 21; friend void swap(AggregateRel_Grouping& a, AggregateRel_Grouping& b) { a.Swap(&b); @@ -5189,7 +5520,7 @@ class AggregateRel_Measure final : &_AggregateRel_Measure_default_instance_); } static constexpr int kIndexInFileMessages = - 21; + 22; friend void swap(AggregateRel_Measure& a, AggregateRel_Measure& b) { a.Swap(&b); @@ -5364,7 +5695,7 @@ class AggregateRel final : &_AggregateRel_default_instance_); } static constexpr int kIndexInFileMessages = - 22; + 23; friend void swap(AggregateRel& a, AggregateRel& b) { a.Swap(&b); @@ -5602,7 +5933,7 @@ class ConsistentPartitionWindowRel_WindowRelFunction final : &_ConsistentPartitionWindowRel_WindowRelFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 23; + 24; friend void swap(ConsistentPartitionWindowRel_WindowRelFunction& a, ConsistentPartitionWindowRel_WindowRelFunction& b) { a.Swap(&b); @@ -5881,7 +6212,7 @@ class ConsistentPartitionWindowRel final : &_ConsistentPartitionWindowRel_default_instance_); } static constexpr int kIndexInFileMessages = - 24; + 25; friend void swap(ConsistentPartitionWindowRel& a, ConsistentPartitionWindowRel& b) { a.Swap(&b); @@ -6138,7 +6469,7 @@ class SortRel final : &_SortRel_default_instance_); } static constexpr int kIndexInFileMessages = - 25; + 26; friend void swap(SortRel& a, SortRel& b) { a.Swap(&b); @@ -6353,7 +6684,7 @@ class FilterRel final : &_FilterRel_default_instance_); } static constexpr int kIndexInFileMessages = - 26; + 27; friend void swap(FilterRel& a, FilterRel& b) { a.Swap(&b); @@ -6568,7 +6899,7 @@ class SetRel final : &_SetRel_default_instance_); } static constexpr int kIndexInFileMessages = - 27; + 28; friend void swap(SetRel& a, SetRel& b) { a.Swap(&b); @@ -6814,7 +7145,7 @@ class ExtensionSingleRel final : &_ExtensionSingleRel_default_instance_); } static constexpr int kIndexInFileMessages = - 28; + 29; friend void swap(ExtensionSingleRel& a, ExtensionSingleRel& b) { a.Swap(&b); @@ -7009,7 +7340,7 @@ class ExtensionLeafRel final : &_ExtensionLeafRel_default_instance_); } static constexpr int kIndexInFileMessages = - 29; + 30; friend void swap(ExtensionLeafRel& a, ExtensionLeafRel& b) { a.Swap(&b); @@ -7184,7 +7515,7 @@ class ExtensionMultiRel final : &_ExtensionMultiRel_default_instance_); } static constexpr int kIndexInFileMessages = - 30; + 31; friend void swap(ExtensionMultiRel& a, ExtensionMultiRel& b) { a.Swap(&b); @@ -7379,7 +7710,7 @@ class ExchangeRel_ScatterFields final : &_ExchangeRel_ScatterFields_default_instance_); } static constexpr int kIndexInFileMessages = - 31; + 32; friend void swap(ExchangeRel_ScatterFields& a, ExchangeRel_ScatterFields& b) { a.Swap(&b); @@ -7534,7 +7865,7 @@ class ExchangeRel_SingleBucketExpression final : &_ExchangeRel_SingleBucketExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 32; + 33; friend void swap(ExchangeRel_SingleBucketExpression& a, ExchangeRel_SingleBucketExpression& b) { a.Swap(&b); @@ -7689,7 +8020,7 @@ class ExchangeRel_MultiBucketExpression final : &_ExchangeRel_MultiBucketExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 33; + 34; friend void swap(ExchangeRel_MultiBucketExpression& a, ExchangeRel_MultiBucketExpression& b) { a.Swap(&b); @@ -7854,7 +8185,7 @@ class ExchangeRel_Broadcast final : &_ExchangeRel_Broadcast_default_instance_); } static constexpr int kIndexInFileMessages = - 34; + 35; friend void swap(ExchangeRel_Broadcast& a, ExchangeRel_Broadcast& b) { a.Swap(&b); @@ -7973,7 +8304,7 @@ class ExchangeRel_RoundRobin final : &_ExchangeRel_RoundRobin_default_instance_); } static constexpr int kIndexInFileMessages = - 35; + 36; friend void swap(ExchangeRel_RoundRobin& a, ExchangeRel_RoundRobin& b) { a.Swap(&b); @@ -8125,7 +8456,7 @@ class ExchangeRel_ExchangeTarget final : &_ExchangeRel_ExchangeTarget_default_instance_); } static constexpr int kIndexInFileMessages = - 36; + 37; friend void swap(ExchangeRel_ExchangeTarget& a, ExchangeRel_ExchangeTarget& b) { a.Swap(&b); @@ -8347,7 +8678,7 @@ class ExchangeRel final : &_ExchangeRel_default_instance_); } static constexpr int kIndexInFileMessages = - 37; + 38; friend void swap(ExchangeRel& a, ExchangeRel& b) { a.Swap(&b); @@ -8702,7 +9033,7 @@ class ExpandRel_ExpandField final : &_ExpandRel_ExpandField_default_instance_); } static constexpr int kIndexInFileMessages = - 38; + 39; friend void swap(ExpandRel_ExpandField& a, ExpandRel_ExpandField& b) { a.Swap(&b); @@ -8890,7 +9221,7 @@ class ExpandRel_SwitchingField final : &_ExpandRel_SwitchingField_default_instance_); } static constexpr int kIndexInFileMessages = - 39; + 40; friend void swap(ExpandRel_SwitchingField& a, ExpandRel_SwitchingField& b) { a.Swap(&b); @@ -9045,7 +9376,7 @@ class ExpandRel final : &_ExpandRel_default_instance_); } static constexpr int kIndexInFileMessages = - 40; + 41; friend void swap(ExpandRel& a, ExpandRel& b) { a.Swap(&b); @@ -9243,7 +9574,7 @@ class RelRoot final : &_RelRoot_default_instance_); } static constexpr int kIndexInFileMessages = - 41; + 42; friend void swap(RelRoot& a, RelRoot& b) { a.Swap(&b); @@ -9438,8 +9769,8 @@ class Rel final : kHashJoin = 13, kMergeJoin = 14, kNestedLoopJoin = 18, - kMarkJoin = 23, - kDelimiterJoin = 24, + kDuplicateEliminatedGet = 23, + kDuplicateEliminatedJoin = 24, kWindow = 17, kExchange = 15, kExpand = 16, @@ -9451,7 +9782,7 @@ class Rel final : &_Rel_default_instance_); } static constexpr int kIndexInFileMessages = - 42; + 43; friend void swap(Rel& a, Rel& b) { a.Swap(&b); @@ -9543,8 +9874,8 @@ class Rel final : kHashJoinFieldNumber = 13, kMergeJoinFieldNumber = 14, kNestedLoopJoinFieldNumber = 18, - kMarkJoinFieldNumber = 23, - kDelimiterJoinFieldNumber = 24, + kDuplicateEliminatedGetFieldNumber = 23, + kDuplicateEliminatedJoinFieldNumber = 24, kWindowFieldNumber = 17, kExchangeFieldNumber = 15, kExpandFieldNumber = 16, @@ -9873,41 +10204,41 @@ class Rel final : ::substrait::NestedLoopJoinRel* nested_loop_join); ::substrait::NestedLoopJoinRel* unsafe_arena_release_nested_loop_join(); - // .substrait.MarkJoinRel mark_join = 23; - bool has_mark_join() const; + // .substrait.DuplicateEliminatedGetRel duplicate_eliminated_get = 23; + bool has_duplicate_eliminated_get() const; private: - bool _internal_has_mark_join() const; + bool _internal_has_duplicate_eliminated_get() const; public: - void clear_mark_join(); - const ::substrait::MarkJoinRel& mark_join() const; - PROTOBUF_NODISCARD ::substrait::MarkJoinRel* release_mark_join(); - ::substrait::MarkJoinRel* mutable_mark_join(); - void set_allocated_mark_join(::substrait::MarkJoinRel* mark_join); + void clear_duplicate_eliminated_get(); + const ::substrait::DuplicateEliminatedGetRel& duplicate_eliminated_get() const; + PROTOBUF_NODISCARD ::substrait::DuplicateEliminatedGetRel* release_duplicate_eliminated_get(); + ::substrait::DuplicateEliminatedGetRel* mutable_duplicate_eliminated_get(); + void set_allocated_duplicate_eliminated_get(::substrait::DuplicateEliminatedGetRel* duplicate_eliminated_get); private: - const ::substrait::MarkJoinRel& _internal_mark_join() const; - ::substrait::MarkJoinRel* _internal_mutable_mark_join(); + const ::substrait::DuplicateEliminatedGetRel& _internal_duplicate_eliminated_get() const; + ::substrait::DuplicateEliminatedGetRel* _internal_mutable_duplicate_eliminated_get(); public: - void unsafe_arena_set_allocated_mark_join( - ::substrait::MarkJoinRel* mark_join); - ::substrait::MarkJoinRel* unsafe_arena_release_mark_join(); + void unsafe_arena_set_allocated_duplicate_eliminated_get( + ::substrait::DuplicateEliminatedGetRel* duplicate_eliminated_get); + ::substrait::DuplicateEliminatedGetRel* unsafe_arena_release_duplicate_eliminated_get(); - // .substrait.DelimiterJoinRel delimiter_join = 24; - bool has_delimiter_join() const; + // .substrait.DuplicateEliminatedJoinRel duplicate_eliminated_join = 24; + bool has_duplicate_eliminated_join() const; private: - bool _internal_has_delimiter_join() const; + bool _internal_has_duplicate_eliminated_join() const; public: - void clear_delimiter_join(); - const ::substrait::DelimiterJoinRel& delimiter_join() const; - PROTOBUF_NODISCARD ::substrait::DelimiterJoinRel* release_delimiter_join(); - ::substrait::DelimiterJoinRel* mutable_delimiter_join(); - void set_allocated_delimiter_join(::substrait::DelimiterJoinRel* delimiter_join); + void clear_duplicate_eliminated_join(); + const ::substrait::DuplicateEliminatedJoinRel& duplicate_eliminated_join() const; + PROTOBUF_NODISCARD ::substrait::DuplicateEliminatedJoinRel* release_duplicate_eliminated_join(); + ::substrait::DuplicateEliminatedJoinRel* mutable_duplicate_eliminated_join(); + void set_allocated_duplicate_eliminated_join(::substrait::DuplicateEliminatedJoinRel* duplicate_eliminated_join); private: - const ::substrait::DelimiterJoinRel& _internal_delimiter_join() const; - ::substrait::DelimiterJoinRel* _internal_mutable_delimiter_join(); + const ::substrait::DuplicateEliminatedJoinRel& _internal_duplicate_eliminated_join() const; + ::substrait::DuplicateEliminatedJoinRel* _internal_mutable_duplicate_eliminated_join(); public: - void unsafe_arena_set_allocated_delimiter_join( - ::substrait::DelimiterJoinRel* delimiter_join); - ::substrait::DelimiterJoinRel* unsafe_arena_release_delimiter_join(); + void unsafe_arena_set_allocated_duplicate_eliminated_join( + ::substrait::DuplicateEliminatedJoinRel* duplicate_eliminated_join); + ::substrait::DuplicateEliminatedJoinRel* unsafe_arena_release_duplicate_eliminated_join(); // .substrait.ConsistentPartitionWindowRel window = 17; bool has_window() const; @@ -9986,8 +10317,8 @@ class Rel final : void set_has_hash_join(); void set_has_merge_join(); void set_has_nested_loop_join(); - void set_has_mark_join(); - void set_has_delimiter_join(); + void set_has_duplicate_eliminated_get(); + void set_has_duplicate_eliminated_join(); void set_has_window(); void set_has_exchange(); void set_has_expand(); @@ -10019,8 +10350,8 @@ class Rel final : ::substrait::HashJoinRel* hash_join_; ::substrait::MergeJoinRel* merge_join_; ::substrait::NestedLoopJoinRel* nested_loop_join_; - ::substrait::MarkJoinRel* mark_join_; - ::substrait::DelimiterJoinRel* delimiter_join_; + ::substrait::DuplicateEliminatedGetRel* duplicate_eliminated_get_; + ::substrait::DuplicateEliminatedJoinRel* duplicate_eliminated_join_; ::substrait::ConsistentPartitionWindowRel* window_; ::substrait::ExchangeRel* exchange_; ::substrait::ExpandRel* expand_; @@ -10080,7 +10411,7 @@ class NamedObjectWrite final : &_NamedObjectWrite_default_instance_); } static constexpr int kIndexInFileMessages = - 43; + 44; friend void swap(NamedObjectWrite& a, NamedObjectWrite& b) { a.Swap(&b); @@ -10261,7 +10592,7 @@ class ExtensionObject final : &_ExtensionObject_default_instance_); } static constexpr int kIndexInFileMessages = - 44; + 45; friend void swap(ExtensionObject& a, ExtensionObject& b) { a.Swap(&b); @@ -10422,7 +10753,7 @@ class DdlRel final : &_DdlRel_default_instance_); } static constexpr int kIndexInFileMessages = - 45; + 46; friend void swap(DdlRel& a, DdlRel& b) { a.Swap(&b); @@ -10788,7 +11119,7 @@ class WriteRel final : &_WriteRel_default_instance_); } static constexpr int kIndexInFileMessages = - 46; + 47; friend void swap(WriteRel& a, WriteRel& b) { a.Swap(&b); @@ -11132,7 +11463,7 @@ class ComparisonJoinKey_ComparisonType final : &_ComparisonJoinKey_ComparisonType_default_instance_); } static constexpr int kIndexInFileMessages = - 47; + 48; friend void swap(ComparisonJoinKey_ComparisonType& a, ComparisonJoinKey_ComparisonType& b) { a.Swap(&b); @@ -11310,7 +11641,7 @@ class ComparisonJoinKey final : &_ComparisonJoinKey_default_instance_); } static constexpr int kIndexInFileMessages = - 48; + 49; friend void swap(ComparisonJoinKey& a, ComparisonJoinKey& b) { a.Swap(&b); @@ -11541,7 +11872,7 @@ class HashJoinRel final : &_HashJoinRel_default_instance_); } static constexpr int kIndexInFileMessages = - 49; + 50; friend void swap(HashJoinRel& a, HashJoinRel& b) { a.Swap(&b); @@ -11631,6 +11962,14 @@ class HashJoinRel final : HashJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = HashJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; + static constexpr JoinType JOIN_TYPE_LEFT_SINGLE = + HashJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE; + static constexpr JoinType JOIN_TYPE_RIGHT_SINGLE = + HashJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE; + static constexpr JoinType JOIN_TYPE_LEFT_MARK = + HashJoinRel_JoinType_JOIN_TYPE_LEFT_MARK; + static constexpr JoinType JOIN_TYPE_RIGHT_MARK = + HashJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; static inline bool JoinType_IsValid(int value) { return HashJoinRel_JoinType_IsValid(value); } @@ -11891,7 +12230,7 @@ class MergeJoinRel final : &_MergeJoinRel_default_instance_); } static constexpr int kIndexInFileMessages = - 50; + 51; friend void swap(MergeJoinRel& a, MergeJoinRel& b) { a.Swap(&b); @@ -11981,6 +12320,14 @@ class MergeJoinRel final : MergeJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; + static constexpr JoinType JOIN_TYPE_LEFT_SINGLE = + MergeJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE; + static constexpr JoinType JOIN_TYPE_RIGHT_SINGLE = + MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE; + static constexpr JoinType JOIN_TYPE_LEFT_MARK = + MergeJoinRel_JoinType_JOIN_TYPE_LEFT_MARK; + static constexpr JoinType JOIN_TYPE_RIGHT_MARK = + MergeJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; static inline bool JoinType_IsValid(int value) { return MergeJoinRel_JoinType_IsValid(value); } @@ -12241,7 +12588,7 @@ class NestedLoopJoinRel final : &_NestedLoopJoinRel_default_instance_); } static constexpr int kIndexInFileMessages = - 51; + 52; friend void swap(NestedLoopJoinRel& a, NestedLoopJoinRel& b) { a.Swap(&b); @@ -12331,6 +12678,14 @@ class NestedLoopJoinRel final : NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; + static constexpr JoinType JOIN_TYPE_LEFT_SINGLE = + NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE; + static constexpr JoinType JOIN_TYPE_RIGHT_SINGLE = + NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE; + static constexpr JoinType JOIN_TYPE_LEFT_MARK = + NestedLoopJoinRel_JoinType_JOIN_TYPE_LEFT_MARK; + static constexpr JoinType JOIN_TYPE_RIGHT_MARK = + NestedLoopJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; static inline bool JoinType_IsValid(int value) { return NestedLoopJoinRel_JoinType_IsValid(value); } @@ -12483,24 +12838,24 @@ class NestedLoopJoinRel final : }; // ------------------------------------------------------------------- -class MarkJoinRel final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.MarkJoinRel) */ { +class DuplicateEliminatedGetRel final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DuplicateEliminatedGetRel) */ { public: - inline MarkJoinRel() : MarkJoinRel(nullptr) {} - ~MarkJoinRel() override; - explicit constexpr MarkJoinRel(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DuplicateEliminatedGetRel() : DuplicateEliminatedGetRel(nullptr) {} + ~DuplicateEliminatedGetRel() override; + explicit constexpr DuplicateEliminatedGetRel(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - MarkJoinRel(const MarkJoinRel& from); - MarkJoinRel(MarkJoinRel&& from) noexcept - : MarkJoinRel() { + DuplicateEliminatedGetRel(const DuplicateEliminatedGetRel& from); + DuplicateEliminatedGetRel(DuplicateEliminatedGetRel&& from) noexcept + : DuplicateEliminatedGetRel() { *this = ::std::move(from); } - inline MarkJoinRel& operator=(const MarkJoinRel& from) { + inline DuplicateEliminatedGetRel& operator=(const DuplicateEliminatedGetRel& from) { CopyFrom(from); return *this; } - inline MarkJoinRel& operator=(MarkJoinRel&& from) noexcept { + inline DuplicateEliminatedGetRel& operator=(DuplicateEliminatedGetRel&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -12523,20 +12878,20 @@ class MarkJoinRel final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const MarkJoinRel& default_instance() { + static const DuplicateEliminatedGetRel& default_instance() { return *internal_default_instance(); } - static inline const MarkJoinRel* internal_default_instance() { - return reinterpret_cast( - &_MarkJoinRel_default_instance_); + static inline const DuplicateEliminatedGetRel* internal_default_instance() { + return reinterpret_cast( + &_DuplicateEliminatedGetRel_default_instance_); } static constexpr int kIndexInFileMessages = - 52; + 53; - friend void swap(MarkJoinRel& a, MarkJoinRel& b) { + friend void swap(DuplicateEliminatedGetRel& a, DuplicateEliminatedGetRel& b) { a.Swap(&b); } - inline void Swap(MarkJoinRel* other) { + inline void Swap(DuplicateEliminatedGetRel* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -12549,7 +12904,7 @@ class MarkJoinRel final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(MarkJoinRel* other) { + void UnsafeArenaSwap(DuplicateEliminatedGetRel* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -12557,13 +12912,13 @@ class MarkJoinRel final : // implements Message ---------------------------------------------- - MarkJoinRel* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DuplicateEliminatedGetRel* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const MarkJoinRel& from); + void CopyFrom(const DuplicateEliminatedGetRel& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const MarkJoinRel& from); + void MergeFrom(const DuplicateEliminatedGetRel& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -12580,15 +12935,15 @@ class MarkJoinRel final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(MarkJoinRel* other); + void InternalSwap(DuplicateEliminatedGetRel* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.MarkJoinRel"; + return "substrait.DuplicateEliminatedGetRel"; } protected: - explicit MarkJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DuplicateEliminatedGetRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -12605,12 +12960,293 @@ class MarkJoinRel final : // accessors ------------------------------------------------------- enum : int { + kCommonFieldNumber = 1, + kInputFieldNumber = 2, + }; + // .substrait.RelCommon common = 1; + bool has_common() const; + private: + bool _internal_has_common() const; + public: + void clear_common(); + const ::substrait::RelCommon& common() const; + PROTOBUF_NODISCARD ::substrait::RelCommon* release_common(); + ::substrait::RelCommon* mutable_common(); + void set_allocated_common(::substrait::RelCommon* common); + private: + const ::substrait::RelCommon& _internal_common() const; + ::substrait::RelCommon* _internal_mutable_common(); + public: + void unsafe_arena_set_allocated_common( + ::substrait::RelCommon* common); + ::substrait::RelCommon* unsafe_arena_release_common(); + + // .substrait.ReferenceRel input = 2; + bool has_input() const; + private: + bool _internal_has_input() const; + public: + void clear_input(); + const ::substrait::ReferenceRel& input() const; + PROTOBUF_NODISCARD ::substrait::ReferenceRel* release_input(); + ::substrait::ReferenceRel* mutable_input(); + void set_allocated_input(::substrait::ReferenceRel* input); + private: + const ::substrait::ReferenceRel& _internal_input() const; + ::substrait::ReferenceRel* _internal_mutable_input(); + public: + void unsafe_arena_set_allocated_input( + ::substrait::ReferenceRel* input); + ::substrait::ReferenceRel* unsafe_arena_release_input(); + + // @@protoc_insertion_point(class_scope:substrait.DuplicateEliminatedGetRel) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::substrait::RelCommon* common_; + ::substrait::ReferenceRel* input_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2falgebra_2eproto; +}; +// ------------------------------------------------------------------- + +class DuplicateEliminatedJoinRel final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DuplicateEliminatedJoinRel) */ { + public: + inline DuplicateEliminatedJoinRel() : DuplicateEliminatedJoinRel(nullptr) {} + ~DuplicateEliminatedJoinRel() override; + explicit constexpr DuplicateEliminatedJoinRel(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + DuplicateEliminatedJoinRel(const DuplicateEliminatedJoinRel& from); + DuplicateEliminatedJoinRel(DuplicateEliminatedJoinRel&& from) noexcept + : DuplicateEliminatedJoinRel() { + *this = ::std::move(from); + } + + inline DuplicateEliminatedJoinRel& operator=(const DuplicateEliminatedJoinRel& from) { + CopyFrom(from); + return *this; + } + inline DuplicateEliminatedJoinRel& operator=(DuplicateEliminatedJoinRel&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const DuplicateEliminatedJoinRel& default_instance() { + return *internal_default_instance(); + } + static inline const DuplicateEliminatedJoinRel* internal_default_instance() { + return reinterpret_cast( + &_DuplicateEliminatedJoinRel_default_instance_); + } + static constexpr int kIndexInFileMessages = + 54; + + friend void swap(DuplicateEliminatedJoinRel& a, DuplicateEliminatedJoinRel& b) { + a.Swap(&b); + } + inline void Swap(DuplicateEliminatedJoinRel* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(DuplicateEliminatedJoinRel* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + DuplicateEliminatedJoinRel* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const DuplicateEliminatedJoinRel& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const DuplicateEliminatedJoinRel& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(DuplicateEliminatedJoinRel* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.DuplicateEliminatedJoinRel"; + } + protected: + explicit DuplicateEliminatedJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + typedef DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedSide; + static constexpr DuplicateEliminatedSide DUPLICATE_ELIMINATED_SIDE_UNSPECIFIED = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_UNSPECIFIED; + static constexpr DuplicateEliminatedSide DUPLICATE_ELIMINATED_SIDE_LEFT = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_LEFT; + static constexpr DuplicateEliminatedSide DUPLICATE_ELIMINATED_SIDE_RIGHT = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DUPLICATE_ELIMINATED_SIDE_RIGHT; + static inline bool DuplicateEliminatedSide_IsValid(int value) { + return DuplicateEliminatedJoinRel_DuplicateEliminatedSide_IsValid(value); + } + static constexpr DuplicateEliminatedSide DuplicateEliminatedSide_MIN = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_MIN; + static constexpr DuplicateEliminatedSide DuplicateEliminatedSide_MAX = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_MAX; + static constexpr int DuplicateEliminatedSide_ARRAYSIZE = + DuplicateEliminatedJoinRel_DuplicateEliminatedSide_DuplicateEliminatedSide_ARRAYSIZE; + static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* + DuplicateEliminatedSide_descriptor() { + return DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor(); + } + template + static inline const std::string& DuplicateEliminatedSide_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function DuplicateEliminatedSide_Name."); + return DuplicateEliminatedJoinRel_DuplicateEliminatedSide_Name(enum_t_value); + } + static inline bool DuplicateEliminatedSide_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, + DuplicateEliminatedSide* value) { + return DuplicateEliminatedJoinRel_DuplicateEliminatedSide_Parse(name, value); + } + + typedef DuplicateEliminatedJoinRel_JoinType JoinType; + static constexpr JoinType JOIN_TYPE_UNSPECIFIED = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; + static constexpr JoinType JOIN_TYPE_INNER = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_INNER; + static constexpr JoinType JOIN_TYPE_OUTER = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_OUTER; + static constexpr JoinType JOIN_TYPE_LEFT = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT; + static constexpr JoinType JOIN_TYPE_RIGHT = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT; + static constexpr JoinType JOIN_TYPE_LEFT_SEMI = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SEMI; + static constexpr JoinType JOIN_TYPE_LEFT_ANTI = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; + static constexpr JoinType JOIN_TYPE_LEFT_SINGLE = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_SINGLE; + static constexpr JoinType JOIN_TYPE_RIGHT_SEMI = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI; + static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; + static constexpr JoinType JOIN_TYPE_RIGHT_SINGLE = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_SINGLE; + static constexpr JoinType JOIN_TYPE_LEFT_MARK = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_LEFT_MARK; + static constexpr JoinType JOIN_TYPE_RIGHT_MARK = + DuplicateEliminatedJoinRel_JoinType_JOIN_TYPE_RIGHT_MARK; + static inline bool JoinType_IsValid(int value) { + return DuplicateEliminatedJoinRel_JoinType_IsValid(value); + } + static constexpr JoinType JoinType_MIN = + DuplicateEliminatedJoinRel_JoinType_JoinType_MIN; + static constexpr JoinType JoinType_MAX = + DuplicateEliminatedJoinRel_JoinType_JoinType_MAX; + static constexpr int JoinType_ARRAYSIZE = + DuplicateEliminatedJoinRel_JoinType_JoinType_ARRAYSIZE; + static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* + JoinType_descriptor() { + return DuplicateEliminatedJoinRel_JoinType_descriptor(); + } + template + static inline const std::string& JoinType_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function JoinType_Name."); + return DuplicateEliminatedJoinRel_JoinType_Name(enum_t_value); + } + static inline bool JoinType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, + JoinType* value) { + return DuplicateEliminatedJoinRel_JoinType_Parse(name, value); + } + + // accessors ------------------------------------------------------- + + enum : int { + kDuplicateEliminatedColumnsFieldNumber = 7, kCommonFieldNumber = 1, kLeftFieldNumber = 2, kRightFieldNumber = 3, kExpressionFieldNumber = 4, + kPostJoinFilterFieldNumber = 5, kAdvancedExtensionFieldNumber = 10, + kTypeFieldNumber = 6, + kDuplicateEliminatedSideFieldNumber = 8, }; + // repeated .substrait.Expression.FieldReference duplicate_eliminated_columns = 7; + int duplicate_eliminated_columns_size() const; + private: + int _internal_duplicate_eliminated_columns_size() const; + public: + void clear_duplicate_eliminated_columns(); + ::substrait::Expression_FieldReference* mutable_duplicate_eliminated_columns(int index); + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* + mutable_duplicate_eliminated_columns(); + private: + const ::substrait::Expression_FieldReference& _internal_duplicate_eliminated_columns(int index) const; + ::substrait::Expression_FieldReference* _internal_add_duplicate_eliminated_columns(); + public: + const ::substrait::Expression_FieldReference& duplicate_eliminated_columns(int index) const; + ::substrait::Expression_FieldReference* add_duplicate_eliminated_columns(); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& + duplicate_eliminated_columns() const; + // .substrait.RelCommon common = 1; bool has_common() const; private: @@ -12683,323 +13319,23 @@ class MarkJoinRel final : ::substrait::Expression* expression); ::substrait::Expression* unsafe_arena_release_expression(); - // .substrait.extensions.AdvancedExtension advanced_extension = 10; - bool has_advanced_extension() const; - private: - bool _internal_has_advanced_extension() const; - public: - void clear_advanced_extension(); - const ::substrait::extensions::AdvancedExtension& advanced_extension() const; - PROTOBUF_NODISCARD ::substrait::extensions::AdvancedExtension* release_advanced_extension(); - ::substrait::extensions::AdvancedExtension* mutable_advanced_extension(); - void set_allocated_advanced_extension(::substrait::extensions::AdvancedExtension* advanced_extension); - private: - const ::substrait::extensions::AdvancedExtension& _internal_advanced_extension() const; - ::substrait::extensions::AdvancedExtension* _internal_mutable_advanced_extension(); - public: - void unsafe_arena_set_allocated_advanced_extension( - ::substrait::extensions::AdvancedExtension* advanced_extension); - ::substrait::extensions::AdvancedExtension* unsafe_arena_release_advanced_extension(); - - // @@protoc_insertion_point(class_scope:substrait.MarkJoinRel) - private: - class _Internal; - - template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; - typedef void InternalArenaConstructable_; - typedef void DestructorSkippable_; - ::substrait::RelCommon* common_; - ::substrait::Rel* left_; - ::substrait::Rel* right_; - ::substrait::Expression* expression_; - ::substrait::extensions::AdvancedExtension* advanced_extension_; - mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; - friend struct ::TableStruct_substrait_2falgebra_2eproto; -}; -// ------------------------------------------------------------------- - -class DelimiterJoinRel final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DelimiterJoinRel) */ { - public: - inline DelimiterJoinRel() : DelimiterJoinRel(nullptr) {} - ~DelimiterJoinRel() override; - explicit constexpr DelimiterJoinRel(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - - DelimiterJoinRel(const DelimiterJoinRel& from); - DelimiterJoinRel(DelimiterJoinRel&& from) noexcept - : DelimiterJoinRel() { - *this = ::std::move(from); - } - - inline DelimiterJoinRel& operator=(const DelimiterJoinRel& from) { - CopyFrom(from); - return *this; - } - inline DelimiterJoinRel& operator=(DelimiterJoinRel&& from) noexcept { - if (this == &from) return *this; - if (GetOwningArena() == from.GetOwningArena() - #ifdef PROTOBUF_FORCE_COPY_IN_MOVE - && GetOwningArena() != nullptr - #endif // !PROTOBUF_FORCE_COPY_IN_MOVE - ) { - InternalSwap(&from); - } else { - CopyFrom(from); - } - return *this; - } - - static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { - return GetDescriptor(); - } - static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { - return default_instance().GetMetadata().descriptor; - } - static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { - return default_instance().GetMetadata().reflection; - } - static const DelimiterJoinRel& default_instance() { - return *internal_default_instance(); - } - static inline const DelimiterJoinRel* internal_default_instance() { - return reinterpret_cast( - &_DelimiterJoinRel_default_instance_); - } - static constexpr int kIndexInFileMessages = - 53; - - friend void swap(DelimiterJoinRel& a, DelimiterJoinRel& b) { - a.Swap(&b); - } - inline void Swap(DelimiterJoinRel* other) { - if (other == this) return; - #ifdef PROTOBUF_FORCE_COPY_IN_SWAP - if (GetOwningArena() != nullptr && - GetOwningArena() == other->GetOwningArena()) { - #else // PROTOBUF_FORCE_COPY_IN_SWAP - if (GetOwningArena() == other->GetOwningArena()) { - #endif // !PROTOBUF_FORCE_COPY_IN_SWAP - InternalSwap(other); - } else { - ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); - } - } - void UnsafeArenaSwap(DelimiterJoinRel* other) { - if (other == this) return; - GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); - InternalSwap(other); - } - - // implements Message ---------------------------------------------- - - DelimiterJoinRel* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); - } - using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DelimiterJoinRel& from); - using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DelimiterJoinRel& from); - private: - static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); - public: - PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; - bool IsInitialized() const final; - - size_t ByteSizeLong() const final; - const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; - uint8_t* _InternalSerialize( - uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; - int GetCachedSize() const final { return _cached_size_.Get(); } - - private: - void SharedCtor(); - void SharedDtor(); - void SetCachedSize(int size) const final; - void InternalSwap(DelimiterJoinRel* other); - - private: - friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; - static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DelimiterJoinRel"; - } - protected: - explicit DelimiterJoinRel(::PROTOBUF_NAMESPACE_ID::Arena* arena, - bool is_message_owned = false); - private: - static void ArenaDtor(void* object); - inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); - public: - - static const ClassData _class_data_; - const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; - - ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; - - // nested types ---------------------------------------------------- - - typedef DelimiterJoinRel_JoinType JoinType; - static constexpr JoinType JOIN_TYPE_UNSPECIFIED = - DelimiterJoinRel_JoinType_JOIN_TYPE_UNSPECIFIED; - static constexpr JoinType JOIN_TYPE_INNER = - DelimiterJoinRel_JoinType_JOIN_TYPE_INNER; - static constexpr JoinType JOIN_TYPE_OUTER = - DelimiterJoinRel_JoinType_JOIN_TYPE_OUTER; - static constexpr JoinType JOIN_TYPE_LEFT = - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT; - static constexpr JoinType JOIN_TYPE_RIGHT = - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT; - static constexpr JoinType JOIN_TYPE_LEFT_SEMI = - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT_SEMI; - static constexpr JoinType JOIN_TYPE_RIGHT_SEMI = - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT_SEMI; - static constexpr JoinType JOIN_TYPE_LEFT_ANTI = - DelimiterJoinRel_JoinType_JOIN_TYPE_LEFT_ANTI; - static constexpr JoinType JOIN_TYPE_RIGHT_ANTI = - DelimiterJoinRel_JoinType_JOIN_TYPE_RIGHT_ANTI; - static inline bool JoinType_IsValid(int value) { - return DelimiterJoinRel_JoinType_IsValid(value); - } - static constexpr JoinType JoinType_MIN = - DelimiterJoinRel_JoinType_JoinType_MIN; - static constexpr JoinType JoinType_MAX = - DelimiterJoinRel_JoinType_JoinType_MAX; - static constexpr int JoinType_ARRAYSIZE = - DelimiterJoinRel_JoinType_JoinType_ARRAYSIZE; - static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* - JoinType_descriptor() { - return DelimiterJoinRel_JoinType_descriptor(); - } - template - static inline const std::string& JoinType_Name(T enum_t_value) { - static_assert(::std::is_same::value || - ::std::is_integral::value, - "Incorrect type passed to function JoinType_Name."); - return DelimiterJoinRel_JoinType_Name(enum_t_value); - } - static inline bool JoinType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, - JoinType* value) { - return DelimiterJoinRel_JoinType_Parse(name, value); - } - - // accessors ------------------------------------------------------- - - enum : int { - kLeftKeysFieldNumber = 4, - kRightKeysFieldNumber = 5, - kCommonFieldNumber = 1, - kLeftFieldNumber = 2, - kRightFieldNumber = 3, - kDelimiterFieldFieldNumber = 7, - kAdvancedExtensionFieldNumber = 10, - kTypeFieldNumber = 6, - }; - // repeated .substrait.Expression.FieldReference left_keys = 4; - int left_keys_size() const; - private: - int _internal_left_keys_size() const; - public: - void clear_left_keys(); - ::substrait::Expression_FieldReference* mutable_left_keys(int index); - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* - mutable_left_keys(); - private: - const ::substrait::Expression_FieldReference& _internal_left_keys(int index) const; - ::substrait::Expression_FieldReference* _internal_add_left_keys(); - public: - const ::substrait::Expression_FieldReference& left_keys(int index) const; - ::substrait::Expression_FieldReference* add_left_keys(); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& - left_keys() const; - - // repeated .substrait.Expression.FieldReference right_keys = 5; - int right_keys_size() const; - private: - int _internal_right_keys_size() const; - public: - void clear_right_keys(); - ::substrait::Expression_FieldReference* mutable_right_keys(int index); - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* - mutable_right_keys(); - private: - const ::substrait::Expression_FieldReference& _internal_right_keys(int index) const; - ::substrait::Expression_FieldReference* _internal_add_right_keys(); - public: - const ::substrait::Expression_FieldReference& right_keys(int index) const; - ::substrait::Expression_FieldReference* add_right_keys(); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& - right_keys() const; - - // .substrait.RelCommon common = 1; - bool has_common() const; - private: - bool _internal_has_common() const; - public: - void clear_common(); - const ::substrait::RelCommon& common() const; - PROTOBUF_NODISCARD ::substrait::RelCommon* release_common(); - ::substrait::RelCommon* mutable_common(); - void set_allocated_common(::substrait::RelCommon* common); - private: - const ::substrait::RelCommon& _internal_common() const; - ::substrait::RelCommon* _internal_mutable_common(); - public: - void unsafe_arena_set_allocated_common( - ::substrait::RelCommon* common); - ::substrait::RelCommon* unsafe_arena_release_common(); - - // .substrait.Rel left = 2; - bool has_left() const; - private: - bool _internal_has_left() const; - public: - void clear_left(); - const ::substrait::Rel& left() const; - PROTOBUF_NODISCARD ::substrait::Rel* release_left(); - ::substrait::Rel* mutable_left(); - void set_allocated_left(::substrait::Rel* left); - private: - const ::substrait::Rel& _internal_left() const; - ::substrait::Rel* _internal_mutable_left(); - public: - void unsafe_arena_set_allocated_left( - ::substrait::Rel* left); - ::substrait::Rel* unsafe_arena_release_left(); - - // .substrait.Rel right = 3; - bool has_right() const; - private: - bool _internal_has_right() const; - public: - void clear_right(); - const ::substrait::Rel& right() const; - PROTOBUF_NODISCARD ::substrait::Rel* release_right(); - ::substrait::Rel* mutable_right(); - void set_allocated_right(::substrait::Rel* right); - private: - const ::substrait::Rel& _internal_right() const; - ::substrait::Rel* _internal_mutable_right(); - public: - void unsafe_arena_set_allocated_right( - ::substrait::Rel* right); - ::substrait::Rel* unsafe_arena_release_right(); - - // .substrait.Expression.FieldReference delimiter_field = 7; - bool has_delimiter_field() const; + // .substrait.Expression post_join_filter = 5; + bool has_post_join_filter() const; private: - bool _internal_has_delimiter_field() const; + bool _internal_has_post_join_filter() const; public: - void clear_delimiter_field(); - const ::substrait::Expression_FieldReference& delimiter_field() const; - PROTOBUF_NODISCARD ::substrait::Expression_FieldReference* release_delimiter_field(); - ::substrait::Expression_FieldReference* mutable_delimiter_field(); - void set_allocated_delimiter_field(::substrait::Expression_FieldReference* delimiter_field); + void clear_post_join_filter(); + const ::substrait::Expression& post_join_filter() const; + PROTOBUF_NODISCARD ::substrait::Expression* release_post_join_filter(); + ::substrait::Expression* mutable_post_join_filter(); + void set_allocated_post_join_filter(::substrait::Expression* post_join_filter); private: - const ::substrait::Expression_FieldReference& _internal_delimiter_field() const; - ::substrait::Expression_FieldReference* _internal_mutable_delimiter_field(); + const ::substrait::Expression& _internal_post_join_filter() const; + ::substrait::Expression* _internal_mutable_post_join_filter(); public: - void unsafe_arena_set_allocated_delimiter_field( - ::substrait::Expression_FieldReference* delimiter_field); - ::substrait::Expression_FieldReference* unsafe_arena_release_delimiter_field(); + void unsafe_arena_set_allocated_post_join_filter( + ::substrait::Expression* post_join_filter); + ::substrait::Expression* unsafe_arena_release_post_join_filter(); // .substrait.extensions.AdvancedExtension advanced_extension = 10; bool has_advanced_extension() const; @@ -13019,30 +13355,40 @@ class DelimiterJoinRel final : ::substrait::extensions::AdvancedExtension* advanced_extension); ::substrait::extensions::AdvancedExtension* unsafe_arena_release_advanced_extension(); - // .substrait.DelimiterJoinRel.JoinType type = 6; + // .substrait.DuplicateEliminatedJoinRel.JoinType type = 6; void clear_type(); - ::substrait::DelimiterJoinRel_JoinType type() const; - void set_type(::substrait::DelimiterJoinRel_JoinType value); + ::substrait::DuplicateEliminatedJoinRel_JoinType type() const; + void set_type(::substrait::DuplicateEliminatedJoinRel_JoinType value); + private: + ::substrait::DuplicateEliminatedJoinRel_JoinType _internal_type() const; + void _internal_set_type(::substrait::DuplicateEliminatedJoinRel_JoinType value); + public: + + // .substrait.DuplicateEliminatedJoinRel.DuplicateEliminatedSide duplicate_eliminated_side = 8; + void clear_duplicate_eliminated_side(); + ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide duplicate_eliminated_side() const; + void set_duplicate_eliminated_side(::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide value); private: - ::substrait::DelimiterJoinRel_JoinType _internal_type() const; - void _internal_set_type(::substrait::DelimiterJoinRel_JoinType value); + ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide _internal_duplicate_eliminated_side() const; + void _internal_set_duplicate_eliminated_side(::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide value); public: - // @@protoc_insertion_point(class_scope:substrait.DelimiterJoinRel) + // @@protoc_insertion_point(class_scope:substrait.DuplicateEliminatedJoinRel) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference > left_keys_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference > right_keys_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference > duplicate_eliminated_columns_; ::substrait::RelCommon* common_; ::substrait::Rel* left_; ::substrait::Rel* right_; - ::substrait::Expression_FieldReference* delimiter_field_; + ::substrait::Expression* expression_; + ::substrait::Expression* post_join_filter_; ::substrait::extensions::AdvancedExtension* advanced_extension_; int type_; + int duplicate_eliminated_side_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; @@ -13103,7 +13449,7 @@ class FunctionArgument final : &_FunctionArgument_default_instance_); } static constexpr int kIndexInFileMessages = - 54; + 55; friend void swap(FunctionArgument& a, FunctionArgument& b) { a.Swap(&b); @@ -13312,7 +13658,7 @@ class FunctionOption final : &_FunctionOption_default_instance_); } static constexpr int kIndexInFileMessages = - 55; + 56; friend void swap(FunctionOption& a, FunctionOption& b) { a.Swap(&b); @@ -13488,7 +13834,7 @@ class Expression_Enum_Empty final : &_Expression_Enum_Empty_default_instance_); } static constexpr int kIndexInFileMessages = - 56; + 57; friend void swap(Expression_Enum_Empty& a, Expression_Enum_Empty& b) { a.Swap(&b); @@ -13613,7 +13959,7 @@ class Expression_Enum final : &_Expression_Enum_default_instance_); } static constexpr int kIndexInFileMessages = - 57; + 58; friend void swap(Expression_Enum& a, Expression_Enum& b) { a.Swap(&b); @@ -13803,7 +14149,7 @@ class Expression_Literal_VarChar final : &_Expression_Literal_VarChar_default_instance_); } static constexpr int kIndexInFileMessages = - 58; + 59; friend void swap(Expression_Literal_VarChar& a, Expression_Literal_VarChar& b) { a.Swap(&b); @@ -13965,7 +14311,7 @@ class Expression_Literal_Decimal final : &_Expression_Literal_Decimal_default_instance_); } static constexpr int kIndexInFileMessages = - 59; + 60; friend void swap(Expression_Literal_Decimal& a, Expression_Literal_Decimal& b) { a.Swap(&b); @@ -14090,24 +14436,24 @@ class Expression_Literal_Decimal final : }; // ------------------------------------------------------------------- -class Expression_Literal_Map_KeyValue final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.Map.KeyValue) */ { +class Expression_Literal_PrecisionTimestamp final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.PrecisionTimestamp) */ { public: - inline Expression_Literal_Map_KeyValue() : Expression_Literal_Map_KeyValue(nullptr) {} - ~Expression_Literal_Map_KeyValue() override; - explicit constexpr Expression_Literal_Map_KeyValue(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline Expression_Literal_PrecisionTimestamp() : Expression_Literal_PrecisionTimestamp(nullptr) {} + ~Expression_Literal_PrecisionTimestamp() override; + explicit constexpr Expression_Literal_PrecisionTimestamp(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - Expression_Literal_Map_KeyValue(const Expression_Literal_Map_KeyValue& from); - Expression_Literal_Map_KeyValue(Expression_Literal_Map_KeyValue&& from) noexcept - : Expression_Literal_Map_KeyValue() { + Expression_Literal_PrecisionTimestamp(const Expression_Literal_PrecisionTimestamp& from); + Expression_Literal_PrecisionTimestamp(Expression_Literal_PrecisionTimestamp&& from) noexcept + : Expression_Literal_PrecisionTimestamp() { *this = ::std::move(from); } - inline Expression_Literal_Map_KeyValue& operator=(const Expression_Literal_Map_KeyValue& from) { + inline Expression_Literal_PrecisionTimestamp& operator=(const Expression_Literal_PrecisionTimestamp& from) { CopyFrom(from); return *this; } - inline Expression_Literal_Map_KeyValue& operator=(Expression_Literal_Map_KeyValue&& from) noexcept { + inline Expression_Literal_PrecisionTimestamp& operator=(Expression_Literal_PrecisionTimestamp&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -14130,20 +14476,20 @@ class Expression_Literal_Map_KeyValue final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const Expression_Literal_Map_KeyValue& default_instance() { + static const Expression_Literal_PrecisionTimestamp& default_instance() { return *internal_default_instance(); } - static inline const Expression_Literal_Map_KeyValue* internal_default_instance() { - return reinterpret_cast( - &_Expression_Literal_Map_KeyValue_default_instance_); + static inline const Expression_Literal_PrecisionTimestamp* internal_default_instance() { + return reinterpret_cast( + &_Expression_Literal_PrecisionTimestamp_default_instance_); } static constexpr int kIndexInFileMessages = - 60; + 61; - friend void swap(Expression_Literal_Map_KeyValue& a, Expression_Literal_Map_KeyValue& b) { + friend void swap(Expression_Literal_PrecisionTimestamp& a, Expression_Literal_PrecisionTimestamp& b) { a.Swap(&b); } - inline void Swap(Expression_Literal_Map_KeyValue* other) { + inline void Swap(Expression_Literal_PrecisionTimestamp* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -14156,7 +14502,7 @@ class Expression_Literal_Map_KeyValue final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(Expression_Literal_Map_KeyValue* other) { + void UnsafeArenaSwap(Expression_Literal_PrecisionTimestamp* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -14164,13 +14510,13 @@ class Expression_Literal_Map_KeyValue final : // implements Message ---------------------------------------------- - Expression_Literal_Map_KeyValue* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + Expression_Literal_PrecisionTimestamp* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const Expression_Literal_Map_KeyValue& from); + void CopyFrom(const Expression_Literal_PrecisionTimestamp& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const Expression_Literal_Map_KeyValue& from); + void MergeFrom(const Expression_Literal_PrecisionTimestamp& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -14187,15 +14533,15 @@ class Expression_Literal_Map_KeyValue final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(Expression_Literal_Map_KeyValue* other); + void InternalSwap(Expression_Literal_PrecisionTimestamp* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.Expression.Literal.Map.KeyValue"; + return "substrait.Expression.Literal.PrecisionTimestamp"; } protected: - explicit Expression_Literal_Map_KeyValue(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit Expression_Literal_PrecisionTimestamp(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -14212,77 +14558,59 @@ class Expression_Literal_Map_KeyValue final : // accessors ------------------------------------------------------- enum : int { - kKeyFieldNumber = 1, kValueFieldNumber = 2, + kPrecisionFieldNumber = 1, }; - // .substrait.Expression.Literal key = 1; - bool has_key() const; - private: - bool _internal_has_key() const; - public: - void clear_key(); - const ::substrait::Expression_Literal& key() const; - PROTOBUF_NODISCARD ::substrait::Expression_Literal* release_key(); - ::substrait::Expression_Literal* mutable_key(); - void set_allocated_key(::substrait::Expression_Literal* key); + // int64 value = 2; + void clear_value(); + int64_t value() const; + void set_value(int64_t value); private: - const ::substrait::Expression_Literal& _internal_key() const; - ::substrait::Expression_Literal* _internal_mutable_key(); + int64_t _internal_value() const; + void _internal_set_value(int64_t value); public: - void unsafe_arena_set_allocated_key( - ::substrait::Expression_Literal* key); - ::substrait::Expression_Literal* unsafe_arena_release_key(); - // .substrait.Expression.Literal value = 2; - bool has_value() const; - private: - bool _internal_has_value() const; - public: - void clear_value(); - const ::substrait::Expression_Literal& value() const; - PROTOBUF_NODISCARD ::substrait::Expression_Literal* release_value(); - ::substrait::Expression_Literal* mutable_value(); - void set_allocated_value(::substrait::Expression_Literal* value); + // int32 precision = 1; + void clear_precision(); + int32_t precision() const; + void set_precision(int32_t value); private: - const ::substrait::Expression_Literal& _internal_value() const; - ::substrait::Expression_Literal* _internal_mutable_value(); + int32_t _internal_precision() const; + void _internal_set_precision(int32_t value); public: - void unsafe_arena_set_allocated_value( - ::substrait::Expression_Literal* value); - ::substrait::Expression_Literal* unsafe_arena_release_value(); - // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.Map.KeyValue) + // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.PrecisionTimestamp) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::Expression_Literal* key_; - ::substrait::Expression_Literal* value_; + int64_t value_; + int32_t precision_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; // ------------------------------------------------------------------- -class Expression_Literal_Map final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.Map) */ { +class Expression_Literal_Map_KeyValue final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.Map.KeyValue) */ { public: - inline Expression_Literal_Map() : Expression_Literal_Map(nullptr) {} - ~Expression_Literal_Map() override; - explicit constexpr Expression_Literal_Map(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline Expression_Literal_Map_KeyValue() : Expression_Literal_Map_KeyValue(nullptr) {} + ~Expression_Literal_Map_KeyValue() override; + explicit constexpr Expression_Literal_Map_KeyValue(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - Expression_Literal_Map(const Expression_Literal_Map& from); - Expression_Literal_Map(Expression_Literal_Map&& from) noexcept - : Expression_Literal_Map() { + Expression_Literal_Map_KeyValue(const Expression_Literal_Map_KeyValue& from); + Expression_Literal_Map_KeyValue(Expression_Literal_Map_KeyValue&& from) noexcept + : Expression_Literal_Map_KeyValue() { *this = ::std::move(from); } - inline Expression_Literal_Map& operator=(const Expression_Literal_Map& from) { + inline Expression_Literal_Map_KeyValue& operator=(const Expression_Literal_Map_KeyValue& from) { CopyFrom(from); return *this; } - inline Expression_Literal_Map& operator=(Expression_Literal_Map&& from) noexcept { + inline Expression_Literal_Map_KeyValue& operator=(Expression_Literal_Map_KeyValue&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -14305,20 +14633,20 @@ class Expression_Literal_Map final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const Expression_Literal_Map& default_instance() { + static const Expression_Literal_Map_KeyValue& default_instance() { return *internal_default_instance(); } - static inline const Expression_Literal_Map* internal_default_instance() { - return reinterpret_cast( - &_Expression_Literal_Map_default_instance_); + static inline const Expression_Literal_Map_KeyValue* internal_default_instance() { + return reinterpret_cast( + &_Expression_Literal_Map_KeyValue_default_instance_); } static constexpr int kIndexInFileMessages = - 61; + 62; - friend void swap(Expression_Literal_Map& a, Expression_Literal_Map& b) { + friend void swap(Expression_Literal_Map_KeyValue& a, Expression_Literal_Map_KeyValue& b) { a.Swap(&b); } - inline void Swap(Expression_Literal_Map* other) { + inline void Swap(Expression_Literal_Map_KeyValue* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -14331,7 +14659,7 @@ class Expression_Literal_Map final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(Expression_Literal_Map* other) { + void UnsafeArenaSwap(Expression_Literal_Map_KeyValue* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -14339,13 +14667,13 @@ class Expression_Literal_Map final : // implements Message ---------------------------------------------- - Expression_Literal_Map* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + Expression_Literal_Map_KeyValue* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const Expression_Literal_Map& from); + void CopyFrom(const Expression_Literal_Map_KeyValue& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const Expression_Literal_Map& from); + void MergeFrom(const Expression_Literal_Map_KeyValue& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -14362,15 +14690,15 @@ class Expression_Literal_Map final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(Expression_Literal_Map* other); + void InternalSwap(Expression_Literal_Map_KeyValue* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.Expression.Literal.Map"; + return "substrait.Expression.Literal.Map.KeyValue"; } protected: - explicit Expression_Literal_Map(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit Expression_Literal_Map_KeyValue(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -14384,62 +14712,80 @@ class Expression_Literal_Map final : // nested types ---------------------------------------------------- - typedef Expression_Literal_Map_KeyValue KeyValue; - // accessors ------------------------------------------------------- enum : int { - kKeyValuesFieldNumber = 1, + kKeyFieldNumber = 1, + kValueFieldNumber = 2, }; - // repeated .substrait.Expression.Literal.Map.KeyValue key_values = 1; - int key_values_size() const; + // .substrait.Expression.Literal key = 1; + bool has_key() const; private: - int _internal_key_values_size() const; + bool _internal_has_key() const; public: - void clear_key_values(); - ::substrait::Expression_Literal_Map_KeyValue* mutable_key_values(int index); - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue >* - mutable_key_values(); + void clear_key(); + const ::substrait::Expression_Literal& key() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal* release_key(); + ::substrait::Expression_Literal* mutable_key(); + void set_allocated_key(::substrait::Expression_Literal* key); private: - const ::substrait::Expression_Literal_Map_KeyValue& _internal_key_values(int index) const; - ::substrait::Expression_Literal_Map_KeyValue* _internal_add_key_values(); + const ::substrait::Expression_Literal& _internal_key() const; + ::substrait::Expression_Literal* _internal_mutable_key(); public: - const ::substrait::Expression_Literal_Map_KeyValue& key_values(int index) const; - ::substrait::Expression_Literal_Map_KeyValue* add_key_values(); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue >& - key_values() const; + void unsafe_arena_set_allocated_key( + ::substrait::Expression_Literal* key); + ::substrait::Expression_Literal* unsafe_arena_release_key(); - // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.Map) + // .substrait.Expression.Literal value = 2; + bool has_value() const; + private: + bool _internal_has_value() const; + public: + void clear_value(); + const ::substrait::Expression_Literal& value() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal* release_value(); + ::substrait::Expression_Literal* mutable_value(); + void set_allocated_value(::substrait::Expression_Literal* value); + private: + const ::substrait::Expression_Literal& _internal_value() const; + ::substrait::Expression_Literal* _internal_mutable_value(); + public: + void unsafe_arena_set_allocated_value( + ::substrait::Expression_Literal* value); + ::substrait::Expression_Literal* unsafe_arena_release_value(); + + // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.Map.KeyValue) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue > key_values_; + ::substrait::Expression_Literal* key_; + ::substrait::Expression_Literal* value_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; // ------------------------------------------------------------------- -class Expression_Literal_IntervalYearToMonth final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.IntervalYearToMonth) */ { +class Expression_Literal_Map final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.Map) */ { public: - inline Expression_Literal_IntervalYearToMonth() : Expression_Literal_IntervalYearToMonth(nullptr) {} - ~Expression_Literal_IntervalYearToMonth() override; - explicit constexpr Expression_Literal_IntervalYearToMonth(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline Expression_Literal_Map() : Expression_Literal_Map(nullptr) {} + ~Expression_Literal_Map() override; + explicit constexpr Expression_Literal_Map(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - Expression_Literal_IntervalYearToMonth(const Expression_Literal_IntervalYearToMonth& from); - Expression_Literal_IntervalYearToMonth(Expression_Literal_IntervalYearToMonth&& from) noexcept - : Expression_Literal_IntervalYearToMonth() { + Expression_Literal_Map(const Expression_Literal_Map& from); + Expression_Literal_Map(Expression_Literal_Map&& from) noexcept + : Expression_Literal_Map() { *this = ::std::move(from); } - inline Expression_Literal_IntervalYearToMonth& operator=(const Expression_Literal_IntervalYearToMonth& from) { + inline Expression_Literal_Map& operator=(const Expression_Literal_Map& from) { CopyFrom(from); return *this; } - inline Expression_Literal_IntervalYearToMonth& operator=(Expression_Literal_IntervalYearToMonth&& from) noexcept { + inline Expression_Literal_Map& operator=(Expression_Literal_Map&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -14462,20 +14808,20 @@ class Expression_Literal_IntervalYearToMonth final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const Expression_Literal_IntervalYearToMonth& default_instance() { + static const Expression_Literal_Map& default_instance() { return *internal_default_instance(); } - static inline const Expression_Literal_IntervalYearToMonth* internal_default_instance() { - return reinterpret_cast( - &_Expression_Literal_IntervalYearToMonth_default_instance_); + static inline const Expression_Literal_Map* internal_default_instance() { + return reinterpret_cast( + &_Expression_Literal_Map_default_instance_); } static constexpr int kIndexInFileMessages = - 62; + 63; - friend void swap(Expression_Literal_IntervalYearToMonth& a, Expression_Literal_IntervalYearToMonth& b) { + friend void swap(Expression_Literal_Map& a, Expression_Literal_Map& b) { a.Swap(&b); } - inline void Swap(Expression_Literal_IntervalYearToMonth* other) { + inline void Swap(Expression_Literal_Map* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -14488,7 +14834,7 @@ class Expression_Literal_IntervalYearToMonth final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(Expression_Literal_IntervalYearToMonth* other) { + void UnsafeArenaSwap(Expression_Literal_Map* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -14496,13 +14842,13 @@ class Expression_Literal_IntervalYearToMonth final : // implements Message ---------------------------------------------- - Expression_Literal_IntervalYearToMonth* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + Expression_Literal_Map* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const Expression_Literal_IntervalYearToMonth& from); + void CopyFrom(const Expression_Literal_Map& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const Expression_Literal_IntervalYearToMonth& from); + void MergeFrom(const Expression_Literal_Map& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -14519,15 +14865,15 @@ class Expression_Literal_IntervalYearToMonth final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(Expression_Literal_IntervalYearToMonth* other); + void InternalSwap(Expression_Literal_Map* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.Expression.Literal.IntervalYearToMonth"; + return "substrait.Expression.Literal.Map"; } protected: - explicit Expression_Literal_IntervalYearToMonth(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit Expression_Literal_Map(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -14541,62 +14887,219 @@ class Expression_Literal_IntervalYearToMonth final : // nested types ---------------------------------------------------- + typedef Expression_Literal_Map_KeyValue KeyValue; + // accessors ------------------------------------------------------- enum : int { - kYearsFieldNumber = 1, - kMonthsFieldNumber = 2, + kKeyValuesFieldNumber = 1, }; - // int32 years = 1; - void clear_years(); - int32_t years() const; - void set_years(int32_t value); + // repeated .substrait.Expression.Literal.Map.KeyValue key_values = 1; + int key_values_size() const; private: - int32_t _internal_years() const; - void _internal_set_years(int32_t value); + int _internal_key_values_size() const; public: - - // int32 months = 2; - void clear_months(); - int32_t months() const; - void set_months(int32_t value); + void clear_key_values(); + ::substrait::Expression_Literal_Map_KeyValue* mutable_key_values(int index); + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue >* + mutable_key_values(); private: - int32_t _internal_months() const; - void _internal_set_months(int32_t value); + const ::substrait::Expression_Literal_Map_KeyValue& _internal_key_values(int index) const; + ::substrait::Expression_Literal_Map_KeyValue* _internal_add_key_values(); public: + const ::substrait::Expression_Literal_Map_KeyValue& key_values(int index) const; + ::substrait::Expression_Literal_Map_KeyValue* add_key_values(); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue >& + key_values() const; - // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.IntervalYearToMonth) + // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.Map) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - int32_t years_; - int32_t months_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_Literal_Map_KeyValue > key_values_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; // ------------------------------------------------------------------- -class Expression_Literal_IntervalDayToSecond final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.IntervalDayToSecond) */ { +class Expression_Literal_IntervalYearToMonth final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.IntervalYearToMonth) */ { public: - inline Expression_Literal_IntervalDayToSecond() : Expression_Literal_IntervalDayToSecond(nullptr) {} - ~Expression_Literal_IntervalDayToSecond() override; - explicit constexpr Expression_Literal_IntervalDayToSecond(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline Expression_Literal_IntervalYearToMonth() : Expression_Literal_IntervalYearToMonth(nullptr) {} + ~Expression_Literal_IntervalYearToMonth() override; + explicit constexpr Expression_Literal_IntervalYearToMonth(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - Expression_Literal_IntervalDayToSecond(const Expression_Literal_IntervalDayToSecond& from); - Expression_Literal_IntervalDayToSecond(Expression_Literal_IntervalDayToSecond&& from) noexcept - : Expression_Literal_IntervalDayToSecond() { + Expression_Literal_IntervalYearToMonth(const Expression_Literal_IntervalYearToMonth& from); + Expression_Literal_IntervalYearToMonth(Expression_Literal_IntervalYearToMonth&& from) noexcept + : Expression_Literal_IntervalYearToMonth() { *this = ::std::move(from); } - inline Expression_Literal_IntervalDayToSecond& operator=(const Expression_Literal_IntervalDayToSecond& from) { + inline Expression_Literal_IntervalYearToMonth& operator=(const Expression_Literal_IntervalYearToMonth& from) { CopyFrom(from); return *this; } - inline Expression_Literal_IntervalDayToSecond& operator=(Expression_Literal_IntervalDayToSecond&& from) noexcept { + inline Expression_Literal_IntervalYearToMonth& operator=(Expression_Literal_IntervalYearToMonth&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const Expression_Literal_IntervalYearToMonth& default_instance() { + return *internal_default_instance(); + } + static inline const Expression_Literal_IntervalYearToMonth* internal_default_instance() { + return reinterpret_cast( + &_Expression_Literal_IntervalYearToMonth_default_instance_); + } + static constexpr int kIndexInFileMessages = + 64; + + friend void swap(Expression_Literal_IntervalYearToMonth& a, Expression_Literal_IntervalYearToMonth& b) { + a.Swap(&b); + } + inline void Swap(Expression_Literal_IntervalYearToMonth* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(Expression_Literal_IntervalYearToMonth* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + Expression_Literal_IntervalYearToMonth* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const Expression_Literal_IntervalYearToMonth& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const Expression_Literal_IntervalYearToMonth& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(Expression_Literal_IntervalYearToMonth* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.Expression.Literal.IntervalYearToMonth"; + } + protected: + explicit Expression_Literal_IntervalYearToMonth(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kYearsFieldNumber = 1, + kMonthsFieldNumber = 2, + }; + // int32 years = 1; + void clear_years(); + int32_t years() const; + void set_years(int32_t value); + private: + int32_t _internal_years() const; + void _internal_set_years(int32_t value); + public: + + // int32 months = 2; + void clear_months(); + int32_t months() const; + void set_months(int32_t value); + private: + int32_t _internal_months() const; + void _internal_set_months(int32_t value); + public: + + // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.IntervalYearToMonth) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + int32_t years_; + int32_t months_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2falgebra_2eproto; +}; +// ------------------------------------------------------------------- + +class Expression_Literal_IntervalDayToSecond final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.IntervalDayToSecond) */ { + public: + inline Expression_Literal_IntervalDayToSecond() : Expression_Literal_IntervalDayToSecond(nullptr) {} + ~Expression_Literal_IntervalDayToSecond() override; + explicit constexpr Expression_Literal_IntervalDayToSecond(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + Expression_Literal_IntervalDayToSecond(const Expression_Literal_IntervalDayToSecond& from); + Expression_Literal_IntervalDayToSecond(Expression_Literal_IntervalDayToSecond&& from) noexcept + : Expression_Literal_IntervalDayToSecond() { + *this = ::std::move(from); + } + + inline Expression_Literal_IntervalDayToSecond& operator=(const Expression_Literal_IntervalDayToSecond& from) { + CopyFrom(from); + return *this; + } + inline Expression_Literal_IntervalDayToSecond& operator=(Expression_Literal_IntervalDayToSecond&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -14622,12 +15125,18 @@ class Expression_Literal_IntervalDayToSecond final : static const Expression_Literal_IntervalDayToSecond& default_instance() { return *internal_default_instance(); } + enum PrecisionModeCase { + kMicroseconds = 3, + kPrecision = 4, + PRECISION_MODE_NOT_SET = 0, + }; + static inline const Expression_Literal_IntervalDayToSecond* internal_default_instance() { return reinterpret_cast( &_Expression_Literal_IntervalDayToSecond_default_instance_); } static constexpr int kIndexInFileMessages = - 63; + 65; friend void swap(Expression_Literal_IntervalDayToSecond& a, Expression_Literal_IntervalDayToSecond& b) { a.Swap(&b); @@ -14703,7 +15212,9 @@ class Expression_Literal_IntervalDayToSecond final : enum : int { kDaysFieldNumber = 1, kSecondsFieldNumber = 2, + kSubsecondsFieldNumber = 5, kMicrosecondsFieldNumber = 3, + kPrecisionFieldNumber = 4, }; // int32 days = 1; void clear_days(); @@ -14723,25 +15234,241 @@ class Expression_Literal_IntervalDayToSecond final : void _internal_set_seconds(int32_t value); public: - // int32 microseconds = 3; - void clear_microseconds(); - int32_t microseconds() const; - void set_microseconds(int32_t value); + // int64 subseconds = 5; + void clear_subseconds(); + int64_t subseconds() const; + void set_subseconds(int64_t value); + private: + int64_t _internal_subseconds() const; + void _internal_set_subseconds(int64_t value); + public: + + // int32 microseconds = 3 [deprecated = true]; + PROTOBUF_DEPRECATED bool has_microseconds() const; + private: + bool _internal_has_microseconds() const; + public: + PROTOBUF_DEPRECATED void clear_microseconds(); + PROTOBUF_DEPRECATED int32_t microseconds() const; + PROTOBUF_DEPRECATED void set_microseconds(int32_t value); private: int32_t _internal_microseconds() const; void _internal_set_microseconds(int32_t value); public: + // int32 precision = 4; + bool has_precision() const; + private: + bool _internal_has_precision() const; + public: + void clear_precision(); + int32_t precision() const; + void set_precision(int32_t value); + private: + int32_t _internal_precision() const; + void _internal_set_precision(int32_t value); + public: + + void clear_precision_mode(); + PrecisionModeCase precision_mode_case() const; // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.IntervalDayToSecond) private: class _Internal; + void set_has_microseconds(); + void set_has_precision(); + + inline bool has_precision_mode() const; + inline void clear_has_precision_mode(); template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; int32_t days_; int32_t seconds_; - int32_t microseconds_; + int64_t subseconds_; + union PrecisionModeUnion { + constexpr PrecisionModeUnion() : _constinit_{} {} + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized _constinit_; + int32_t microseconds_; + int32_t precision_; + } precision_mode_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + uint32_t _oneof_case_[1]; + + friend struct ::TableStruct_substrait_2falgebra_2eproto; +}; +// ------------------------------------------------------------------- + +class Expression_Literal_IntervalCompound final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Expression.Literal.IntervalCompound) */ { + public: + inline Expression_Literal_IntervalCompound() : Expression_Literal_IntervalCompound(nullptr) {} + ~Expression_Literal_IntervalCompound() override; + explicit constexpr Expression_Literal_IntervalCompound(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + Expression_Literal_IntervalCompound(const Expression_Literal_IntervalCompound& from); + Expression_Literal_IntervalCompound(Expression_Literal_IntervalCompound&& from) noexcept + : Expression_Literal_IntervalCompound() { + *this = ::std::move(from); + } + + inline Expression_Literal_IntervalCompound& operator=(const Expression_Literal_IntervalCompound& from) { + CopyFrom(from); + return *this; + } + inline Expression_Literal_IntervalCompound& operator=(Expression_Literal_IntervalCompound&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const Expression_Literal_IntervalCompound& default_instance() { + return *internal_default_instance(); + } + static inline const Expression_Literal_IntervalCompound* internal_default_instance() { + return reinterpret_cast( + &_Expression_Literal_IntervalCompound_default_instance_); + } + static constexpr int kIndexInFileMessages = + 66; + + friend void swap(Expression_Literal_IntervalCompound& a, Expression_Literal_IntervalCompound& b) { + a.Swap(&b); + } + inline void Swap(Expression_Literal_IntervalCompound* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(Expression_Literal_IntervalCompound* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + Expression_Literal_IntervalCompound* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const Expression_Literal_IntervalCompound& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const Expression_Literal_IntervalCompound& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(Expression_Literal_IntervalCompound* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.Expression.Literal.IntervalCompound"; + } + protected: + explicit Expression_Literal_IntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kIntervalYearToMonthFieldNumber = 1, + kIntervalDayToSecondFieldNumber = 2, + }; + // .substrait.Expression.Literal.IntervalYearToMonth interval_year_to_month = 1; + bool has_interval_year_to_month() const; + private: + bool _internal_has_interval_year_to_month() const; + public: + void clear_interval_year_to_month(); + const ::substrait::Expression_Literal_IntervalYearToMonth& interval_year_to_month() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal_IntervalYearToMonth* release_interval_year_to_month(); + ::substrait::Expression_Literal_IntervalYearToMonth* mutable_interval_year_to_month(); + void set_allocated_interval_year_to_month(::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month); + private: + const ::substrait::Expression_Literal_IntervalYearToMonth& _internal_interval_year_to_month() const; + ::substrait::Expression_Literal_IntervalYearToMonth* _internal_mutable_interval_year_to_month(); + public: + void unsafe_arena_set_allocated_interval_year_to_month( + ::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month); + ::substrait::Expression_Literal_IntervalYearToMonth* unsafe_arena_release_interval_year_to_month(); + + // .substrait.Expression.Literal.IntervalDayToSecond interval_day_to_second = 2; + bool has_interval_day_to_second() const; + private: + bool _internal_has_interval_day_to_second() const; + public: + void clear_interval_day_to_second(); + const ::substrait::Expression_Literal_IntervalDayToSecond& interval_day_to_second() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal_IntervalDayToSecond* release_interval_day_to_second(); + ::substrait::Expression_Literal_IntervalDayToSecond* mutable_interval_day_to_second(); + void set_allocated_interval_day_to_second(::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second); + private: + const ::substrait::Expression_Literal_IntervalDayToSecond& _internal_interval_day_to_second() const; + ::substrait::Expression_Literal_IntervalDayToSecond* _internal_mutable_interval_day_to_second(); + public: + void unsafe_arena_set_allocated_interval_day_to_second( + ::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second); + ::substrait::Expression_Literal_IntervalDayToSecond* unsafe_arena_release_interval_day_to_second(); + + // @@protoc_insertion_point(class_scope:substrait.Expression.Literal.IntervalCompound) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month_; + ::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2falgebra_2eproto; }; @@ -14795,7 +15522,7 @@ class Expression_Literal_Struct final : &_Expression_Literal_Struct_default_instance_); } static constexpr int kIndexInFileMessages = - 64; + 67; friend void swap(Expression_Literal_Struct& a, Expression_Literal_Struct& b) { a.Swap(&b); @@ -14950,7 +15677,7 @@ class Expression_Literal_List final : &_Expression_Literal_List_default_instance_); } static constexpr int kIndexInFileMessages = - 65; + 68; friend void swap(Expression_Literal_List& a, Expression_Literal_List& b) { a.Swap(&b); @@ -15111,7 +15838,7 @@ class Expression_Literal_UserDefined final : &_Expression_Literal_UserDefined_default_instance_); } static constexpr int kIndexInFileMessages = - 66; + 69; friend void swap(Expression_Literal_UserDefined& a, Expression_Literal_UserDefined& b) { a.Swap(&b); @@ -15340,6 +16067,7 @@ class Expression_Literal final : kTime = 17, kIntervalYearToMonth = 19, kIntervalDayToSecond = 20, + kIntervalCompound = 36, kFixedChar = 21, kVarChar = 22, kFixedBinary = 23, @@ -15363,7 +16091,7 @@ class Expression_Literal final : &_Expression_Literal_default_instance_); } static constexpr int kIndexInFileMessages = - 67; + 70; friend void swap(Expression_Literal& a, Expression_Literal& b) { a.Swap(&b); @@ -15436,9 +16164,11 @@ class Expression_Literal final : typedef Expression_Literal_VarChar VarChar; typedef Expression_Literal_Decimal Decimal; + typedef Expression_Literal_PrecisionTimestamp PrecisionTimestamp; typedef Expression_Literal_Map Map; typedef Expression_Literal_IntervalYearToMonth IntervalYearToMonth; typedef Expression_Literal_IntervalDayToSecond IntervalDayToSecond; + typedef Expression_Literal_IntervalCompound IntervalCompound; typedef Expression_Literal_Struct Struct; typedef Expression_Literal_List List; typedef Expression_Literal_UserDefined UserDefined; @@ -15462,6 +16192,7 @@ class Expression_Literal final : kTimeFieldNumber = 17, kIntervalYearToMonthFieldNumber = 19, kIntervalDayToSecondFieldNumber = 20, + kIntervalCompoundFieldNumber = 36, kFixedCharFieldNumber = 21, kVarCharFieldNumber = 22, kFixedBinaryFieldNumber = 23, @@ -15698,6 +16429,24 @@ class Expression_Literal final : ::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second); ::substrait::Expression_Literal_IntervalDayToSecond* unsafe_arena_release_interval_day_to_second(); + // .substrait.Expression.Literal.IntervalCompound interval_compound = 36; + bool has_interval_compound() const; + private: + bool _internal_has_interval_compound() const; + public: + void clear_interval_compound(); + const ::substrait::Expression_Literal_IntervalCompound& interval_compound() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal_IntervalCompound* release_interval_compound(); + ::substrait::Expression_Literal_IntervalCompound* mutable_interval_compound(); + void set_allocated_interval_compound(::substrait::Expression_Literal_IntervalCompound* interval_compound); + private: + const ::substrait::Expression_Literal_IntervalCompound& _internal_interval_compound() const; + ::substrait::Expression_Literal_IntervalCompound* _internal_mutable_interval_compound(); + public: + void unsafe_arena_set_allocated_interval_compound( + ::substrait::Expression_Literal_IntervalCompound* interval_compound); + ::substrait::Expression_Literal_IntervalCompound* unsafe_arena_release_interval_compound(); + // string fixed_char = 21; bool has_fixed_char() const; private: @@ -15770,31 +16519,41 @@ class Expression_Literal final : ::substrait::Expression_Literal_Decimal* decimal); ::substrait::Expression_Literal_Decimal* unsafe_arena_release_decimal(); - // uint64 precision_timestamp = 34; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp = 34; bool has_precision_timestamp() const; private: bool _internal_has_precision_timestamp() const; public: void clear_precision_timestamp(); - uint64_t precision_timestamp() const; - void set_precision_timestamp(uint64_t value); + const ::substrait::Expression_Literal_PrecisionTimestamp& precision_timestamp() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal_PrecisionTimestamp* release_precision_timestamp(); + ::substrait::Expression_Literal_PrecisionTimestamp* mutable_precision_timestamp(); + void set_allocated_precision_timestamp(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp); private: - uint64_t _internal_precision_timestamp() const; - void _internal_set_precision_timestamp(uint64_t value); + const ::substrait::Expression_Literal_PrecisionTimestamp& _internal_precision_timestamp() const; + ::substrait::Expression_Literal_PrecisionTimestamp* _internal_mutable_precision_timestamp(); public: + void unsafe_arena_set_allocated_precision_timestamp( + ::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp); + ::substrait::Expression_Literal_PrecisionTimestamp* unsafe_arena_release_precision_timestamp(); - // uint64 precision_timestamp_tz = 35; + // .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp_tz = 35; bool has_precision_timestamp_tz() const; private: bool _internal_has_precision_timestamp_tz() const; public: void clear_precision_timestamp_tz(); - uint64_t precision_timestamp_tz() const; - void set_precision_timestamp_tz(uint64_t value); + const ::substrait::Expression_Literal_PrecisionTimestamp& precision_timestamp_tz() const; + PROTOBUF_NODISCARD ::substrait::Expression_Literal_PrecisionTimestamp* release_precision_timestamp_tz(); + ::substrait::Expression_Literal_PrecisionTimestamp* mutable_precision_timestamp_tz(); + void set_allocated_precision_timestamp_tz(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_tz); private: - uint64_t _internal_precision_timestamp_tz() const; - void _internal_set_precision_timestamp_tz(uint64_t value); + const ::substrait::Expression_Literal_PrecisionTimestamp& _internal_precision_timestamp_tz() const; + ::substrait::Expression_Literal_PrecisionTimestamp* _internal_mutable_precision_timestamp_tz(); public: + void unsafe_arena_set_allocated_precision_timestamp_tz( + ::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_tz); + ::substrait::Expression_Literal_PrecisionTimestamp* unsafe_arena_release_precision_timestamp_tz(); // .substrait.Expression.Literal.Struct struct = 25; bool has_struct_() const; @@ -15972,6 +16731,7 @@ class Expression_Literal final : void set_has_time(); void set_has_interval_year_to_month(); void set_has_interval_day_to_second(); + void set_has_interval_compound(); void set_has_fixed_char(); void set_has_var_char(); void set_has_fixed_binary(); @@ -16013,12 +16773,13 @@ class Expression_Literal final : int64_t time_; ::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month_; ::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second_; + ::substrait::Expression_Literal_IntervalCompound* interval_compound_; ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr fixed_char_; ::substrait::Expression_Literal_VarChar* var_char_; ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr fixed_binary_; ::substrait::Expression_Literal_Decimal* decimal_; - uint64_t precision_timestamp_; - uint64_t precision_timestamp_tz_; + ::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_; + ::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_tz_; ::substrait::Expression_Literal_Struct* struct__; ::substrait::Expression_Literal_Map* map_; int64_t timestamp_tz_; @@ -16084,7 +16845,7 @@ class Expression_Nested_Map_KeyValue final : &_Expression_Nested_Map_KeyValue_default_instance_); } static constexpr int kIndexInFileMessages = - 68; + 71; friend void swap(Expression_Nested_Map_KeyValue& a, Expression_Nested_Map_KeyValue& b) { a.Swap(&b); @@ -16259,7 +17020,7 @@ class Expression_Nested_Map final : &_Expression_Nested_Map_default_instance_); } static constexpr int kIndexInFileMessages = - 69; + 72; friend void swap(Expression_Nested_Map& a, Expression_Nested_Map& b) { a.Swap(&b); @@ -16416,7 +17177,7 @@ class Expression_Nested_Struct final : &_Expression_Nested_Struct_default_instance_); } static constexpr int kIndexInFileMessages = - 70; + 73; friend void swap(Expression_Nested_Struct& a, Expression_Nested_Struct& b) { a.Swap(&b); @@ -16571,7 +17332,7 @@ class Expression_Nested_List final : &_Expression_Nested_List_default_instance_); } static constexpr int kIndexInFileMessages = - 71; + 74; friend void swap(Expression_Nested_List& a, Expression_Nested_List& b) { a.Swap(&b); @@ -16733,7 +17494,7 @@ class Expression_Nested final : &_Expression_Nested_default_instance_); } static constexpr int kIndexInFileMessages = - 72; + 75; friend void swap(Expression_Nested& a, Expression_Nested& b) { a.Swap(&b); @@ -16968,7 +17729,7 @@ class Expression_ScalarFunction final : &_Expression_ScalarFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 73; + 76; friend void swap(Expression_ScalarFunction& a, Expression_ScalarFunction& b) { a.Swap(&b); @@ -17194,7 +17955,7 @@ class Expression_WindowFunction_Bound_Preceding final : &_Expression_WindowFunction_Bound_Preceding_default_instance_); } static constexpr int kIndexInFileMessages = - 74; + 77; friend void swap(Expression_WindowFunction_Bound_Preceding& a, Expression_WindowFunction_Bound_Preceding& b) { a.Swap(&b); @@ -17340,7 +18101,7 @@ class Expression_WindowFunction_Bound_Following final : &_Expression_WindowFunction_Bound_Following_default_instance_); } static constexpr int kIndexInFileMessages = - 75; + 78; friend void swap(Expression_WindowFunction_Bound_Following& a, Expression_WindowFunction_Bound_Following& b) { a.Swap(&b); @@ -17485,7 +18246,7 @@ class Expression_WindowFunction_Bound_CurrentRow final : &_Expression_WindowFunction_Bound_CurrentRow_default_instance_); } static constexpr int kIndexInFileMessages = - 76; + 79; friend void swap(Expression_WindowFunction_Bound_CurrentRow& a, Expression_WindowFunction_Bound_CurrentRow& b) { a.Swap(&b); @@ -17603,7 +18364,7 @@ class Expression_WindowFunction_Bound_Unbounded final : &_Expression_WindowFunction_Bound_Unbounded_default_instance_); } static constexpr int kIndexInFileMessages = - 77; + 80; friend void swap(Expression_WindowFunction_Bound_Unbounded& a, Expression_WindowFunction_Bound_Unbounded& b) { a.Swap(&b); @@ -17730,7 +18491,7 @@ class Expression_WindowFunction_Bound final : &_Expression_WindowFunction_Bound_default_instance_); } static constexpr int kIndexInFileMessages = - 78; + 81; friend void swap(Expression_WindowFunction_Bound& a, Expression_WindowFunction_Bound& b) { a.Swap(&b); @@ -17965,7 +18726,7 @@ class Expression_WindowFunction final : &_Expression_WindowFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 79; + 82; friend void swap(Expression_WindowFunction& a, Expression_WindowFunction& b) { a.Swap(&b); @@ -18338,7 +19099,7 @@ class Expression_IfThen_IfClause final : &_Expression_IfThen_IfClause_default_instance_); } static constexpr int kIndexInFileMessages = - 80; + 83; friend void swap(Expression_IfThen_IfClause& a, Expression_IfThen_IfClause& b) { a.Swap(&b); @@ -18513,7 +19274,7 @@ class Expression_IfThen final : &_Expression_IfThen_default_instance_); } static constexpr int kIndexInFileMessages = - 81; + 84; friend void swap(Expression_IfThen& a, Expression_IfThen& b) { a.Swap(&b); @@ -18690,7 +19451,7 @@ class Expression_Cast final : &_Expression_Cast_default_instance_); } static constexpr int kIndexInFileMessages = - 82; + 85; friend void swap(Expression_Cast& a, Expression_Cast& b) { a.Swap(&b); @@ -18908,7 +19669,7 @@ class Expression_SwitchExpression_IfValue final : &_Expression_SwitchExpression_IfValue_default_instance_); } static constexpr int kIndexInFileMessages = - 83; + 86; friend void swap(Expression_SwitchExpression_IfValue& a, Expression_SwitchExpression_IfValue& b) { a.Swap(&b); @@ -19083,7 +19844,7 @@ class Expression_SwitchExpression final : &_Expression_SwitchExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 84; + 87; friend void swap(Expression_SwitchExpression& a, Expression_SwitchExpression& b) { a.Swap(&b); @@ -19280,7 +20041,7 @@ class Expression_SingularOrList final : &_Expression_SingularOrList_default_instance_); } static constexpr int kIndexInFileMessages = - 85; + 88; friend void swap(Expression_SingularOrList& a, Expression_SingularOrList& b) { a.Swap(&b); @@ -19455,7 +20216,7 @@ class Expression_MultiOrList_Record final : &_Expression_MultiOrList_Record_default_instance_); } static constexpr int kIndexInFileMessages = - 86; + 89; friend void swap(Expression_MultiOrList_Record& a, Expression_MultiOrList_Record& b) { a.Swap(&b); @@ -19610,7 +20371,7 @@ class Expression_MultiOrList final : &_Expression_MultiOrList_default_instance_); } static constexpr int kIndexInFileMessages = - 87; + 90; friend void swap(Expression_MultiOrList& a, Expression_MultiOrList& b) { a.Swap(&b); @@ -19787,7 +20548,7 @@ class Expression_EmbeddedFunction_PythonPickleFunction final : &_Expression_EmbeddedFunction_PythonPickleFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 88; + 91; friend void swap(Expression_EmbeddedFunction_PythonPickleFunction& a, Expression_EmbeddedFunction_PythonPickleFunction& b) { a.Swap(&b); @@ -19964,7 +20725,7 @@ class Expression_EmbeddedFunction_WebAssemblyFunction final : &_Expression_EmbeddedFunction_WebAssemblyFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 89; + 92; friend void swap(Expression_EmbeddedFunction_WebAssemblyFunction& a, Expression_EmbeddedFunction_WebAssemblyFunction& b) { a.Swap(&b); @@ -20147,7 +20908,7 @@ class Expression_EmbeddedFunction final : &_Expression_EmbeddedFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 90; + 93; friend void swap(Expression_EmbeddedFunction& a, Expression_EmbeddedFunction& b) { a.Swap(&b); @@ -20378,7 +21139,7 @@ class Expression_ReferenceSegment_MapKey final : &_Expression_ReferenceSegment_MapKey_default_instance_); } static constexpr int kIndexInFileMessages = - 91; + 94; friend void swap(Expression_ReferenceSegment_MapKey& a, Expression_ReferenceSegment_MapKey& b) { a.Swap(&b); @@ -20553,7 +21314,7 @@ class Expression_ReferenceSegment_StructField final : &_Expression_ReferenceSegment_StructField_default_instance_); } static constexpr int kIndexInFileMessages = - 92; + 95; friend void swap(Expression_ReferenceSegment_StructField& a, Expression_ReferenceSegment_StructField& b) { a.Swap(&b); @@ -20719,7 +21480,7 @@ class Expression_ReferenceSegment_ListElement final : &_Expression_ReferenceSegment_ListElement_default_instance_); } static constexpr int kIndexInFileMessages = - 93; + 96; friend void swap(Expression_ReferenceSegment_ListElement& a, Expression_ReferenceSegment_ListElement& b) { a.Swap(&b); @@ -20892,7 +21653,7 @@ class Expression_ReferenceSegment final : &_Expression_ReferenceSegment_default_instance_); } static constexpr int kIndexInFileMessages = - 94; + 97; friend void swap(Expression_ReferenceSegment& a, Expression_ReferenceSegment& b) { a.Swap(&b); @@ -21112,7 +21873,7 @@ class Expression_MaskExpression_Select final : &_Expression_MaskExpression_Select_default_instance_); } static constexpr int kIndexInFileMessages = - 95; + 98; friend void swap(Expression_MaskExpression_Select& a, Expression_MaskExpression_Select& b) { a.Swap(&b); @@ -21321,7 +22082,7 @@ class Expression_MaskExpression_StructSelect final : &_Expression_MaskExpression_StructSelect_default_instance_); } static constexpr int kIndexInFileMessages = - 96; + 99; friend void swap(Expression_MaskExpression_StructSelect& a, Expression_MaskExpression_StructSelect& b) { a.Swap(&b); @@ -21476,7 +22237,7 @@ class Expression_MaskExpression_StructItem final : &_Expression_MaskExpression_StructItem_default_instance_); } static constexpr int kIndexInFileMessages = - 97; + 100; friend void swap(Expression_MaskExpression_StructItem& a, Expression_MaskExpression_StructItem& b) { a.Swap(&b); @@ -21642,7 +22403,7 @@ class Expression_MaskExpression_ListSelect_ListSelectItem_ListElement final : &_Expression_MaskExpression_ListSelect_ListSelectItem_ListElement_default_instance_); } static constexpr int kIndexInFileMessages = - 98; + 101; friend void swap(Expression_MaskExpression_ListSelect_ListSelectItem_ListElement& a, Expression_MaskExpression_ListSelect_ListSelectItem_ListElement& b) { a.Swap(&b); @@ -21788,7 +22549,7 @@ class Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice final : &_Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice_default_instance_); } static constexpr int kIndexInFileMessages = - 99; + 102; friend void swap(Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice& a, Expression_MaskExpression_ListSelect_ListSelectItem_ListSlice& b) { a.Swap(&b); @@ -21951,7 +22712,7 @@ class Expression_MaskExpression_ListSelect_ListSelectItem final : &_Expression_MaskExpression_ListSelect_ListSelectItem_default_instance_); } static constexpr int kIndexInFileMessages = - 100; + 103; friend void swap(Expression_MaskExpression_ListSelect_ListSelectItem& a, Expression_MaskExpression_ListSelect_ListSelectItem& b) { a.Swap(&b); @@ -22142,7 +22903,7 @@ class Expression_MaskExpression_ListSelect final : &_Expression_MaskExpression_ListSelect_default_instance_); } static constexpr int kIndexInFileMessages = - 101; + 104; friend void swap(Expression_MaskExpression_ListSelect& a, Expression_MaskExpression_ListSelect& b) { a.Swap(&b); @@ -22319,7 +23080,7 @@ class Expression_MaskExpression_MapSelect_MapKey final : &_Expression_MaskExpression_MapSelect_MapKey_default_instance_); } static constexpr int kIndexInFileMessages = - 102; + 105; friend void swap(Expression_MaskExpression_MapSelect_MapKey& a, Expression_MaskExpression_MapSelect_MapKey& b) { a.Swap(&b); @@ -22470,7 +23231,7 @@ class Expression_MaskExpression_MapSelect_MapKeyExpression final : &_Expression_MaskExpression_MapSelect_MapKeyExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 103; + 106; friend void swap(Expression_MaskExpression_MapSelect_MapKeyExpression& a, Expression_MaskExpression_MapSelect_MapKeyExpression& b) { a.Swap(&b); @@ -22627,7 +23388,7 @@ class Expression_MaskExpression_MapSelect final : &_Expression_MaskExpression_MapSelect_default_instance_); } static constexpr int kIndexInFileMessages = - 104; + 107; friend void swap(Expression_MaskExpression_MapSelect& a, Expression_MaskExpression_MapSelect& b) { a.Swap(&b); @@ -22838,7 +23599,7 @@ class Expression_MaskExpression final : &_Expression_MaskExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 105; + 108; friend void swap(Expression_MaskExpression& a, Expression_MaskExpression& b) { a.Swap(&b); @@ -23009,7 +23770,7 @@ class Expression_FieldReference_RootReference final : &_Expression_FieldReference_RootReference_default_instance_); } static constexpr int kIndexInFileMessages = - 106; + 109; friend void swap(Expression_FieldReference_RootReference& a, Expression_FieldReference_RootReference& b) { a.Swap(&b); @@ -23128,7 +23889,7 @@ class Expression_FieldReference_OuterReference final : &_Expression_FieldReference_OuterReference_default_instance_); } static constexpr int kIndexInFileMessages = - 107; + 110; friend void swap(Expression_FieldReference_OuterReference& a, Expression_FieldReference_OuterReference& b) { a.Swap(&b); @@ -23287,7 +24048,7 @@ class Expression_FieldReference final : &_Expression_FieldReference_default_instance_); } static constexpr int kIndexInFileMessages = - 108; + 111; friend void swap(Expression_FieldReference& a, Expression_FieldReference& b) { a.Swap(&b); @@ -23550,7 +24311,7 @@ class Expression_Subquery_Scalar final : &_Expression_Subquery_Scalar_default_instance_); } static constexpr int kIndexInFileMessages = - 109; + 112; friend void swap(Expression_Subquery_Scalar& a, Expression_Subquery_Scalar& b) { a.Swap(&b); @@ -23705,7 +24466,7 @@ class Expression_Subquery_InPredicate final : &_Expression_Subquery_InPredicate_default_instance_); } static constexpr int kIndexInFileMessages = - 110; + 113; friend void swap(Expression_Subquery_InPredicate& a, Expression_Subquery_InPredicate& b) { a.Swap(&b); @@ -23880,7 +24641,7 @@ class Expression_Subquery_SetPredicate final : &_Expression_Subquery_SetPredicate_default_instance_); } static constexpr int kIndexInFileMessages = - 111; + 114; friend void swap(Expression_Subquery_SetPredicate& a, Expression_Subquery_SetPredicate& b) { a.Swap(&b); @@ -24078,7 +24839,7 @@ class Expression_Subquery_SetComparison final : &_Expression_Subquery_SetComparison_default_instance_); } static constexpr int kIndexInFileMessages = - 112; + 115; friend void swap(Expression_Subquery_SetComparison& a, Expression_Subquery_SetComparison& b) { a.Swap(&b); @@ -24355,7 +25116,7 @@ class Expression_Subquery final : &_Expression_Subquery_default_instance_); } static constexpr int kIndexInFileMessages = - 113; + 116; friend void swap(Expression_Subquery& a, Expression_Subquery& b) { a.Swap(&b); @@ -24606,7 +25367,7 @@ class Expression final : &_Expression_default_instance_); } static constexpr int kIndexInFileMessages = - 114; + 117; friend void swap(Expression& a, Expression& b) { a.Swap(&b); @@ -25026,7 +25787,7 @@ class SortField final : &_SortField_default_instance_); } static constexpr int kIndexInFileMessages = - 115; + 118; friend void swap(SortField& a, SortField& b) { a.Swap(&b); @@ -25262,7 +26023,7 @@ class AggregateFunction final : &_AggregateFunction_default_instance_); } static constexpr int kIndexInFileMessages = - 116; + 119; friend void swap(AggregateFunction& a, AggregateFunction& b) { a.Swap(&b); @@ -25562,7 +26323,7 @@ class ReferenceRel final : &_ReferenceRel_default_instance_); } static constexpr int kIndexInFileMessages = - 117; + 120; friend void swap(ReferenceRel& a, ReferenceRel& b) { a.Swap(&b); @@ -26124,6 +26885,57 @@ inline void RelCommon_Hint::set_allocated_constraint(::substrait::RelCommon_Hint // @@protoc_insertion_point(field_set_allocated:substrait.RelCommon.Hint.constraint) } +// string alias = 3; +inline void RelCommon_Hint::clear_alias() { + alias_.ClearToEmpty(); +} +inline const std::string& RelCommon_Hint::alias() const { + // @@protoc_insertion_point(field_get:substrait.RelCommon.Hint.alias) + return _internal_alias(); +} +template +inline PROTOBUF_ALWAYS_INLINE +void RelCommon_Hint::set_alias(ArgT0&& arg0, ArgT... args) { + + alias_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, static_cast(arg0), args..., GetArenaForAllocation()); + // @@protoc_insertion_point(field_set:substrait.RelCommon.Hint.alias) +} +inline std::string* RelCommon_Hint::mutable_alias() { + std::string* _s = _internal_mutable_alias(); + // @@protoc_insertion_point(field_mutable:substrait.RelCommon.Hint.alias) + return _s; +} +inline const std::string& RelCommon_Hint::_internal_alias() const { + return alias_.Get(); +} +inline void RelCommon_Hint::_internal_set_alias(const std::string& value) { + + alias_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, value, GetArenaForAllocation()); +} +inline std::string* RelCommon_Hint::_internal_mutable_alias() { + + return alias_.Mutable(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); +} +inline std::string* RelCommon_Hint::release_alias() { + // @@protoc_insertion_point(field_release:substrait.RelCommon.Hint.alias) + return alias_.Release(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), GetArenaForAllocation()); +} +inline void RelCommon_Hint::set_allocated_alias(std::string* alias) { + if (alias != nullptr) { + + } else { + + } + alias_.SetAllocated(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), alias, + GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (alias_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + alias_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + // @@protoc_insertion_point(field_set_allocated:substrait.RelCommon.Hint.alias) +} + // .substrait.extensions.AdvancedExtension advanced_extension = 10; inline bool RelCommon_Hint::_internal_has_advanced_extension() const { return this != internal_default_instance() && advanced_extension_ != nullptr; @@ -26864,6 +27676,272 @@ inline void ReadRel_ExtensionTable::set_allocated_detail(::PROTOBUF_NAMESPACE_ID // ------------------------------------------------------------------- +// ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions + +// string field_delimiter = 1; +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_field_delimiter() { + field_delimiter_.ClearToEmpty(); +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::field_delimiter() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter) + return _internal_field_delimiter(); +} +template +inline PROTOBUF_ALWAYS_INLINE +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_field_delimiter(ArgT0&& arg0, ArgT... args) { + + field_delimiter_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, static_cast(arg0), args..., GetArenaForAllocation()); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter) +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::mutable_field_delimiter() { + std::string* _s = _internal_mutable_field_delimiter(); + // @@protoc_insertion_point(field_mutable:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter) + return _s; +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_field_delimiter() const { + return field_delimiter_.Get(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_field_delimiter(const std::string& value) { + + field_delimiter_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, value, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_mutable_field_delimiter() { + + return field_delimiter_.Mutable(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::release_field_delimiter() { + // @@protoc_insertion_point(field_release:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter) + return field_delimiter_.Release(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), GetArenaForAllocation()); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_allocated_field_delimiter(std::string* field_delimiter) { + if (field_delimiter != nullptr) { + + } else { + + } + field_delimiter_.SetAllocated(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), field_delimiter, + GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (field_delimiter_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + field_delimiter_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.field_delimiter) +} + +// uint64 max_line_size = 2; +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_max_line_size() { + max_line_size_ = uint64_t{0u}; +} +inline uint64_t ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_max_line_size() const { + return max_line_size_; +} +inline uint64_t ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::max_line_size() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.max_line_size) + return _internal_max_line_size(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_max_line_size(uint64_t value) { + + max_line_size_ = value; +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_max_line_size(uint64_t value) { + _internal_set_max_line_size(value); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.max_line_size) +} + +// string quote = 3; +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_quote() { + quote_.ClearToEmpty(); +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::quote() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote) + return _internal_quote(); +} +template +inline PROTOBUF_ALWAYS_INLINE +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_quote(ArgT0&& arg0, ArgT... args) { + + quote_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, static_cast(arg0), args..., GetArenaForAllocation()); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote) +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::mutable_quote() { + std::string* _s = _internal_mutable_quote(); + // @@protoc_insertion_point(field_mutable:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote) + return _s; +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_quote() const { + return quote_.Get(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_quote(const std::string& value) { + + quote_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, value, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_mutable_quote() { + + return quote_.Mutable(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::release_quote() { + // @@protoc_insertion_point(field_release:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote) + return quote_.Release(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), GetArenaForAllocation()); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_allocated_quote(std::string* quote) { + if (quote != nullptr) { + + } else { + + } + quote_.SetAllocated(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), quote, + GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (quote_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + quote_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.quote) +} + +// uint64 header_lines_to_skip = 4; +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_header_lines_to_skip() { + header_lines_to_skip_ = uint64_t{0u}; +} +inline uint64_t ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_header_lines_to_skip() const { + return header_lines_to_skip_; +} +inline uint64_t ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::header_lines_to_skip() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.header_lines_to_skip) + return _internal_header_lines_to_skip(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_header_lines_to_skip(uint64_t value) { + + header_lines_to_skip_ = value; +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_header_lines_to_skip(uint64_t value) { + _internal_set_header_lines_to_skip(value); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.header_lines_to_skip) +} + +// string escape = 5; +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_escape() { + escape_.ClearToEmpty(); +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::escape() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape) + return _internal_escape(); +} +template +inline PROTOBUF_ALWAYS_INLINE +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_escape(ArgT0&& arg0, ArgT... args) { + + escape_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, static_cast(arg0), args..., GetArenaForAllocation()); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape) +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::mutable_escape() { + std::string* _s = _internal_mutable_escape(); + // @@protoc_insertion_point(field_mutable:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape) + return _s; +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_escape() const { + return escape_.Get(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_escape(const std::string& value) { + + escape_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, value, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_mutable_escape() { + + return escape_.Mutable(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::release_escape() { + // @@protoc_insertion_point(field_release:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape) + return escape_.Release(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), GetArenaForAllocation()); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_allocated_escape(std::string* escape) { + if (escape != nullptr) { + + } else { + + } + escape_.SetAllocated(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), escape, + GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (escape_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + escape_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.escape) +} + +// optional string value_treated_as_null = 6; +inline bool ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_has_value_treated_as_null() const { + bool value = (_has_bits_[0] & 0x00000001u) != 0; + return value; +} +inline bool ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::has_value_treated_as_null() const { + return _internal_has_value_treated_as_null(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::clear_value_treated_as_null() { + value_treated_as_null_.ClearToEmpty(); + _has_bits_[0] &= ~0x00000001u; +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::value_treated_as_null() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null) + return _internal_value_treated_as_null(); +} +template +inline PROTOBUF_ALWAYS_INLINE +void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_value_treated_as_null(ArgT0&& arg0, ArgT... args) { + _has_bits_[0] |= 0x00000001u; + value_treated_as_null_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, static_cast(arg0), args..., GetArenaForAllocation()); + // @@protoc_insertion_point(field_set:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null) +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::mutable_value_treated_as_null() { + std::string* _s = _internal_mutable_value_treated_as_null(); + // @@protoc_insertion_point(field_mutable:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null) + return _s; +} +inline const std::string& ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_value_treated_as_null() const { + return value_treated_as_null_.Get(); +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_set_value_treated_as_null(const std::string& value) { + _has_bits_[0] |= 0x00000001u; + value_treated_as_null_.Set(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, value, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::_internal_mutable_value_treated_as_null() { + _has_bits_[0] |= 0x00000001u; + return value_treated_as_null_.Mutable(::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr::EmptyDefault{}, GetArenaForAllocation()); +} +inline std::string* ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::release_value_treated_as_null() { + // @@protoc_insertion_point(field_release:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null) + if (!_internal_has_value_treated_as_null()) { + return nullptr; + } + _has_bits_[0] &= ~0x00000001u; + auto* p = value_treated_as_null_.ReleaseNonDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (value_treated_as_null_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + value_treated_as_null_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + return p; +} +inline void ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions::set_allocated_value_treated_as_null(std::string* value_treated_as_null) { + if (value_treated_as_null != nullptr) { + _has_bits_[0] |= 0x00000001u; + } else { + _has_bits_[0] &= ~0x00000001u; + } + value_treated_as_null_.SetAllocated(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), value_treated_as_null, + GetArenaForAllocation()); +#ifdef PROTOBUF_FORCE_COPY_DEFAULT_STRING + if (value_treated_as_null_.IsDefault(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited())) { + value_treated_as_null_.Set(&::PROTOBUF_NAMESPACE_ID::internal::GetEmptyStringAlreadyInited(), "", GetArenaForAllocation()); + } +#endif // PROTOBUF_FORCE_COPY_DEFAULT_STRING + // @@protoc_insertion_point(field_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions.value_treated_as_null) +} + +// ------------------------------------------------------------------- + // ReadRel_LocalFiles_FileOrFiles // string uri_path = 1; @@ -27616,6 +28694,80 @@ inline ::substrait::ReadRel_LocalFiles_FileOrFiles_DwrfReadOptions* ReadRel_Loca return _msg; } +// .substrait.ReadRel.LocalFiles.FileOrFiles.DelimiterSeparatedTextReadOptions text = 14; +inline bool ReadRel_LocalFiles_FileOrFiles::_internal_has_text() const { + return file_format_case() == kText; +} +inline bool ReadRel_LocalFiles_FileOrFiles::has_text() const { + return _internal_has_text(); +} +inline void ReadRel_LocalFiles_FileOrFiles::set_has_text() { + _oneof_case_[1] = kText; +} +inline void ReadRel_LocalFiles_FileOrFiles::clear_text() { + if (_internal_has_text()) { + if (GetArenaForAllocation() == nullptr) { + delete file_format_.text_; + } + clear_has_file_format(); + } +} +inline ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* ReadRel_LocalFiles_FileOrFiles::release_text() { + // @@protoc_insertion_point(field_release:substrait.ReadRel.LocalFiles.FileOrFiles.text) + if (_internal_has_text()) { + clear_has_file_format(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* temp = file_format_.text_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + file_format_.text_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& ReadRel_LocalFiles_FileOrFiles::_internal_text() const { + return _internal_has_text() + ? *file_format_.text_ + : reinterpret_cast< ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions&>(::substrait::_ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions_default_instance_); +} +inline const ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions& ReadRel_LocalFiles_FileOrFiles::text() const { + // @@protoc_insertion_point(field_get:substrait.ReadRel.LocalFiles.FileOrFiles.text) + return _internal_text(); +} +inline ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* ReadRel_LocalFiles_FileOrFiles::unsafe_arena_release_text() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.ReadRel.LocalFiles.FileOrFiles.text) + if (_internal_has_text()) { + clear_has_file_format(); + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* temp = file_format_.text_; + file_format_.text_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void ReadRel_LocalFiles_FileOrFiles::unsafe_arena_set_allocated_text(::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* text) { + clear_file_format(); + if (text) { + set_has_text(); + file_format_.text_ = text; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.ReadRel.LocalFiles.FileOrFiles.text) +} +inline ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* ReadRel_LocalFiles_FileOrFiles::_internal_mutable_text() { + if (!_internal_has_text()) { + clear_file_format(); + set_has_text(); + file_format_.text_ = CreateMaybeMessage< ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions >(GetArenaForAllocation()); + } + return file_format_.text_; +} +inline ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* ReadRel_LocalFiles_FileOrFiles::mutable_text() { + ::substrait::ReadRel_LocalFiles_FileOrFiles_DelimiterSeparatedTextReadOptions* _msg = _internal_mutable_text(); + // @@protoc_insertion_point(field_mutable:substrait.ReadRel.LocalFiles.FileOrFiles.text) + return _msg; +} + inline bool ReadRel_LocalFiles_FileOrFiles::has_path_type() const { return path_type_case() != PATH_TYPE_NOT_SET; } @@ -36254,151 +37406,151 @@ inline ::substrait::NestedLoopJoinRel* Rel::mutable_nested_loop_join() { return _msg; } -// .substrait.MarkJoinRel mark_join = 23; -inline bool Rel::_internal_has_mark_join() const { - return rel_type_case() == kMarkJoin; +// .substrait.DuplicateEliminatedGetRel duplicate_eliminated_get = 23; +inline bool Rel::_internal_has_duplicate_eliminated_get() const { + return rel_type_case() == kDuplicateEliminatedGet; } -inline bool Rel::has_mark_join() const { - return _internal_has_mark_join(); +inline bool Rel::has_duplicate_eliminated_get() const { + return _internal_has_duplicate_eliminated_get(); } -inline void Rel::set_has_mark_join() { - _oneof_case_[0] = kMarkJoin; +inline void Rel::set_has_duplicate_eliminated_get() { + _oneof_case_[0] = kDuplicateEliminatedGet; } -inline void Rel::clear_mark_join() { - if (_internal_has_mark_join()) { +inline void Rel::clear_duplicate_eliminated_get() { + if (_internal_has_duplicate_eliminated_get()) { if (GetArenaForAllocation() == nullptr) { - delete rel_type_.mark_join_; + delete rel_type_.duplicate_eliminated_get_; } clear_has_rel_type(); } } -inline ::substrait::MarkJoinRel* Rel::release_mark_join() { - // @@protoc_insertion_point(field_release:substrait.Rel.mark_join) - if (_internal_has_mark_join()) { +inline ::substrait::DuplicateEliminatedGetRel* Rel::release_duplicate_eliminated_get() { + // @@protoc_insertion_point(field_release:substrait.Rel.duplicate_eliminated_get) + if (_internal_has_duplicate_eliminated_get()) { clear_has_rel_type(); - ::substrait::MarkJoinRel* temp = rel_type_.mark_join_; + ::substrait::DuplicateEliminatedGetRel* temp = rel_type_.duplicate_eliminated_get_; if (GetArenaForAllocation() != nullptr) { temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); } - rel_type_.mark_join_ = nullptr; + rel_type_.duplicate_eliminated_get_ = nullptr; return temp; } else { return nullptr; } } -inline const ::substrait::MarkJoinRel& Rel::_internal_mark_join() const { - return _internal_has_mark_join() - ? *rel_type_.mark_join_ - : reinterpret_cast< ::substrait::MarkJoinRel&>(::substrait::_MarkJoinRel_default_instance_); +inline const ::substrait::DuplicateEliminatedGetRel& Rel::_internal_duplicate_eliminated_get() const { + return _internal_has_duplicate_eliminated_get() + ? *rel_type_.duplicate_eliminated_get_ + : reinterpret_cast< ::substrait::DuplicateEliminatedGetRel&>(::substrait::_DuplicateEliminatedGetRel_default_instance_); } -inline const ::substrait::MarkJoinRel& Rel::mark_join() const { - // @@protoc_insertion_point(field_get:substrait.Rel.mark_join) - return _internal_mark_join(); +inline const ::substrait::DuplicateEliminatedGetRel& Rel::duplicate_eliminated_get() const { + // @@protoc_insertion_point(field_get:substrait.Rel.duplicate_eliminated_get) + return _internal_duplicate_eliminated_get(); } -inline ::substrait::MarkJoinRel* Rel::unsafe_arena_release_mark_join() { - // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Rel.mark_join) - if (_internal_has_mark_join()) { +inline ::substrait::DuplicateEliminatedGetRel* Rel::unsafe_arena_release_duplicate_eliminated_get() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Rel.duplicate_eliminated_get) + if (_internal_has_duplicate_eliminated_get()) { clear_has_rel_type(); - ::substrait::MarkJoinRel* temp = rel_type_.mark_join_; - rel_type_.mark_join_ = nullptr; + ::substrait::DuplicateEliminatedGetRel* temp = rel_type_.duplicate_eliminated_get_; + rel_type_.duplicate_eliminated_get_ = nullptr; return temp; } else { return nullptr; } } -inline void Rel::unsafe_arena_set_allocated_mark_join(::substrait::MarkJoinRel* mark_join) { +inline void Rel::unsafe_arena_set_allocated_duplicate_eliminated_get(::substrait::DuplicateEliminatedGetRel* duplicate_eliminated_get) { clear_rel_type(); - if (mark_join) { - set_has_mark_join(); - rel_type_.mark_join_ = mark_join; + if (duplicate_eliminated_get) { + set_has_duplicate_eliminated_get(); + rel_type_.duplicate_eliminated_get_ = duplicate_eliminated_get; } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Rel.mark_join) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Rel.duplicate_eliminated_get) } -inline ::substrait::MarkJoinRel* Rel::_internal_mutable_mark_join() { - if (!_internal_has_mark_join()) { +inline ::substrait::DuplicateEliminatedGetRel* Rel::_internal_mutable_duplicate_eliminated_get() { + if (!_internal_has_duplicate_eliminated_get()) { clear_rel_type(); - set_has_mark_join(); - rel_type_.mark_join_ = CreateMaybeMessage< ::substrait::MarkJoinRel >(GetArenaForAllocation()); + set_has_duplicate_eliminated_get(); + rel_type_.duplicate_eliminated_get_ = CreateMaybeMessage< ::substrait::DuplicateEliminatedGetRel >(GetArenaForAllocation()); } - return rel_type_.mark_join_; + return rel_type_.duplicate_eliminated_get_; } -inline ::substrait::MarkJoinRel* Rel::mutable_mark_join() { - ::substrait::MarkJoinRel* _msg = _internal_mutable_mark_join(); - // @@protoc_insertion_point(field_mutable:substrait.Rel.mark_join) +inline ::substrait::DuplicateEliminatedGetRel* Rel::mutable_duplicate_eliminated_get() { + ::substrait::DuplicateEliminatedGetRel* _msg = _internal_mutable_duplicate_eliminated_get(); + // @@protoc_insertion_point(field_mutable:substrait.Rel.duplicate_eliminated_get) return _msg; } -// .substrait.DelimiterJoinRel delimiter_join = 24; -inline bool Rel::_internal_has_delimiter_join() const { - return rel_type_case() == kDelimiterJoin; +// .substrait.DuplicateEliminatedJoinRel duplicate_eliminated_join = 24; +inline bool Rel::_internal_has_duplicate_eliminated_join() const { + return rel_type_case() == kDuplicateEliminatedJoin; } -inline bool Rel::has_delimiter_join() const { - return _internal_has_delimiter_join(); +inline bool Rel::has_duplicate_eliminated_join() const { + return _internal_has_duplicate_eliminated_join(); } -inline void Rel::set_has_delimiter_join() { - _oneof_case_[0] = kDelimiterJoin; +inline void Rel::set_has_duplicate_eliminated_join() { + _oneof_case_[0] = kDuplicateEliminatedJoin; } -inline void Rel::clear_delimiter_join() { - if (_internal_has_delimiter_join()) { +inline void Rel::clear_duplicate_eliminated_join() { + if (_internal_has_duplicate_eliminated_join()) { if (GetArenaForAllocation() == nullptr) { - delete rel_type_.delimiter_join_; + delete rel_type_.duplicate_eliminated_join_; } clear_has_rel_type(); } } -inline ::substrait::DelimiterJoinRel* Rel::release_delimiter_join() { - // @@protoc_insertion_point(field_release:substrait.Rel.delimiter_join) - if (_internal_has_delimiter_join()) { +inline ::substrait::DuplicateEliminatedJoinRel* Rel::release_duplicate_eliminated_join() { + // @@protoc_insertion_point(field_release:substrait.Rel.duplicate_eliminated_join) + if (_internal_has_duplicate_eliminated_join()) { clear_has_rel_type(); - ::substrait::DelimiterJoinRel* temp = rel_type_.delimiter_join_; + ::substrait::DuplicateEliminatedJoinRel* temp = rel_type_.duplicate_eliminated_join_; if (GetArenaForAllocation() != nullptr) { temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); } - rel_type_.delimiter_join_ = nullptr; + rel_type_.duplicate_eliminated_join_ = nullptr; return temp; } else { return nullptr; } } -inline const ::substrait::DelimiterJoinRel& Rel::_internal_delimiter_join() const { - return _internal_has_delimiter_join() - ? *rel_type_.delimiter_join_ - : reinterpret_cast< ::substrait::DelimiterJoinRel&>(::substrait::_DelimiterJoinRel_default_instance_); +inline const ::substrait::DuplicateEliminatedJoinRel& Rel::_internal_duplicate_eliminated_join() const { + return _internal_has_duplicate_eliminated_join() + ? *rel_type_.duplicate_eliminated_join_ + : reinterpret_cast< ::substrait::DuplicateEliminatedJoinRel&>(::substrait::_DuplicateEliminatedJoinRel_default_instance_); } -inline const ::substrait::DelimiterJoinRel& Rel::delimiter_join() const { - // @@protoc_insertion_point(field_get:substrait.Rel.delimiter_join) - return _internal_delimiter_join(); +inline const ::substrait::DuplicateEliminatedJoinRel& Rel::duplicate_eliminated_join() const { + // @@protoc_insertion_point(field_get:substrait.Rel.duplicate_eliminated_join) + return _internal_duplicate_eliminated_join(); } -inline ::substrait::DelimiterJoinRel* Rel::unsafe_arena_release_delimiter_join() { - // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Rel.delimiter_join) - if (_internal_has_delimiter_join()) { +inline ::substrait::DuplicateEliminatedJoinRel* Rel::unsafe_arena_release_duplicate_eliminated_join() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Rel.duplicate_eliminated_join) + if (_internal_has_duplicate_eliminated_join()) { clear_has_rel_type(); - ::substrait::DelimiterJoinRel* temp = rel_type_.delimiter_join_; - rel_type_.delimiter_join_ = nullptr; + ::substrait::DuplicateEliminatedJoinRel* temp = rel_type_.duplicate_eliminated_join_; + rel_type_.duplicate_eliminated_join_ = nullptr; return temp; } else { return nullptr; } } -inline void Rel::unsafe_arena_set_allocated_delimiter_join(::substrait::DelimiterJoinRel* delimiter_join) { +inline void Rel::unsafe_arena_set_allocated_duplicate_eliminated_join(::substrait::DuplicateEliminatedJoinRel* duplicate_eliminated_join) { clear_rel_type(); - if (delimiter_join) { - set_has_delimiter_join(); - rel_type_.delimiter_join_ = delimiter_join; + if (duplicate_eliminated_join) { + set_has_duplicate_eliminated_join(); + rel_type_.duplicate_eliminated_join_ = duplicate_eliminated_join; } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Rel.delimiter_join) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Rel.duplicate_eliminated_join) } -inline ::substrait::DelimiterJoinRel* Rel::_internal_mutable_delimiter_join() { - if (!_internal_has_delimiter_join()) { +inline ::substrait::DuplicateEliminatedJoinRel* Rel::_internal_mutable_duplicate_eliminated_join() { + if (!_internal_has_duplicate_eliminated_join()) { clear_rel_type(); - set_has_delimiter_join(); - rel_type_.delimiter_join_ = CreateMaybeMessage< ::substrait::DelimiterJoinRel >(GetArenaForAllocation()); + set_has_duplicate_eliminated_join(); + rel_type_.duplicate_eliminated_join_ = CreateMaybeMessage< ::substrait::DuplicateEliminatedJoinRel >(GetArenaForAllocation()); } - return rel_type_.delimiter_join_; + return rel_type_.duplicate_eliminated_join_; } -inline ::substrait::DelimiterJoinRel* Rel::mutable_delimiter_join() { - ::substrait::DelimiterJoinRel* _msg = _internal_mutable_delimiter_join(); - // @@protoc_insertion_point(field_mutable:substrait.Rel.delimiter_join) +inline ::substrait::DuplicateEliminatedJoinRel* Rel::mutable_duplicate_eliminated_join() { + ::substrait::DuplicateEliminatedJoinRel* _msg = _internal_mutable_duplicate_eliminated_join(); + // @@protoc_insertion_point(field_mutable:substrait.Rel.duplicate_eliminated_join) return _msg; } @@ -39927,31 +41079,31 @@ inline void NestedLoopJoinRel::set_allocated_advanced_extension(::substrait::ext // ------------------------------------------------------------------- -// MarkJoinRel +// DuplicateEliminatedGetRel // .substrait.RelCommon common = 1; -inline bool MarkJoinRel::_internal_has_common() const { +inline bool DuplicateEliminatedGetRel::_internal_has_common() const { return this != internal_default_instance() && common_ != nullptr; } -inline bool MarkJoinRel::has_common() const { +inline bool DuplicateEliminatedGetRel::has_common() const { return _internal_has_common(); } -inline void MarkJoinRel::clear_common() { +inline void DuplicateEliminatedGetRel::clear_common() { if (GetArenaForAllocation() == nullptr && common_ != nullptr) { delete common_; } common_ = nullptr; } -inline const ::substrait::RelCommon& MarkJoinRel::_internal_common() const { +inline const ::substrait::RelCommon& DuplicateEliminatedGetRel::_internal_common() const { const ::substrait::RelCommon* p = common_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_RelCommon_default_instance_); } -inline const ::substrait::RelCommon& MarkJoinRel::common() const { - // @@protoc_insertion_point(field_get:substrait.MarkJoinRel.common) +inline const ::substrait::RelCommon& DuplicateEliminatedGetRel::common() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedGetRel.common) return _internal_common(); } -inline void MarkJoinRel::unsafe_arena_set_allocated_common( +inline void DuplicateEliminatedGetRel::unsafe_arena_set_allocated_common( ::substrait::RelCommon* common) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(common_); @@ -39962,9 +41114,9 @@ inline void MarkJoinRel::unsafe_arena_set_allocated_common( } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.MarkJoinRel.common) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedGetRel.common) } -inline ::substrait::RelCommon* MarkJoinRel::release_common() { +inline ::substrait::RelCommon* DuplicateEliminatedGetRel::release_common() { ::substrait::RelCommon* temp = common_; common_ = nullptr; @@ -39979,14 +41131,14 @@ inline ::substrait::RelCommon* MarkJoinRel::release_common() { #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::RelCommon* MarkJoinRel::unsafe_arena_release_common() { - // @@protoc_insertion_point(field_release:substrait.MarkJoinRel.common) +inline ::substrait::RelCommon* DuplicateEliminatedGetRel::unsafe_arena_release_common() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedGetRel.common) ::substrait::RelCommon* temp = common_; common_ = nullptr; return temp; } -inline ::substrait::RelCommon* MarkJoinRel::_internal_mutable_common() { +inline ::substrait::RelCommon* DuplicateEliminatedGetRel::_internal_mutable_common() { if (common_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::RelCommon>(GetArenaForAllocation()); @@ -39994,12 +41146,12 @@ inline ::substrait::RelCommon* MarkJoinRel::_internal_mutable_common() { } return common_; } -inline ::substrait::RelCommon* MarkJoinRel::mutable_common() { +inline ::substrait::RelCommon* DuplicateEliminatedGetRel::mutable_common() { ::substrait::RelCommon* _msg = _internal_mutable_common(); - // @@protoc_insertion_point(field_mutable:substrait.MarkJoinRel.common) + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedGetRel.common) return _msg; } -inline void MarkJoinRel::set_allocated_common(::substrait::RelCommon* common) { +inline void DuplicateEliminatedGetRel::set_allocated_common(::substrait::RelCommon* common) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete common_; @@ -40016,312 +41168,48 @@ inline void MarkJoinRel::set_allocated_common(::substrait::RelCommon* common) { } common_ = common; - // @@protoc_insertion_point(field_set_allocated:substrait.MarkJoinRel.common) + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedGetRel.common) } -// .substrait.Rel left = 2; -inline bool MarkJoinRel::_internal_has_left() const { - return this != internal_default_instance() && left_ != nullptr; -} -inline bool MarkJoinRel::has_left() const { - return _internal_has_left(); -} -inline void MarkJoinRel::clear_left() { - if (GetArenaForAllocation() == nullptr && left_ != nullptr) { - delete left_; - } - left_ = nullptr; -} -inline const ::substrait::Rel& MarkJoinRel::_internal_left() const { - const ::substrait::Rel* p = left_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::_Rel_default_instance_); -} -inline const ::substrait::Rel& MarkJoinRel::left() const { - // @@protoc_insertion_point(field_get:substrait.MarkJoinRel.left) - return _internal_left(); -} -inline void MarkJoinRel::unsafe_arena_set_allocated_left( - ::substrait::Rel* left) { - if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(left_); - } - left_ = left; - if (left) { - - } else { - - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.MarkJoinRel.left) -} -inline ::substrait::Rel* MarkJoinRel::release_left() { - - ::substrait::Rel* temp = left_; - left_ = nullptr; -#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE - auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - if (GetArenaForAllocation() == nullptr) { delete old; } -#else // PROTOBUF_FORCE_COPY_IN_RELEASE - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } -#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE - return temp; -} -inline ::substrait::Rel* MarkJoinRel::unsafe_arena_release_left() { - // @@protoc_insertion_point(field_release:substrait.MarkJoinRel.left) - - ::substrait::Rel* temp = left_; - left_ = nullptr; - return temp; -} -inline ::substrait::Rel* MarkJoinRel::_internal_mutable_left() { - - if (left_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::Rel>(GetArenaForAllocation()); - left_ = p; - } - return left_; -} -inline ::substrait::Rel* MarkJoinRel::mutable_left() { - ::substrait::Rel* _msg = _internal_mutable_left(); - // @@protoc_insertion_point(field_mutable:substrait.MarkJoinRel.left) - return _msg; -} -inline void MarkJoinRel::set_allocated_left(::substrait::Rel* left) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - if (message_arena == nullptr) { - delete left_; - } - if (left) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Rel>::GetOwningArena(left); - if (message_arena != submessage_arena) { - left = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, left, submessage_arena); - } - - } else { - - } - left_ = left; - // @@protoc_insertion_point(field_set_allocated:substrait.MarkJoinRel.left) -} - -// .substrait.Rel right = 3; -inline bool MarkJoinRel::_internal_has_right() const { - return this != internal_default_instance() && right_ != nullptr; -} -inline bool MarkJoinRel::has_right() const { - return _internal_has_right(); -} -inline void MarkJoinRel::clear_right() { - if (GetArenaForAllocation() == nullptr && right_ != nullptr) { - delete right_; - } - right_ = nullptr; -} -inline const ::substrait::Rel& MarkJoinRel::_internal_right() const { - const ::substrait::Rel* p = right_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::_Rel_default_instance_); -} -inline const ::substrait::Rel& MarkJoinRel::right() const { - // @@protoc_insertion_point(field_get:substrait.MarkJoinRel.right) - return _internal_right(); -} -inline void MarkJoinRel::unsafe_arena_set_allocated_right( - ::substrait::Rel* right) { - if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(right_); - } - right_ = right; - if (right) { - - } else { - - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.MarkJoinRel.right) -} -inline ::substrait::Rel* MarkJoinRel::release_right() { - - ::substrait::Rel* temp = right_; - right_ = nullptr; -#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE - auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - if (GetArenaForAllocation() == nullptr) { delete old; } -#else // PROTOBUF_FORCE_COPY_IN_RELEASE - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } -#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE - return temp; -} -inline ::substrait::Rel* MarkJoinRel::unsafe_arena_release_right() { - // @@protoc_insertion_point(field_release:substrait.MarkJoinRel.right) - - ::substrait::Rel* temp = right_; - right_ = nullptr; - return temp; -} -inline ::substrait::Rel* MarkJoinRel::_internal_mutable_right() { - - if (right_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::Rel>(GetArenaForAllocation()); - right_ = p; - } - return right_; -} -inline ::substrait::Rel* MarkJoinRel::mutable_right() { - ::substrait::Rel* _msg = _internal_mutable_right(); - // @@protoc_insertion_point(field_mutable:substrait.MarkJoinRel.right) - return _msg; -} -inline void MarkJoinRel::set_allocated_right(::substrait::Rel* right) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - if (message_arena == nullptr) { - delete right_; - } - if (right) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Rel>::GetOwningArena(right); - if (message_arena != submessage_arena) { - right = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, right, submessage_arena); - } - - } else { - - } - right_ = right; - // @@protoc_insertion_point(field_set_allocated:substrait.MarkJoinRel.right) -} - -// .substrait.Expression expression = 4; -inline bool MarkJoinRel::_internal_has_expression() const { - return this != internal_default_instance() && expression_ != nullptr; -} -inline bool MarkJoinRel::has_expression() const { - return _internal_has_expression(); -} -inline void MarkJoinRel::clear_expression() { - if (GetArenaForAllocation() == nullptr && expression_ != nullptr) { - delete expression_; - } - expression_ = nullptr; -} -inline const ::substrait::Expression& MarkJoinRel::_internal_expression() const { - const ::substrait::Expression* p = expression_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::_Expression_default_instance_); -} -inline const ::substrait::Expression& MarkJoinRel::expression() const { - // @@protoc_insertion_point(field_get:substrait.MarkJoinRel.expression) - return _internal_expression(); -} -inline void MarkJoinRel::unsafe_arena_set_allocated_expression( - ::substrait::Expression* expression) { - if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(expression_); - } - expression_ = expression; - if (expression) { - - } else { - - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.MarkJoinRel.expression) -} -inline ::substrait::Expression* MarkJoinRel::release_expression() { - - ::substrait::Expression* temp = expression_; - expression_ = nullptr; -#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE - auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - if (GetArenaForAllocation() == nullptr) { delete old; } -#else // PROTOBUF_FORCE_COPY_IN_RELEASE - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } -#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE - return temp; -} -inline ::substrait::Expression* MarkJoinRel::unsafe_arena_release_expression() { - // @@protoc_insertion_point(field_release:substrait.MarkJoinRel.expression) - - ::substrait::Expression* temp = expression_; - expression_ = nullptr; - return temp; -} -inline ::substrait::Expression* MarkJoinRel::_internal_mutable_expression() { - - if (expression_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::Expression>(GetArenaForAllocation()); - expression_ = p; - } - return expression_; +// .substrait.ReferenceRel input = 2; +inline bool DuplicateEliminatedGetRel::_internal_has_input() const { + return this != internal_default_instance() && input_ != nullptr; } -inline ::substrait::Expression* MarkJoinRel::mutable_expression() { - ::substrait::Expression* _msg = _internal_mutable_expression(); - // @@protoc_insertion_point(field_mutable:substrait.MarkJoinRel.expression) - return _msg; +inline bool DuplicateEliminatedGetRel::has_input() const { + return _internal_has_input(); } -inline void MarkJoinRel::set_allocated_expression(::substrait::Expression* expression) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - if (message_arena == nullptr) { - delete expression_; - } - if (expression) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression>::GetOwningArena(expression); - if (message_arena != submessage_arena) { - expression = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, expression, submessage_arena); - } - - } else { - +inline void DuplicateEliminatedGetRel::clear_input() { + if (GetArenaForAllocation() == nullptr && input_ != nullptr) { + delete input_; } - expression_ = expression; - // @@protoc_insertion_point(field_set_allocated:substrait.MarkJoinRel.expression) -} - -// .substrait.extensions.AdvancedExtension advanced_extension = 10; -inline bool MarkJoinRel::_internal_has_advanced_extension() const { - return this != internal_default_instance() && advanced_extension_ != nullptr; -} -inline bool MarkJoinRel::has_advanced_extension() const { - return _internal_has_advanced_extension(); + input_ = nullptr; } -inline const ::substrait::extensions::AdvancedExtension& MarkJoinRel::_internal_advanced_extension() const { - const ::substrait::extensions::AdvancedExtension* p = advanced_extension_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::extensions::_AdvancedExtension_default_instance_); +inline const ::substrait::ReferenceRel& DuplicateEliminatedGetRel::_internal_input() const { + const ::substrait::ReferenceRel* p = input_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_ReferenceRel_default_instance_); } -inline const ::substrait::extensions::AdvancedExtension& MarkJoinRel::advanced_extension() const { - // @@protoc_insertion_point(field_get:substrait.MarkJoinRel.advanced_extension) - return _internal_advanced_extension(); +inline const ::substrait::ReferenceRel& DuplicateEliminatedGetRel::input() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedGetRel.input) + return _internal_input(); } -inline void MarkJoinRel::unsafe_arena_set_allocated_advanced_extension( - ::substrait::extensions::AdvancedExtension* advanced_extension) { +inline void DuplicateEliminatedGetRel::unsafe_arena_set_allocated_input( + ::substrait::ReferenceRel* input) { if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(advanced_extension_); + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(input_); } - advanced_extension_ = advanced_extension; - if (advanced_extension) { + input_ = input; + if (input) { } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.MarkJoinRel.advanced_extension) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedGetRel.input) } -inline ::substrait::extensions::AdvancedExtension* MarkJoinRel::release_advanced_extension() { +inline ::substrait::ReferenceRel* DuplicateEliminatedGetRel::release_input() { - ::substrait::extensions::AdvancedExtension* temp = advanced_extension_; - advanced_extension_ = nullptr; + ::substrait::ReferenceRel* temp = input_; + input_ = nullptr; #ifdef PROTOBUF_FORCE_COPY_IN_RELEASE auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); @@ -40333,75 +41221,73 @@ inline ::substrait::extensions::AdvancedExtension* MarkJoinRel::release_advanced #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::extensions::AdvancedExtension* MarkJoinRel::unsafe_arena_release_advanced_extension() { - // @@protoc_insertion_point(field_release:substrait.MarkJoinRel.advanced_extension) +inline ::substrait::ReferenceRel* DuplicateEliminatedGetRel::unsafe_arena_release_input() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedGetRel.input) - ::substrait::extensions::AdvancedExtension* temp = advanced_extension_; - advanced_extension_ = nullptr; + ::substrait::ReferenceRel* temp = input_; + input_ = nullptr; return temp; } -inline ::substrait::extensions::AdvancedExtension* MarkJoinRel::_internal_mutable_advanced_extension() { +inline ::substrait::ReferenceRel* DuplicateEliminatedGetRel::_internal_mutable_input() { - if (advanced_extension_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::extensions::AdvancedExtension>(GetArenaForAllocation()); - advanced_extension_ = p; + if (input_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::ReferenceRel>(GetArenaForAllocation()); + input_ = p; } - return advanced_extension_; + return input_; } -inline ::substrait::extensions::AdvancedExtension* MarkJoinRel::mutable_advanced_extension() { - ::substrait::extensions::AdvancedExtension* _msg = _internal_mutable_advanced_extension(); - // @@protoc_insertion_point(field_mutable:substrait.MarkJoinRel.advanced_extension) +inline ::substrait::ReferenceRel* DuplicateEliminatedGetRel::mutable_input() { + ::substrait::ReferenceRel* _msg = _internal_mutable_input(); + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedGetRel.input) return _msg; } -inline void MarkJoinRel::set_allocated_advanced_extension(::substrait::extensions::AdvancedExtension* advanced_extension) { +inline void DuplicateEliminatedGetRel::set_allocated_input(::substrait::ReferenceRel* input) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { - delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(advanced_extension_); + delete input_; } - if (advanced_extension) { + if (input) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper< - ::PROTOBUF_NAMESPACE_ID::MessageLite>::GetOwningArena( - reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(advanced_extension)); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ReferenceRel>::GetOwningArena(input); if (message_arena != submessage_arena) { - advanced_extension = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, advanced_extension, submessage_arena); + input = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, input, submessage_arena); } } else { } - advanced_extension_ = advanced_extension; - // @@protoc_insertion_point(field_set_allocated:substrait.MarkJoinRel.advanced_extension) + input_ = input; + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedGetRel.input) } // ------------------------------------------------------------------- -// DelimiterJoinRel +// DuplicateEliminatedJoinRel // .substrait.RelCommon common = 1; -inline bool DelimiterJoinRel::_internal_has_common() const { +inline bool DuplicateEliminatedJoinRel::_internal_has_common() const { return this != internal_default_instance() && common_ != nullptr; } -inline bool DelimiterJoinRel::has_common() const { +inline bool DuplicateEliminatedJoinRel::has_common() const { return _internal_has_common(); } -inline void DelimiterJoinRel::clear_common() { +inline void DuplicateEliminatedJoinRel::clear_common() { if (GetArenaForAllocation() == nullptr && common_ != nullptr) { delete common_; } common_ = nullptr; } -inline const ::substrait::RelCommon& DelimiterJoinRel::_internal_common() const { +inline const ::substrait::RelCommon& DuplicateEliminatedJoinRel::_internal_common() const { const ::substrait::RelCommon* p = common_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_RelCommon_default_instance_); } -inline const ::substrait::RelCommon& DelimiterJoinRel::common() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.common) +inline const ::substrait::RelCommon& DuplicateEliminatedJoinRel::common() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.common) return _internal_common(); } -inline void DelimiterJoinRel::unsafe_arena_set_allocated_common( +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_common( ::substrait::RelCommon* common) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(common_); @@ -40412,9 +41298,9 @@ inline void DelimiterJoinRel::unsafe_arena_set_allocated_common( } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DelimiterJoinRel.common) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.common) } -inline ::substrait::RelCommon* DelimiterJoinRel::release_common() { +inline ::substrait::RelCommon* DuplicateEliminatedJoinRel::release_common() { ::substrait::RelCommon* temp = common_; common_ = nullptr; @@ -40429,14 +41315,14 @@ inline ::substrait::RelCommon* DelimiterJoinRel::release_common() { #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::RelCommon* DelimiterJoinRel::unsafe_arena_release_common() { - // @@protoc_insertion_point(field_release:substrait.DelimiterJoinRel.common) +inline ::substrait::RelCommon* DuplicateEliminatedJoinRel::unsafe_arena_release_common() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.common) ::substrait::RelCommon* temp = common_; common_ = nullptr; return temp; } -inline ::substrait::RelCommon* DelimiterJoinRel::_internal_mutable_common() { +inline ::substrait::RelCommon* DuplicateEliminatedJoinRel::_internal_mutable_common() { if (common_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::RelCommon>(GetArenaForAllocation()); @@ -40444,12 +41330,12 @@ inline ::substrait::RelCommon* DelimiterJoinRel::_internal_mutable_common() { } return common_; } -inline ::substrait::RelCommon* DelimiterJoinRel::mutable_common() { +inline ::substrait::RelCommon* DuplicateEliminatedJoinRel::mutable_common() { ::substrait::RelCommon* _msg = _internal_mutable_common(); - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.common) + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.common) return _msg; } -inline void DelimiterJoinRel::set_allocated_common(::substrait::RelCommon* common) { +inline void DuplicateEliminatedJoinRel::set_allocated_common(::substrait::RelCommon* common) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete common_; @@ -40466,32 +41352,32 @@ inline void DelimiterJoinRel::set_allocated_common(::substrait::RelCommon* commo } common_ = common; - // @@protoc_insertion_point(field_set_allocated:substrait.DelimiterJoinRel.common) + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.common) } // .substrait.Rel left = 2; -inline bool DelimiterJoinRel::_internal_has_left() const { +inline bool DuplicateEliminatedJoinRel::_internal_has_left() const { return this != internal_default_instance() && left_ != nullptr; } -inline bool DelimiterJoinRel::has_left() const { +inline bool DuplicateEliminatedJoinRel::has_left() const { return _internal_has_left(); } -inline void DelimiterJoinRel::clear_left() { +inline void DuplicateEliminatedJoinRel::clear_left() { if (GetArenaForAllocation() == nullptr && left_ != nullptr) { delete left_; } left_ = nullptr; } -inline const ::substrait::Rel& DelimiterJoinRel::_internal_left() const { +inline const ::substrait::Rel& DuplicateEliminatedJoinRel::_internal_left() const { const ::substrait::Rel* p = left_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_Rel_default_instance_); } -inline const ::substrait::Rel& DelimiterJoinRel::left() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.left) +inline const ::substrait::Rel& DuplicateEliminatedJoinRel::left() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.left) return _internal_left(); } -inline void DelimiterJoinRel::unsafe_arena_set_allocated_left( +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_left( ::substrait::Rel* left) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(left_); @@ -40502,9 +41388,9 @@ inline void DelimiterJoinRel::unsafe_arena_set_allocated_left( } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DelimiterJoinRel.left) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.left) } -inline ::substrait::Rel* DelimiterJoinRel::release_left() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::release_left() { ::substrait::Rel* temp = left_; left_ = nullptr; @@ -40519,14 +41405,14 @@ inline ::substrait::Rel* DelimiterJoinRel::release_left() { #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::Rel* DelimiterJoinRel::unsafe_arena_release_left() { - // @@protoc_insertion_point(field_release:substrait.DelimiterJoinRel.left) +inline ::substrait::Rel* DuplicateEliminatedJoinRel::unsafe_arena_release_left() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.left) ::substrait::Rel* temp = left_; left_ = nullptr; return temp; } -inline ::substrait::Rel* DelimiterJoinRel::_internal_mutable_left() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::_internal_mutable_left() { if (left_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::Rel>(GetArenaForAllocation()); @@ -40534,12 +41420,12 @@ inline ::substrait::Rel* DelimiterJoinRel::_internal_mutable_left() { } return left_; } -inline ::substrait::Rel* DelimiterJoinRel::mutable_left() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::mutable_left() { ::substrait::Rel* _msg = _internal_mutable_left(); - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.left) + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.left) return _msg; } -inline void DelimiterJoinRel::set_allocated_left(::substrait::Rel* left) { +inline void DuplicateEliminatedJoinRel::set_allocated_left(::substrait::Rel* left) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete left_; @@ -40556,32 +41442,32 @@ inline void DelimiterJoinRel::set_allocated_left(::substrait::Rel* left) { } left_ = left; - // @@protoc_insertion_point(field_set_allocated:substrait.DelimiterJoinRel.left) + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.left) } // .substrait.Rel right = 3; -inline bool DelimiterJoinRel::_internal_has_right() const { +inline bool DuplicateEliminatedJoinRel::_internal_has_right() const { return this != internal_default_instance() && right_ != nullptr; } -inline bool DelimiterJoinRel::has_right() const { +inline bool DuplicateEliminatedJoinRel::has_right() const { return _internal_has_right(); } -inline void DelimiterJoinRel::clear_right() { +inline void DuplicateEliminatedJoinRel::clear_right() { if (GetArenaForAllocation() == nullptr && right_ != nullptr) { delete right_; } right_ = nullptr; } -inline const ::substrait::Rel& DelimiterJoinRel::_internal_right() const { +inline const ::substrait::Rel& DuplicateEliminatedJoinRel::_internal_right() const { const ::substrait::Rel* p = right_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_Rel_default_instance_); } -inline const ::substrait::Rel& DelimiterJoinRel::right() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.right) +inline const ::substrait::Rel& DuplicateEliminatedJoinRel::right() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.right) return _internal_right(); } -inline void DelimiterJoinRel::unsafe_arena_set_allocated_right( +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_right( ::substrait::Rel* right) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(right_); @@ -40592,9 +41478,9 @@ inline void DelimiterJoinRel::unsafe_arena_set_allocated_right( } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DelimiterJoinRel.right) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.right) } -inline ::substrait::Rel* DelimiterJoinRel::release_right() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::release_right() { ::substrait::Rel* temp = right_; right_ = nullptr; @@ -40609,14 +41495,14 @@ inline ::substrait::Rel* DelimiterJoinRel::release_right() { #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::Rel* DelimiterJoinRel::unsafe_arena_release_right() { - // @@protoc_insertion_point(field_release:substrait.DelimiterJoinRel.right) +inline ::substrait::Rel* DuplicateEliminatedJoinRel::unsafe_arena_release_right() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.right) ::substrait::Rel* temp = right_; right_ = nullptr; return temp; } -inline ::substrait::Rel* DelimiterJoinRel::_internal_mutable_right() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::_internal_mutable_right() { if (right_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::Rel>(GetArenaForAllocation()); @@ -40624,12 +41510,12 @@ inline ::substrait::Rel* DelimiterJoinRel::_internal_mutable_right() { } return right_; } -inline ::substrait::Rel* DelimiterJoinRel::mutable_right() { +inline ::substrait::Rel* DuplicateEliminatedJoinRel::mutable_right() { ::substrait::Rel* _msg = _internal_mutable_right(); - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.right) + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.right) return _msg; } -inline void DelimiterJoinRel::set_allocated_right(::substrait::Rel* right) { +inline void DuplicateEliminatedJoinRel::set_allocated_right(::substrait::Rel* right) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete right_; @@ -40646,148 +41532,138 @@ inline void DelimiterJoinRel::set_allocated_right(::substrait::Rel* right) { } right_ = right; - // @@protoc_insertion_point(field_set_allocated:substrait.DelimiterJoinRel.right) -} - -// repeated .substrait.Expression.FieldReference left_keys = 4; -inline int DelimiterJoinRel::_internal_left_keys_size() const { - return left_keys_.size(); -} -inline int DelimiterJoinRel::left_keys_size() const { - return _internal_left_keys_size(); -} -inline void DelimiterJoinRel::clear_left_keys() { - left_keys_.Clear(); -} -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::mutable_left_keys(int index) { - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.left_keys) - return left_keys_.Mutable(index); -} -inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* -DelimiterJoinRel::mutable_left_keys() { - // @@protoc_insertion_point(field_mutable_list:substrait.DelimiterJoinRel.left_keys) - return &left_keys_; -} -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::_internal_left_keys(int index) const { - return left_keys_.Get(index); -} -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::left_keys(int index) const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.left_keys) - return _internal_left_keys(index); -} -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::_internal_add_left_keys() { - return left_keys_.Add(); -} -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::add_left_keys() { - ::substrait::Expression_FieldReference* _add = _internal_add_left_keys(); - // @@protoc_insertion_point(field_add:substrait.DelimiterJoinRel.left_keys) - return _add; -} -inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& -DelimiterJoinRel::left_keys() const { - // @@protoc_insertion_point(field_list:substrait.DelimiterJoinRel.left_keys) - return left_keys_; + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.right) } -// repeated .substrait.Expression.FieldReference right_keys = 5; -inline int DelimiterJoinRel::_internal_right_keys_size() const { - return right_keys_.size(); -} -inline int DelimiterJoinRel::right_keys_size() const { - return _internal_right_keys_size(); -} -inline void DelimiterJoinRel::clear_right_keys() { - right_keys_.Clear(); -} -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::mutable_right_keys(int index) { - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.right_keys) - return right_keys_.Mutable(index); -} -inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* -DelimiterJoinRel::mutable_right_keys() { - // @@protoc_insertion_point(field_mutable_list:substrait.DelimiterJoinRel.right_keys) - return &right_keys_; -} -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::_internal_right_keys(int index) const { - return right_keys_.Get(index); +// .substrait.Expression expression = 4; +inline bool DuplicateEliminatedJoinRel::_internal_has_expression() const { + return this != internal_default_instance() && expression_ != nullptr; } -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::right_keys(int index) const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.right_keys) - return _internal_right_keys(index); +inline bool DuplicateEliminatedJoinRel::has_expression() const { + return _internal_has_expression(); } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::_internal_add_right_keys() { - return right_keys_.Add(); +inline void DuplicateEliminatedJoinRel::clear_expression() { + if (GetArenaForAllocation() == nullptr && expression_ != nullptr) { + delete expression_; + } + expression_ = nullptr; } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::add_right_keys() { - ::substrait::Expression_FieldReference* _add = _internal_add_right_keys(); - // @@protoc_insertion_point(field_add:substrait.DelimiterJoinRel.right_keys) - return _add; +inline const ::substrait::Expression& DuplicateEliminatedJoinRel::_internal_expression() const { + const ::substrait::Expression* p = expression_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_Expression_default_instance_); } -inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& -DelimiterJoinRel::right_keys() const { - // @@protoc_insertion_point(field_list:substrait.DelimiterJoinRel.right_keys) - return right_keys_; +inline const ::substrait::Expression& DuplicateEliminatedJoinRel::expression() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.expression) + return _internal_expression(); } - -// .substrait.DelimiterJoinRel.JoinType type = 6; -inline void DelimiterJoinRel::clear_type() { - type_ = 0; +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_expression( + ::substrait::Expression* expression) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(expression_); + } + expression_ = expression; + if (expression) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.expression) } -inline ::substrait::DelimiterJoinRel_JoinType DelimiterJoinRel::_internal_type() const { - return static_cast< ::substrait::DelimiterJoinRel_JoinType >(type_); +inline ::substrait::Expression* DuplicateEliminatedJoinRel::release_expression() { + + ::substrait::Expression* temp = expression_; + expression_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; } -inline ::substrait::DelimiterJoinRel_JoinType DelimiterJoinRel::type() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.type) - return _internal_type(); +inline ::substrait::Expression* DuplicateEliminatedJoinRel::unsafe_arena_release_expression() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.expression) + + ::substrait::Expression* temp = expression_; + expression_ = nullptr; + return temp; } -inline void DelimiterJoinRel::_internal_set_type(::substrait::DelimiterJoinRel_JoinType value) { +inline ::substrait::Expression* DuplicateEliminatedJoinRel::_internal_mutable_expression() { - type_ = value; + if (expression_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::Expression>(GetArenaForAllocation()); + expression_ = p; + } + return expression_; } -inline void DelimiterJoinRel::set_type(::substrait::DelimiterJoinRel_JoinType value) { - _internal_set_type(value); - // @@protoc_insertion_point(field_set:substrait.DelimiterJoinRel.type) +inline ::substrait::Expression* DuplicateEliminatedJoinRel::mutable_expression() { + ::substrait::Expression* _msg = _internal_mutable_expression(); + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.expression) + return _msg; +} +inline void DuplicateEliminatedJoinRel::set_allocated_expression(::substrait::Expression* expression) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete expression_; + } + if (expression) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression>::GetOwningArena(expression); + if (message_arena != submessage_arena) { + expression = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, expression, submessage_arena); + } + + } else { + + } + expression_ = expression; + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.expression) } -// .substrait.Expression.FieldReference delimiter_field = 7; -inline bool DelimiterJoinRel::_internal_has_delimiter_field() const { - return this != internal_default_instance() && delimiter_field_ != nullptr; +// .substrait.Expression post_join_filter = 5; +inline bool DuplicateEliminatedJoinRel::_internal_has_post_join_filter() const { + return this != internal_default_instance() && post_join_filter_ != nullptr; } -inline bool DelimiterJoinRel::has_delimiter_field() const { - return _internal_has_delimiter_field(); +inline bool DuplicateEliminatedJoinRel::has_post_join_filter() const { + return _internal_has_post_join_filter(); } -inline void DelimiterJoinRel::clear_delimiter_field() { - if (GetArenaForAllocation() == nullptr && delimiter_field_ != nullptr) { - delete delimiter_field_; +inline void DuplicateEliminatedJoinRel::clear_post_join_filter() { + if (GetArenaForAllocation() == nullptr && post_join_filter_ != nullptr) { + delete post_join_filter_; } - delimiter_field_ = nullptr; + post_join_filter_ = nullptr; } -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::_internal_delimiter_field() const { - const ::substrait::Expression_FieldReference* p = delimiter_field_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::_Expression_FieldReference_default_instance_); +inline const ::substrait::Expression& DuplicateEliminatedJoinRel::_internal_post_join_filter() const { + const ::substrait::Expression* p = post_join_filter_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_Expression_default_instance_); } -inline const ::substrait::Expression_FieldReference& DelimiterJoinRel::delimiter_field() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.delimiter_field) - return _internal_delimiter_field(); +inline const ::substrait::Expression& DuplicateEliminatedJoinRel::post_join_filter() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.post_join_filter) + return _internal_post_join_filter(); } -inline void DelimiterJoinRel::unsafe_arena_set_allocated_delimiter_field( - ::substrait::Expression_FieldReference* delimiter_field) { +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_post_join_filter( + ::substrait::Expression* post_join_filter) { if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(delimiter_field_); + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(post_join_filter_); } - delimiter_field_ = delimiter_field; - if (delimiter_field) { + post_join_filter_ = post_join_filter; + if (post_join_filter) { } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DelimiterJoinRel.delimiter_field) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.post_join_filter) } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::release_delimiter_field() { +inline ::substrait::Expression* DuplicateEliminatedJoinRel::release_post_join_filter() { - ::substrait::Expression_FieldReference* temp = delimiter_field_; - delimiter_field_ = nullptr; + ::substrait::Expression* temp = post_join_filter_; + post_join_filter_ = nullptr; #ifdef PROTOBUF_FORCE_COPY_IN_RELEASE auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); @@ -40799,63 +41675,143 @@ inline ::substrait::Expression_FieldReference* DelimiterJoinRel::release_delimit #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::unsafe_arena_release_delimiter_field() { - // @@protoc_insertion_point(field_release:substrait.DelimiterJoinRel.delimiter_field) +inline ::substrait::Expression* DuplicateEliminatedJoinRel::unsafe_arena_release_post_join_filter() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.post_join_filter) - ::substrait::Expression_FieldReference* temp = delimiter_field_; - delimiter_field_ = nullptr; + ::substrait::Expression* temp = post_join_filter_; + post_join_filter_ = nullptr; return temp; } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::_internal_mutable_delimiter_field() { +inline ::substrait::Expression* DuplicateEliminatedJoinRel::_internal_mutable_post_join_filter() { - if (delimiter_field_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::Expression_FieldReference>(GetArenaForAllocation()); - delimiter_field_ = p; + if (post_join_filter_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::Expression>(GetArenaForAllocation()); + post_join_filter_ = p; } - return delimiter_field_; + return post_join_filter_; } -inline ::substrait::Expression_FieldReference* DelimiterJoinRel::mutable_delimiter_field() { - ::substrait::Expression_FieldReference* _msg = _internal_mutable_delimiter_field(); - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.delimiter_field) +inline ::substrait::Expression* DuplicateEliminatedJoinRel::mutable_post_join_filter() { + ::substrait::Expression* _msg = _internal_mutable_post_join_filter(); + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.post_join_filter) return _msg; } -inline void DelimiterJoinRel::set_allocated_delimiter_field(::substrait::Expression_FieldReference* delimiter_field) { +inline void DuplicateEliminatedJoinRel::set_allocated_post_join_filter(::substrait::Expression* post_join_filter) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { - delete delimiter_field_; + delete post_join_filter_; } - if (delimiter_field) { + if (post_join_filter) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_FieldReference>::GetOwningArena(delimiter_field); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression>::GetOwningArena(post_join_filter); if (message_arena != submessage_arena) { - delimiter_field = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, delimiter_field, submessage_arena); + post_join_filter = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, post_join_filter, submessage_arena); } } else { } - delimiter_field_ = delimiter_field; - // @@protoc_insertion_point(field_set_allocated:substrait.DelimiterJoinRel.delimiter_field) + post_join_filter_ = post_join_filter; + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.post_join_filter) +} + +// .substrait.DuplicateEliminatedJoinRel.JoinType type = 6; +inline void DuplicateEliminatedJoinRel::clear_type() { + type_ = 0; +} +inline ::substrait::DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::_internal_type() const { + return static_cast< ::substrait::DuplicateEliminatedJoinRel_JoinType >(type_); +} +inline ::substrait::DuplicateEliminatedJoinRel_JoinType DuplicateEliminatedJoinRel::type() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.type) + return _internal_type(); +} +inline void DuplicateEliminatedJoinRel::_internal_set_type(::substrait::DuplicateEliminatedJoinRel_JoinType value) { + + type_ = value; +} +inline void DuplicateEliminatedJoinRel::set_type(::substrait::DuplicateEliminatedJoinRel_JoinType value) { + _internal_set_type(value); + // @@protoc_insertion_point(field_set:substrait.DuplicateEliminatedJoinRel.type) +} + +// repeated .substrait.Expression.FieldReference duplicate_eliminated_columns = 7; +inline int DuplicateEliminatedJoinRel::_internal_duplicate_eliminated_columns_size() const { + return duplicate_eliminated_columns_.size(); +} +inline int DuplicateEliminatedJoinRel::duplicate_eliminated_columns_size() const { + return _internal_duplicate_eliminated_columns_size(); +} +inline void DuplicateEliminatedJoinRel::clear_duplicate_eliminated_columns() { + duplicate_eliminated_columns_.Clear(); +} +inline ::substrait::Expression_FieldReference* DuplicateEliminatedJoinRel::mutable_duplicate_eliminated_columns(int index) { + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_columns) + return duplicate_eliminated_columns_.Mutable(index); +} +inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >* +DuplicateEliminatedJoinRel::mutable_duplicate_eliminated_columns() { + // @@protoc_insertion_point(field_mutable_list:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_columns) + return &duplicate_eliminated_columns_; +} +inline const ::substrait::Expression_FieldReference& DuplicateEliminatedJoinRel::_internal_duplicate_eliminated_columns(int index) const { + return duplicate_eliminated_columns_.Get(index); +} +inline const ::substrait::Expression_FieldReference& DuplicateEliminatedJoinRel::duplicate_eliminated_columns(int index) const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_columns) + return _internal_duplicate_eliminated_columns(index); +} +inline ::substrait::Expression_FieldReference* DuplicateEliminatedJoinRel::_internal_add_duplicate_eliminated_columns() { + return duplicate_eliminated_columns_.Add(); +} +inline ::substrait::Expression_FieldReference* DuplicateEliminatedJoinRel::add_duplicate_eliminated_columns() { + ::substrait::Expression_FieldReference* _add = _internal_add_duplicate_eliminated_columns(); + // @@protoc_insertion_point(field_add:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_columns) + return _add; +} +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::Expression_FieldReference >& +DuplicateEliminatedJoinRel::duplicate_eliminated_columns() const { + // @@protoc_insertion_point(field_list:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_columns) + return duplicate_eliminated_columns_; +} + +// .substrait.DuplicateEliminatedJoinRel.DuplicateEliminatedSide duplicate_eliminated_side = 8; +inline void DuplicateEliminatedJoinRel::clear_duplicate_eliminated_side() { + duplicate_eliminated_side_ = 0; +} +inline ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::_internal_duplicate_eliminated_side() const { + return static_cast< ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide >(duplicate_eliminated_side_); +} +inline ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide DuplicateEliminatedJoinRel::duplicate_eliminated_side() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_side) + return _internal_duplicate_eliminated_side(); +} +inline void DuplicateEliminatedJoinRel::_internal_set_duplicate_eliminated_side(::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide value) { + + duplicate_eliminated_side_ = value; +} +inline void DuplicateEliminatedJoinRel::set_duplicate_eliminated_side(::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide value) { + _internal_set_duplicate_eliminated_side(value); + // @@protoc_insertion_point(field_set:substrait.DuplicateEliminatedJoinRel.duplicate_eliminated_side) } // .substrait.extensions.AdvancedExtension advanced_extension = 10; -inline bool DelimiterJoinRel::_internal_has_advanced_extension() const { +inline bool DuplicateEliminatedJoinRel::_internal_has_advanced_extension() const { return this != internal_default_instance() && advanced_extension_ != nullptr; } -inline bool DelimiterJoinRel::has_advanced_extension() const { +inline bool DuplicateEliminatedJoinRel::has_advanced_extension() const { return _internal_has_advanced_extension(); } -inline const ::substrait::extensions::AdvancedExtension& DelimiterJoinRel::_internal_advanced_extension() const { +inline const ::substrait::extensions::AdvancedExtension& DuplicateEliminatedJoinRel::_internal_advanced_extension() const { const ::substrait::extensions::AdvancedExtension* p = advanced_extension_; return p != nullptr ? *p : reinterpret_cast( ::substrait::extensions::_AdvancedExtension_default_instance_); } -inline const ::substrait::extensions::AdvancedExtension& DelimiterJoinRel::advanced_extension() const { - // @@protoc_insertion_point(field_get:substrait.DelimiterJoinRel.advanced_extension) +inline const ::substrait::extensions::AdvancedExtension& DuplicateEliminatedJoinRel::advanced_extension() const { + // @@protoc_insertion_point(field_get:substrait.DuplicateEliminatedJoinRel.advanced_extension) return _internal_advanced_extension(); } -inline void DelimiterJoinRel::unsafe_arena_set_allocated_advanced_extension( +inline void DuplicateEliminatedJoinRel::unsafe_arena_set_allocated_advanced_extension( ::substrait::extensions::AdvancedExtension* advanced_extension) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(advanced_extension_); @@ -40866,9 +41822,9 @@ inline void DelimiterJoinRel::unsafe_arena_set_allocated_advanced_extension( } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DelimiterJoinRel.advanced_extension) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DuplicateEliminatedJoinRel.advanced_extension) } -inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::release_advanced_extension() { +inline ::substrait::extensions::AdvancedExtension* DuplicateEliminatedJoinRel::release_advanced_extension() { ::substrait::extensions::AdvancedExtension* temp = advanced_extension_; advanced_extension_ = nullptr; @@ -40883,14 +41839,14 @@ inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::release_adv #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::unsafe_arena_release_advanced_extension() { - // @@protoc_insertion_point(field_release:substrait.DelimiterJoinRel.advanced_extension) +inline ::substrait::extensions::AdvancedExtension* DuplicateEliminatedJoinRel::unsafe_arena_release_advanced_extension() { + // @@protoc_insertion_point(field_release:substrait.DuplicateEliminatedJoinRel.advanced_extension) ::substrait::extensions::AdvancedExtension* temp = advanced_extension_; advanced_extension_ = nullptr; return temp; } -inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::_internal_mutable_advanced_extension() { +inline ::substrait::extensions::AdvancedExtension* DuplicateEliminatedJoinRel::_internal_mutable_advanced_extension() { if (advanced_extension_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::extensions::AdvancedExtension>(GetArenaForAllocation()); @@ -40898,12 +41854,12 @@ inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::_internal_m } return advanced_extension_; } -inline ::substrait::extensions::AdvancedExtension* DelimiterJoinRel::mutable_advanced_extension() { +inline ::substrait::extensions::AdvancedExtension* DuplicateEliminatedJoinRel::mutable_advanced_extension() { ::substrait::extensions::AdvancedExtension* _msg = _internal_mutable_advanced_extension(); - // @@protoc_insertion_point(field_mutable:substrait.DelimiterJoinRel.advanced_extension) + // @@protoc_insertion_point(field_mutable:substrait.DuplicateEliminatedJoinRel.advanced_extension) return _msg; } -inline void DelimiterJoinRel::set_allocated_advanced_extension(::substrait::extensions::AdvancedExtension* advanced_extension) { +inline void DuplicateEliminatedJoinRel::set_allocated_advanced_extension(::substrait::extensions::AdvancedExtension* advanced_extension) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(advanced_extension_); @@ -40922,7 +41878,7 @@ inline void DelimiterJoinRel::set_allocated_advanced_extension(::substrait::exte } advanced_extension_ = advanced_extension; - // @@protoc_insertion_point(field_set_allocated:substrait.DelimiterJoinRel.advanced_extension) + // @@protoc_insertion_point(field_set_allocated:substrait.DuplicateEliminatedJoinRel.advanced_extension) } // ------------------------------------------------------------------- @@ -41635,6 +42591,50 @@ inline void Expression_Literal_Decimal::set_scale(int32_t value) { // ------------------------------------------------------------------- +// Expression_Literal_PrecisionTimestamp + +// int32 precision = 1; +inline void Expression_Literal_PrecisionTimestamp::clear_precision() { + precision_ = 0; +} +inline int32_t Expression_Literal_PrecisionTimestamp::_internal_precision() const { + return precision_; +} +inline int32_t Expression_Literal_PrecisionTimestamp::precision() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.PrecisionTimestamp.precision) + return _internal_precision(); +} +inline void Expression_Literal_PrecisionTimestamp::_internal_set_precision(int32_t value) { + + precision_ = value; +} +inline void Expression_Literal_PrecisionTimestamp::set_precision(int32_t value) { + _internal_set_precision(value); + // @@protoc_insertion_point(field_set:substrait.Expression.Literal.PrecisionTimestamp.precision) +} + +// int64 value = 2; +inline void Expression_Literal_PrecisionTimestamp::clear_value() { + value_ = int64_t{0}; +} +inline int64_t Expression_Literal_PrecisionTimestamp::_internal_value() const { + return value_; +} +inline int64_t Expression_Literal_PrecisionTimestamp::value() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.PrecisionTimestamp.value) + return _internal_value(); +} +inline void Expression_Literal_PrecisionTimestamp::_internal_set_value(int64_t value) { + + value_ = value; +} +inline void Expression_Literal_PrecisionTimestamp::set_value(int64_t value) { + _internal_set_value(value); + // @@protoc_insertion_point(field_set:substrait.Expression.Literal.PrecisionTimestamp.value) +} + +// ------------------------------------------------------------------- + // Expression_Literal_Map_KeyValue // .substrait.Expression.Literal key = 1; @@ -41949,26 +42949,295 @@ inline void Expression_Literal_IntervalDayToSecond::set_seconds(int32_t value) { // @@protoc_insertion_point(field_set:substrait.Expression.Literal.IntervalDayToSecond.seconds) } -// int32 microseconds = 3; +// int32 microseconds = 3 [deprecated = true]; +inline bool Expression_Literal_IntervalDayToSecond::_internal_has_microseconds() const { + return precision_mode_case() == kMicroseconds; +} +inline bool Expression_Literal_IntervalDayToSecond::has_microseconds() const { + return _internal_has_microseconds(); +} +inline void Expression_Literal_IntervalDayToSecond::set_has_microseconds() { + _oneof_case_[0] = kMicroseconds; +} inline void Expression_Literal_IntervalDayToSecond::clear_microseconds() { - microseconds_ = 0; + if (_internal_has_microseconds()) { + precision_mode_.microseconds_ = 0; + clear_has_precision_mode(); + } } inline int32_t Expression_Literal_IntervalDayToSecond::_internal_microseconds() const { - return microseconds_; + if (_internal_has_microseconds()) { + return precision_mode_.microseconds_; + } + return 0; +} +inline void Expression_Literal_IntervalDayToSecond::_internal_set_microseconds(int32_t value) { + if (!_internal_has_microseconds()) { + clear_precision_mode(); + set_has_microseconds(); + } + precision_mode_.microseconds_ = value; } inline int32_t Expression_Literal_IntervalDayToSecond::microseconds() const { // @@protoc_insertion_point(field_get:substrait.Expression.Literal.IntervalDayToSecond.microseconds) return _internal_microseconds(); } -inline void Expression_Literal_IntervalDayToSecond::_internal_set_microseconds(int32_t value) { - - microseconds_ = value; -} inline void Expression_Literal_IntervalDayToSecond::set_microseconds(int32_t value) { _internal_set_microseconds(value); // @@protoc_insertion_point(field_set:substrait.Expression.Literal.IntervalDayToSecond.microseconds) } +// int32 precision = 4; +inline bool Expression_Literal_IntervalDayToSecond::_internal_has_precision() const { + return precision_mode_case() == kPrecision; +} +inline bool Expression_Literal_IntervalDayToSecond::has_precision() const { + return _internal_has_precision(); +} +inline void Expression_Literal_IntervalDayToSecond::set_has_precision() { + _oneof_case_[0] = kPrecision; +} +inline void Expression_Literal_IntervalDayToSecond::clear_precision() { + if (_internal_has_precision()) { + precision_mode_.precision_ = 0; + clear_has_precision_mode(); + } +} +inline int32_t Expression_Literal_IntervalDayToSecond::_internal_precision() const { + if (_internal_has_precision()) { + return precision_mode_.precision_; + } + return 0; +} +inline void Expression_Literal_IntervalDayToSecond::_internal_set_precision(int32_t value) { + if (!_internal_has_precision()) { + clear_precision_mode(); + set_has_precision(); + } + precision_mode_.precision_ = value; +} +inline int32_t Expression_Literal_IntervalDayToSecond::precision() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.IntervalDayToSecond.precision) + return _internal_precision(); +} +inline void Expression_Literal_IntervalDayToSecond::set_precision(int32_t value) { + _internal_set_precision(value); + // @@protoc_insertion_point(field_set:substrait.Expression.Literal.IntervalDayToSecond.precision) +} + +// int64 subseconds = 5; +inline void Expression_Literal_IntervalDayToSecond::clear_subseconds() { + subseconds_ = int64_t{0}; +} +inline int64_t Expression_Literal_IntervalDayToSecond::_internal_subseconds() const { + return subseconds_; +} +inline int64_t Expression_Literal_IntervalDayToSecond::subseconds() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.IntervalDayToSecond.subseconds) + return _internal_subseconds(); +} +inline void Expression_Literal_IntervalDayToSecond::_internal_set_subseconds(int64_t value) { + + subseconds_ = value; +} +inline void Expression_Literal_IntervalDayToSecond::set_subseconds(int64_t value) { + _internal_set_subseconds(value); + // @@protoc_insertion_point(field_set:substrait.Expression.Literal.IntervalDayToSecond.subseconds) +} + +inline bool Expression_Literal_IntervalDayToSecond::has_precision_mode() const { + return precision_mode_case() != PRECISION_MODE_NOT_SET; +} +inline void Expression_Literal_IntervalDayToSecond::clear_has_precision_mode() { + _oneof_case_[0] = PRECISION_MODE_NOT_SET; +} +inline Expression_Literal_IntervalDayToSecond::PrecisionModeCase Expression_Literal_IntervalDayToSecond::precision_mode_case() const { + return Expression_Literal_IntervalDayToSecond::PrecisionModeCase(_oneof_case_[0]); +} +// ------------------------------------------------------------------- + +// Expression_Literal_IntervalCompound + +// .substrait.Expression.Literal.IntervalYearToMonth interval_year_to_month = 1; +inline bool Expression_Literal_IntervalCompound::_internal_has_interval_year_to_month() const { + return this != internal_default_instance() && interval_year_to_month_ != nullptr; +} +inline bool Expression_Literal_IntervalCompound::has_interval_year_to_month() const { + return _internal_has_interval_year_to_month(); +} +inline void Expression_Literal_IntervalCompound::clear_interval_year_to_month() { + if (GetArenaForAllocation() == nullptr && interval_year_to_month_ != nullptr) { + delete interval_year_to_month_; + } + interval_year_to_month_ = nullptr; +} +inline const ::substrait::Expression_Literal_IntervalYearToMonth& Expression_Literal_IntervalCompound::_internal_interval_year_to_month() const { + const ::substrait::Expression_Literal_IntervalYearToMonth* p = interval_year_to_month_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_Expression_Literal_IntervalYearToMonth_default_instance_); +} +inline const ::substrait::Expression_Literal_IntervalYearToMonth& Expression_Literal_IntervalCompound::interval_year_to_month() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.IntervalCompound.interval_year_to_month) + return _internal_interval_year_to_month(); +} +inline void Expression_Literal_IntervalCompound::unsafe_arena_set_allocated_interval_year_to_month( + ::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(interval_year_to_month_); + } + interval_year_to_month_ = interval_year_to_month; + if (interval_year_to_month) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Expression.Literal.IntervalCompound.interval_year_to_month) +} +inline ::substrait::Expression_Literal_IntervalYearToMonth* Expression_Literal_IntervalCompound::release_interval_year_to_month() { + + ::substrait::Expression_Literal_IntervalYearToMonth* temp = interval_year_to_month_; + interval_year_to_month_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::Expression_Literal_IntervalYearToMonth* Expression_Literal_IntervalCompound::unsafe_arena_release_interval_year_to_month() { + // @@protoc_insertion_point(field_release:substrait.Expression.Literal.IntervalCompound.interval_year_to_month) + + ::substrait::Expression_Literal_IntervalYearToMonth* temp = interval_year_to_month_; + interval_year_to_month_ = nullptr; + return temp; +} +inline ::substrait::Expression_Literal_IntervalYearToMonth* Expression_Literal_IntervalCompound::_internal_mutable_interval_year_to_month() { + + if (interval_year_to_month_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::Expression_Literal_IntervalYearToMonth>(GetArenaForAllocation()); + interval_year_to_month_ = p; + } + return interval_year_to_month_; +} +inline ::substrait::Expression_Literal_IntervalYearToMonth* Expression_Literal_IntervalCompound::mutable_interval_year_to_month() { + ::substrait::Expression_Literal_IntervalYearToMonth* _msg = _internal_mutable_interval_year_to_month(); + // @@protoc_insertion_point(field_mutable:substrait.Expression.Literal.IntervalCompound.interval_year_to_month) + return _msg; +} +inline void Expression_Literal_IntervalCompound::set_allocated_interval_year_to_month(::substrait::Expression_Literal_IntervalYearToMonth* interval_year_to_month) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete interval_year_to_month_; + } + if (interval_year_to_month) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_Literal_IntervalYearToMonth>::GetOwningArena(interval_year_to_month); + if (message_arena != submessage_arena) { + interval_year_to_month = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_year_to_month, submessage_arena); + } + + } else { + + } + interval_year_to_month_ = interval_year_to_month; + // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.IntervalCompound.interval_year_to_month) +} + +// .substrait.Expression.Literal.IntervalDayToSecond interval_day_to_second = 2; +inline bool Expression_Literal_IntervalCompound::_internal_has_interval_day_to_second() const { + return this != internal_default_instance() && interval_day_to_second_ != nullptr; +} +inline bool Expression_Literal_IntervalCompound::has_interval_day_to_second() const { + return _internal_has_interval_day_to_second(); +} +inline void Expression_Literal_IntervalCompound::clear_interval_day_to_second() { + if (GetArenaForAllocation() == nullptr && interval_day_to_second_ != nullptr) { + delete interval_day_to_second_; + } + interval_day_to_second_ = nullptr; +} +inline const ::substrait::Expression_Literal_IntervalDayToSecond& Expression_Literal_IntervalCompound::_internal_interval_day_to_second() const { + const ::substrait::Expression_Literal_IntervalDayToSecond* p = interval_day_to_second_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_Expression_Literal_IntervalDayToSecond_default_instance_); +} +inline const ::substrait::Expression_Literal_IntervalDayToSecond& Expression_Literal_IntervalCompound::interval_day_to_second() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.IntervalCompound.interval_day_to_second) + return _internal_interval_day_to_second(); +} +inline void Expression_Literal_IntervalCompound::unsafe_arena_set_allocated_interval_day_to_second( + ::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(interval_day_to_second_); + } + interval_day_to_second_ = interval_day_to_second; + if (interval_day_to_second) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Expression.Literal.IntervalCompound.interval_day_to_second) +} +inline ::substrait::Expression_Literal_IntervalDayToSecond* Expression_Literal_IntervalCompound::release_interval_day_to_second() { + + ::substrait::Expression_Literal_IntervalDayToSecond* temp = interval_day_to_second_; + interval_day_to_second_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::Expression_Literal_IntervalDayToSecond* Expression_Literal_IntervalCompound::unsafe_arena_release_interval_day_to_second() { + // @@protoc_insertion_point(field_release:substrait.Expression.Literal.IntervalCompound.interval_day_to_second) + + ::substrait::Expression_Literal_IntervalDayToSecond* temp = interval_day_to_second_; + interval_day_to_second_ = nullptr; + return temp; +} +inline ::substrait::Expression_Literal_IntervalDayToSecond* Expression_Literal_IntervalCompound::_internal_mutable_interval_day_to_second() { + + if (interval_day_to_second_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::Expression_Literal_IntervalDayToSecond>(GetArenaForAllocation()); + interval_day_to_second_ = p; + } + return interval_day_to_second_; +} +inline ::substrait::Expression_Literal_IntervalDayToSecond* Expression_Literal_IntervalCompound::mutable_interval_day_to_second() { + ::substrait::Expression_Literal_IntervalDayToSecond* _msg = _internal_mutable_interval_day_to_second(); + // @@protoc_insertion_point(field_mutable:substrait.Expression.Literal.IntervalCompound.interval_day_to_second) + return _msg; +} +inline void Expression_Literal_IntervalCompound::set_allocated_interval_day_to_second(::substrait::Expression_Literal_IntervalDayToSecond* interval_day_to_second) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete interval_day_to_second_; + } + if (interval_day_to_second) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Expression_Literal_IntervalDayToSecond>::GetOwningArena(interval_day_to_second); + if (message_arena != submessage_arena) { + interval_day_to_second = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_day_to_second, submessage_arena); + } + + } else { + + } + interval_day_to_second_ = interval_day_to_second; + // @@protoc_insertion_point(field_set_allocated:substrait.Expression.Literal.IntervalCompound.interval_day_to_second) +} + // ------------------------------------------------------------------- // Expression_Literal_Struct @@ -42963,6 +44232,80 @@ inline ::substrait::Expression_Literal_IntervalDayToSecond* Expression_Literal:: return _msg; } +// .substrait.Expression.Literal.IntervalCompound interval_compound = 36; +inline bool Expression_Literal::_internal_has_interval_compound() const { + return literal_type_case() == kIntervalCompound; +} +inline bool Expression_Literal::has_interval_compound() const { + return _internal_has_interval_compound(); +} +inline void Expression_Literal::set_has_interval_compound() { + _oneof_case_[0] = kIntervalCompound; +} +inline void Expression_Literal::clear_interval_compound() { + if (_internal_has_interval_compound()) { + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.interval_compound_; + } + clear_has_literal_type(); + } +} +inline ::substrait::Expression_Literal_IntervalCompound* Expression_Literal::release_interval_compound() { + // @@protoc_insertion_point(field_release:substrait.Expression.Literal.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_literal_type(); + ::substrait::Expression_Literal_IntervalCompound* temp = literal_type_.interval_compound_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + literal_type_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::Expression_Literal_IntervalCompound& Expression_Literal::_internal_interval_compound() const { + return _internal_has_interval_compound() + ? *literal_type_.interval_compound_ + : reinterpret_cast< ::substrait::Expression_Literal_IntervalCompound&>(::substrait::_Expression_Literal_IntervalCompound_default_instance_); +} +inline const ::substrait::Expression_Literal_IntervalCompound& Expression_Literal::interval_compound() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.interval_compound) + return _internal_interval_compound(); +} +inline ::substrait::Expression_Literal_IntervalCompound* Expression_Literal::unsafe_arena_release_interval_compound() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Expression.Literal.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_literal_type(); + ::substrait::Expression_Literal_IntervalCompound* temp = literal_type_.interval_compound_; + literal_type_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void Expression_Literal::unsafe_arena_set_allocated_interval_compound(::substrait::Expression_Literal_IntervalCompound* interval_compound) { + clear_literal_type(); + if (interval_compound) { + set_has_interval_compound(); + literal_type_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Expression.Literal.interval_compound) +} +inline ::substrait::Expression_Literal_IntervalCompound* Expression_Literal::_internal_mutable_interval_compound() { + if (!_internal_has_interval_compound()) { + clear_literal_type(); + set_has_interval_compound(); + literal_type_.interval_compound_ = CreateMaybeMessage< ::substrait::Expression_Literal_IntervalCompound >(GetArenaForAllocation()); + } + return literal_type_.interval_compound_; +} +inline ::substrait::Expression_Literal_IntervalCompound* Expression_Literal::mutable_interval_compound() { + ::substrait::Expression_Literal_IntervalCompound* _msg = _internal_mutable_interval_compound(); + // @@protoc_insertion_point(field_mutable:substrait.Expression.Literal.interval_compound) + return _msg; +} + // string fixed_char = 21; inline bool Expression_Literal::_internal_has_fixed_char() const { return literal_type_case() == kFixedChar; @@ -43275,7 +44618,7 @@ inline ::substrait::Expression_Literal_Decimal* Expression_Literal::mutable_deci return _msg; } -// uint64 precision_timestamp = 34; +// .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp = 34; inline bool Expression_Literal::_internal_has_precision_timestamp() const { return literal_type_case() == kPrecisionTimestamp; } @@ -43287,33 +44630,69 @@ inline void Expression_Literal::set_has_precision_timestamp() { } inline void Expression_Literal::clear_precision_timestamp() { if (_internal_has_precision_timestamp()) { - literal_type_.precision_timestamp_ = uint64_t{0u}; + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.precision_timestamp_; + } clear_has_literal_type(); } } -inline uint64_t Expression_Literal::_internal_precision_timestamp() const { +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::release_precision_timestamp() { + // @@protoc_insertion_point(field_release:substrait.Expression.Literal.precision_timestamp) if (_internal_has_precision_timestamp()) { - return literal_type_.precision_timestamp_; + clear_has_literal_type(); + ::substrait::Expression_Literal_PrecisionTimestamp* temp = literal_type_.precision_timestamp_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + literal_type_.precision_timestamp_ = nullptr; + return temp; + } else { + return nullptr; } - return uint64_t{0u}; } -inline void Expression_Literal::_internal_set_precision_timestamp(uint64_t value) { +inline const ::substrait::Expression_Literal_PrecisionTimestamp& Expression_Literal::_internal_precision_timestamp() const { + return _internal_has_precision_timestamp() + ? *literal_type_.precision_timestamp_ + : reinterpret_cast< ::substrait::Expression_Literal_PrecisionTimestamp&>(::substrait::_Expression_Literal_PrecisionTimestamp_default_instance_); +} +inline const ::substrait::Expression_Literal_PrecisionTimestamp& Expression_Literal::precision_timestamp() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.precision_timestamp) + return _internal_precision_timestamp(); +} +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::unsafe_arena_release_precision_timestamp() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Expression.Literal.precision_timestamp) + if (_internal_has_precision_timestamp()) { + clear_has_literal_type(); + ::substrait::Expression_Literal_PrecisionTimestamp* temp = literal_type_.precision_timestamp_; + literal_type_.precision_timestamp_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void Expression_Literal::unsafe_arena_set_allocated_precision_timestamp(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp) { + clear_literal_type(); + if (precision_timestamp) { + set_has_precision_timestamp(); + literal_type_.precision_timestamp_ = precision_timestamp; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Expression.Literal.precision_timestamp) +} +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::_internal_mutable_precision_timestamp() { if (!_internal_has_precision_timestamp()) { clear_literal_type(); set_has_precision_timestamp(); + literal_type_.precision_timestamp_ = CreateMaybeMessage< ::substrait::Expression_Literal_PrecisionTimestamp >(GetArenaForAllocation()); } - literal_type_.precision_timestamp_ = value; + return literal_type_.precision_timestamp_; } -inline uint64_t Expression_Literal::precision_timestamp() const { - // @@protoc_insertion_point(field_get:substrait.Expression.Literal.precision_timestamp) - return _internal_precision_timestamp(); -} -inline void Expression_Literal::set_precision_timestamp(uint64_t value) { - _internal_set_precision_timestamp(value); - // @@protoc_insertion_point(field_set:substrait.Expression.Literal.precision_timestamp) +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::mutable_precision_timestamp() { + ::substrait::Expression_Literal_PrecisionTimestamp* _msg = _internal_mutable_precision_timestamp(); + // @@protoc_insertion_point(field_mutable:substrait.Expression.Literal.precision_timestamp) + return _msg; } -// uint64 precision_timestamp_tz = 35; +// .substrait.Expression.Literal.PrecisionTimestamp precision_timestamp_tz = 35; inline bool Expression_Literal::_internal_has_precision_timestamp_tz() const { return literal_type_case() == kPrecisionTimestampTz; } @@ -43325,30 +44704,66 @@ inline void Expression_Literal::set_has_precision_timestamp_tz() { } inline void Expression_Literal::clear_precision_timestamp_tz() { if (_internal_has_precision_timestamp_tz()) { - literal_type_.precision_timestamp_tz_ = uint64_t{0u}; + if (GetArenaForAllocation() == nullptr) { + delete literal_type_.precision_timestamp_tz_; + } clear_has_literal_type(); } } -inline uint64_t Expression_Literal::_internal_precision_timestamp_tz() const { +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::release_precision_timestamp_tz() { + // @@protoc_insertion_point(field_release:substrait.Expression.Literal.precision_timestamp_tz) if (_internal_has_precision_timestamp_tz()) { - return literal_type_.precision_timestamp_tz_; + clear_has_literal_type(); + ::substrait::Expression_Literal_PrecisionTimestamp* temp = literal_type_.precision_timestamp_tz_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + literal_type_.precision_timestamp_tz_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::Expression_Literal_PrecisionTimestamp& Expression_Literal::_internal_precision_timestamp_tz() const { + return _internal_has_precision_timestamp_tz() + ? *literal_type_.precision_timestamp_tz_ + : reinterpret_cast< ::substrait::Expression_Literal_PrecisionTimestamp&>(::substrait::_Expression_Literal_PrecisionTimestamp_default_instance_); +} +inline const ::substrait::Expression_Literal_PrecisionTimestamp& Expression_Literal::precision_timestamp_tz() const { + // @@protoc_insertion_point(field_get:substrait.Expression.Literal.precision_timestamp_tz) + return _internal_precision_timestamp_tz(); +} +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::unsafe_arena_release_precision_timestamp_tz() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Expression.Literal.precision_timestamp_tz) + if (_internal_has_precision_timestamp_tz()) { + clear_has_literal_type(); + ::substrait::Expression_Literal_PrecisionTimestamp* temp = literal_type_.precision_timestamp_tz_; + literal_type_.precision_timestamp_tz_ = nullptr; + return temp; + } else { + return nullptr; } - return uint64_t{0u}; } -inline void Expression_Literal::_internal_set_precision_timestamp_tz(uint64_t value) { +inline void Expression_Literal::unsafe_arena_set_allocated_precision_timestamp_tz(::substrait::Expression_Literal_PrecisionTimestamp* precision_timestamp_tz) { + clear_literal_type(); + if (precision_timestamp_tz) { + set_has_precision_timestamp_tz(); + literal_type_.precision_timestamp_tz_ = precision_timestamp_tz; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Expression.Literal.precision_timestamp_tz) +} +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::_internal_mutable_precision_timestamp_tz() { if (!_internal_has_precision_timestamp_tz()) { clear_literal_type(); set_has_precision_timestamp_tz(); + literal_type_.precision_timestamp_tz_ = CreateMaybeMessage< ::substrait::Expression_Literal_PrecisionTimestamp >(GetArenaForAllocation()); } - literal_type_.precision_timestamp_tz_ = value; + return literal_type_.precision_timestamp_tz_; } -inline uint64_t Expression_Literal::precision_timestamp_tz() const { - // @@protoc_insertion_point(field_get:substrait.Expression.Literal.precision_timestamp_tz) - return _internal_precision_timestamp_tz(); -} -inline void Expression_Literal::set_precision_timestamp_tz(uint64_t value) { - _internal_set_precision_timestamp_tz(value); - // @@protoc_insertion_point(field_set:substrait.Expression.Literal.precision_timestamp_tz) +inline ::substrait::Expression_Literal_PrecisionTimestamp* Expression_Literal::mutable_precision_timestamp_tz() { + ::substrait::Expression_Literal_PrecisionTimestamp* _msg = _internal_mutable_precision_timestamp_tz(); + // @@protoc_insertion_point(field_mutable:substrait.Expression.Literal.precision_timestamp_tz) + return _msg; } // .substrait.Expression.Literal.Struct struct = 25; @@ -52309,6 +53724,12 @@ inline void ReferenceRel::set_subtree_ordinal(int32_t value) { // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) @@ -52366,10 +53787,15 @@ template <> inline const EnumDescriptor* GetEnumDescriptor< ::substrait::NestedLoopJoinRel_JoinType>() { return ::substrait::NestedLoopJoinRel_JoinType_descriptor(); } -template <> struct is_proto_enum< ::substrait::DelimiterJoinRel_JoinType> : ::std::true_type {}; +template <> struct is_proto_enum< ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide> : ::std::true_type {}; +template <> +inline const EnumDescriptor* GetEnumDescriptor< ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide>() { + return ::substrait::DuplicateEliminatedJoinRel_DuplicateEliminatedSide_descriptor(); +} +template <> struct is_proto_enum< ::substrait::DuplicateEliminatedJoinRel_JoinType> : ::std::true_type {}; template <> -inline const EnumDescriptor* GetEnumDescriptor< ::substrait::DelimiterJoinRel_JoinType>() { - return ::substrait::DelimiterJoinRel_JoinType_descriptor(); +inline const EnumDescriptor* GetEnumDescriptor< ::substrait::DuplicateEliminatedJoinRel_JoinType>() { + return ::substrait::DuplicateEliminatedJoinRel_JoinType_descriptor(); } template <> struct is_proto_enum< ::substrait::Expression_WindowFunction_BoundsType> : ::std::true_type {}; template <> diff --git a/third_party/substrait/substrait/extensions/extensions.pb.cc b/third_party/substrait/substrait/extensions/extensions.pb.cc index 62b10af..4d4582e 100644 --- a/third_party/substrait/substrait/extensions/extensions.pb.cc +++ b/third_party/substrait/substrait/extensions/extensions.pb.cc @@ -87,7 +87,7 @@ struct SimpleExtensionDeclarationDefaultTypeInternal { PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT SimpleExtensionDeclarationDefaultTypeInternal _SimpleExtensionDeclaration_default_instance_; constexpr AdvancedExtension::AdvancedExtension( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) - : optimization_(nullptr) + : optimization_() , enhancement_(nullptr){} struct AdvancedExtensionDefaultTypeInternal { constexpr AdvancedExtensionDefaultTypeInternal() @@ -198,7 +198,7 @@ const char descriptor_table_protodef_substrait_2fextensions_2fextensions_2eproto "ionFunction\022\037\n\027extension_uri_reference\030\001" " \001(\r\022\027\n\017function_anchor\030\002 \001(\r\022\014\n\004name\030\003 " "\001(\tB\016\n\014mapping_type\"j\n\021AdvancedExtension" - "\022*\n\014optimization\030\001 \001(\0132\024.google.protobuf" + "\022*\n\014optimization\030\001 \003(\0132\024.google.protobuf" ".Any\022)\n\013enhancement\030\002 \001(\0132\024.google.proto" "buf.AnyBb\n\022io.substrait.protoP\001Z5github." "com/substrait-io/substrait-go/proto/exte" @@ -1602,23 +1602,15 @@ ::PROTOBUF_NAMESPACE_ID::Metadata SimpleExtensionDeclaration::GetMetadata() cons class AdvancedExtension::_Internal { public: - static const ::PROTOBUF_NAMESPACE_ID::Any& optimization(const AdvancedExtension* msg); static const ::PROTOBUF_NAMESPACE_ID::Any& enhancement(const AdvancedExtension* msg); }; -const ::PROTOBUF_NAMESPACE_ID::Any& -AdvancedExtension::_Internal::optimization(const AdvancedExtension* msg) { - return *msg->optimization_; -} const ::PROTOBUF_NAMESPACE_ID::Any& AdvancedExtension::_Internal::enhancement(const AdvancedExtension* msg) { return *msg->enhancement_; } void AdvancedExtension::clear_optimization() { - if (GetArenaForAllocation() == nullptr && optimization_ != nullptr) { - delete optimization_; - } - optimization_ = nullptr; + optimization_.Clear(); } void AdvancedExtension::clear_enhancement() { if (GetArenaForAllocation() == nullptr && enhancement_ != nullptr) { @@ -1628,7 +1620,8 @@ void AdvancedExtension::clear_enhancement() { } AdvancedExtension::AdvancedExtension(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) - : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned), + optimization_(arena) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); @@ -1636,13 +1629,9 @@ AdvancedExtension::AdvancedExtension(::PROTOBUF_NAMESPACE_ID::Arena* arena, // @@protoc_insertion_point(arena_constructor:substrait.extensions.AdvancedExtension) } AdvancedExtension::AdvancedExtension(const AdvancedExtension& from) - : ::PROTOBUF_NAMESPACE_ID::Message() { + : ::PROTOBUF_NAMESPACE_ID::Message(), + optimization_(from.optimization_) { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); - if (from._internal_has_optimization()) { - optimization_ = new ::PROTOBUF_NAMESPACE_ID::Any(*from.optimization_); - } else { - optimization_ = nullptr; - } if (from._internal_has_enhancement()) { enhancement_ = new ::PROTOBUF_NAMESPACE_ID::Any(*from.enhancement_); } else { @@ -1652,10 +1641,7 @@ AdvancedExtension::AdvancedExtension(const AdvancedExtension& from) } inline void AdvancedExtension::SharedCtor() { -::memset(reinterpret_cast(this) + static_cast( - reinterpret_cast(&optimization_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(&enhancement_) - - reinterpret_cast(&optimization_)) + sizeof(enhancement_)); +enhancement_ = nullptr; } AdvancedExtension::~AdvancedExtension() { @@ -1667,7 +1653,6 @@ AdvancedExtension::~AdvancedExtension() { inline void AdvancedExtension::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); - if (this != internal_default_instance()) delete optimization_; if (this != internal_default_instance()) delete enhancement_; } @@ -1687,10 +1672,7 @@ void AdvancedExtension::Clear() { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - if (GetArenaForAllocation() == nullptr && optimization_ != nullptr) { - delete optimization_; - } - optimization_ = nullptr; + optimization_.Clear(); if (GetArenaForAllocation() == nullptr && enhancement_ != nullptr) { delete enhancement_; } @@ -1704,11 +1686,16 @@ const char* AdvancedExtension::_InternalParse(const char* ptr, ::PROTOBUF_NAMESP uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); switch (tag >> 3) { - // .google.protobuf.Any optimization = 1; + // repeated .google.protobuf.Any optimization = 1; case 1: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { - ptr = ctx->ParseMessage(_internal_mutable_optimization(), ptr); - CHK_(ptr); + ptr -= 1; + do { + ptr += 1; + ptr = ctx->ParseMessage(_internal_add_optimization(), ptr); + CHK_(ptr); + if (!ctx->DataAvailable(ptr)) break; + } while (::PROTOBUF_NAMESPACE_ID::internal::ExpectTag<10>(ptr)); } else goto handle_unusual; continue; @@ -1749,12 +1736,12 @@ uint8_t* AdvancedExtension::_InternalSerialize( uint32_t cached_has_bits = 0; (void) cached_has_bits; - // .google.protobuf.Any optimization = 1; - if (this->_internal_has_optimization()) { + // repeated .google.protobuf.Any optimization = 1; + for (unsigned int i = 0, + n = static_cast(this->_internal_optimization_size()); i < n; i++) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 1, _Internal::optimization(this), target, stream); + InternalWriteMessage(1, this->_internal_optimization(i), target, stream); } // .google.protobuf.Any enhancement = 2; @@ -1781,11 +1768,11 @@ size_t AdvancedExtension::ByteSizeLong() const { // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // .google.protobuf.Any optimization = 1; - if (this->_internal_has_optimization()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *optimization_); + // repeated .google.protobuf.Any optimization = 1; + total_size += 1UL * this->_internal_optimization_size(); + for (const auto& msg : this->optimization_) { + total_size += + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize(msg); } // .google.protobuf.Any enhancement = 2; @@ -1817,9 +1804,7 @@ void AdvancedExtension::MergeFrom(const AdvancedExtension& from) { uint32_t cached_has_bits = 0; (void) cached_has_bits; - if (from._internal_has_optimization()) { - _internal_mutable_optimization()->::PROTOBUF_NAMESPACE_ID::Any::MergeFrom(from._internal_optimization()); - } + optimization_.MergeFrom(from.optimization_); if (from._internal_has_enhancement()) { _internal_mutable_enhancement()->::PROTOBUF_NAMESPACE_ID::Any::MergeFrom(from._internal_enhancement()); } @@ -1840,12 +1825,8 @@ bool AdvancedExtension::IsInitialized() const { void AdvancedExtension::InternalSwap(AdvancedExtension* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); - ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(AdvancedExtension, enhancement_) - + sizeof(AdvancedExtension::enhancement_) - - PROTOBUF_FIELD_OFFSET(AdvancedExtension, optimization_)>( - reinterpret_cast(&optimization_), - reinterpret_cast(&other->optimization_)); + optimization_.InternalSwap(&other->optimization_); + swap(enhancement_, other->enhancement_); } ::PROTOBUF_NAMESPACE_ID::Metadata AdvancedExtension::GetMetadata() const { diff --git a/third_party/substrait/substrait/extensions/extensions.pb.h b/third_party/substrait/substrait/extensions/extensions.pb.h index d1cdd0f..60f2fb2 100644 --- a/third_party/substrait/substrait/extensions/extensions.pb.h +++ b/third_party/substrait/substrait/extensions/extensions.pb.h @@ -1115,23 +1115,23 @@ class AdvancedExtension final : kOptimizationFieldNumber = 1, kEnhancementFieldNumber = 2, }; - // .google.protobuf.Any optimization = 1; - bool has_optimization() const; + // repeated .google.protobuf.Any optimization = 1; + int optimization_size() const; private: - bool _internal_has_optimization() const; + int _internal_optimization_size() const; public: void clear_optimization(); - const ::PROTOBUF_NAMESPACE_ID::Any& optimization() const; - PROTOBUF_NODISCARD ::PROTOBUF_NAMESPACE_ID::Any* release_optimization(); - ::PROTOBUF_NAMESPACE_ID::Any* mutable_optimization(); - void set_allocated_optimization(::PROTOBUF_NAMESPACE_ID::Any* optimization); + ::PROTOBUF_NAMESPACE_ID::Any* mutable_optimization(int index); + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::PROTOBUF_NAMESPACE_ID::Any >* + mutable_optimization(); private: - const ::PROTOBUF_NAMESPACE_ID::Any& _internal_optimization() const; - ::PROTOBUF_NAMESPACE_ID::Any* _internal_mutable_optimization(); + const ::PROTOBUF_NAMESPACE_ID::Any& _internal_optimization(int index) const; + ::PROTOBUF_NAMESPACE_ID::Any* _internal_add_optimization(); public: - void unsafe_arena_set_allocated_optimization( - ::PROTOBUF_NAMESPACE_ID::Any* optimization); - ::PROTOBUF_NAMESPACE_ID::Any* unsafe_arena_release_optimization(); + const ::PROTOBUF_NAMESPACE_ID::Any& optimization(int index) const; + ::PROTOBUF_NAMESPACE_ID::Any* add_optimization(); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::PROTOBUF_NAMESPACE_ID::Any >& + optimization() const; // .google.protobuf.Any enhancement = 2; bool has_enhancement() const; @@ -1158,7 +1158,7 @@ class AdvancedExtension final : template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::Any* optimization_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::PROTOBUF_NAMESPACE_ID::Any > optimization_; ::PROTOBUF_NAMESPACE_ID::Any* enhancement_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2fextensions_2fextensions_2eproto; @@ -1769,90 +1769,41 @@ inline SimpleExtensionDeclaration::MappingTypeCase SimpleExtensionDeclaration::m // AdvancedExtension -// .google.protobuf.Any optimization = 1; -inline bool AdvancedExtension::_internal_has_optimization() const { - return this != internal_default_instance() && optimization_ != nullptr; +// repeated .google.protobuf.Any optimization = 1; +inline int AdvancedExtension::_internal_optimization_size() const { + return optimization_.size(); } -inline bool AdvancedExtension::has_optimization() const { - return _internal_has_optimization(); +inline int AdvancedExtension::optimization_size() const { + return _internal_optimization_size(); } -inline const ::PROTOBUF_NAMESPACE_ID::Any& AdvancedExtension::_internal_optimization() const { - const ::PROTOBUF_NAMESPACE_ID::Any* p = optimization_; - return p != nullptr ? *p : reinterpret_cast( - ::PROTOBUF_NAMESPACE_ID::_Any_default_instance_); -} -inline const ::PROTOBUF_NAMESPACE_ID::Any& AdvancedExtension::optimization() const { - // @@protoc_insertion_point(field_get:substrait.extensions.AdvancedExtension.optimization) - return _internal_optimization(); +inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::mutable_optimization(int index) { + // @@protoc_insertion_point(field_mutable:substrait.extensions.AdvancedExtension.optimization) + return optimization_.Mutable(index); } -inline void AdvancedExtension::unsafe_arena_set_allocated_optimization( - ::PROTOBUF_NAMESPACE_ID::Any* optimization) { - if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(optimization_); - } - optimization_ = optimization; - if (optimization) { - - } else { - - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.extensions.AdvancedExtension.optimization) +inline ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::PROTOBUF_NAMESPACE_ID::Any >* +AdvancedExtension::mutable_optimization() { + // @@protoc_insertion_point(field_mutable_list:substrait.extensions.AdvancedExtension.optimization) + return &optimization_; } -inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::release_optimization() { - - ::PROTOBUF_NAMESPACE_ID::Any* temp = optimization_; - optimization_ = nullptr; -#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE - auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - if (GetArenaForAllocation() == nullptr) { delete old; } -#else // PROTOBUF_FORCE_COPY_IN_RELEASE - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } -#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE - return temp; +inline const ::PROTOBUF_NAMESPACE_ID::Any& AdvancedExtension::_internal_optimization(int index) const { + return optimization_.Get(index); } -inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::unsafe_arena_release_optimization() { - // @@protoc_insertion_point(field_release:substrait.extensions.AdvancedExtension.optimization) - - ::PROTOBUF_NAMESPACE_ID::Any* temp = optimization_; - optimization_ = nullptr; - return temp; +inline const ::PROTOBUF_NAMESPACE_ID::Any& AdvancedExtension::optimization(int index) const { + // @@protoc_insertion_point(field_get:substrait.extensions.AdvancedExtension.optimization) + return _internal_optimization(index); } -inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::_internal_mutable_optimization() { - - if (optimization_ == nullptr) { - auto* p = CreateMaybeMessage<::PROTOBUF_NAMESPACE_ID::Any>(GetArenaForAllocation()); - optimization_ = p; - } - return optimization_; +inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::_internal_add_optimization() { + return optimization_.Add(); } -inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::mutable_optimization() { - ::PROTOBUF_NAMESPACE_ID::Any* _msg = _internal_mutable_optimization(); - // @@protoc_insertion_point(field_mutable:substrait.extensions.AdvancedExtension.optimization) - return _msg; +inline ::PROTOBUF_NAMESPACE_ID::Any* AdvancedExtension::add_optimization() { + ::PROTOBUF_NAMESPACE_ID::Any* _add = _internal_add_optimization(); + // @@protoc_insertion_point(field_add:substrait.extensions.AdvancedExtension.optimization) + return _add; } -inline void AdvancedExtension::set_allocated_optimization(::PROTOBUF_NAMESPACE_ID::Any* optimization) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - if (message_arena == nullptr) { - delete reinterpret_cast< ::PROTOBUF_NAMESPACE_ID::MessageLite*>(optimization_); - } - if (optimization) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper< - ::PROTOBUF_NAMESPACE_ID::MessageLite>::GetOwningArena( - reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(optimization)); - if (message_arena != submessage_arena) { - optimization = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, optimization, submessage_arena); - } - - } else { - - } - optimization_ = optimization; - // @@protoc_insertion_point(field_set_allocated:substrait.extensions.AdvancedExtension.optimization) +inline const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::PROTOBUF_NAMESPACE_ID::Any >& +AdvancedExtension::optimization() const { + // @@protoc_insertion_point(field_list:substrait.extensions.AdvancedExtension.optimization) + return optimization_; } // .google.protobuf.Any enhancement = 2; diff --git a/third_party/substrait/substrait/parameterized_types.pb.cc b/third_party/substrait/substrait/parameterized_types.pb.cc index 8cb7ebc..1adc066 100644 --- a/third_party/substrait/substrait/parameterized_types.pb.cc +++ b/third_party/substrait/substrait/parameterized_types.pb.cc @@ -117,6 +117,36 @@ struct ParameterizedType_ParameterizedDecimalDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ParameterizedType_ParameterizedDecimalDefaultTypeInternal _ParameterizedType_ParameterizedDecimal_default_instance_; +constexpr ParameterizedType_ParameterizedIntervalDay::ParameterizedType_ParameterizedIntervalDay( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : precision_(nullptr) + , variation_pointer_(0u) + , nullability_(0) +{} +struct ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal { + constexpr ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal() {} + union { + ParameterizedType_ParameterizedIntervalDay _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal _ParameterizedType_ParameterizedIntervalDay_default_instance_; +constexpr ParameterizedType_ParameterizedIntervalCompound::ParameterizedType_ParameterizedIntervalCompound( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : precision_(nullptr) + , variation_pointer_(0u) + , nullability_(0) +{} +struct ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal { + constexpr ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal() {} + union { + ParameterizedType_ParameterizedIntervalCompound _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal _ParameterizedType_ParameterizedIntervalCompound_default_instance_; constexpr ParameterizedType_ParameterizedPrecisionTimestamp::ParameterizedType_ParameterizedPrecisionTimestamp( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : precision_(nullptr) @@ -246,7 +276,7 @@ struct ParameterizedTypeDefaultTypeInternal { }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT ParameterizedTypeDefaultTypeInternal _ParameterizedType_default_instance_; } // namespace substrait -static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[16]; +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[18]; static constexpr ::PROTOBUF_NAMESPACE_ID::EnumDescriptor const** file_level_enum_descriptors_substrait_2fparameterized_5ftypes_2eproto = nullptr; static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_substrait_2fparameterized_5ftypes_2eproto = nullptr; @@ -313,6 +343,24 @@ const uint32_t TableStruct_substrait_2fparameterized_5ftypes_2eproto::offsets[] PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedDecimal, variation_pointer_), PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedDecimal, nullability_), ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalDay, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalDay, precision_), + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalDay, variation_pointer_), + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalDay, nullability_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalCompound, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalCompound, precision_), + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalCompound, variation_pointer_), + PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedIntervalCompound, nullability_), + ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType_ParameterizedPrecisionTimestamp, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ @@ -418,6 +466,7 @@ const uint32_t TableStruct_substrait_2fparameterized_5ftypes_2eproto::offsets[] ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, PROTOBUF_FIELD_OFFSET(::substrait::ParameterizedType, kind_), }; static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { @@ -428,15 +477,17 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOB { 33, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedVarChar)}, { 42, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedFixedBinary)}, { 51, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedDecimal)}, - { 61, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedPrecisionTimestamp)}, - { 70, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedPrecisionTimestampTZ)}, - { 79, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedStruct)}, - { 88, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedNamedStruct)}, - { 96, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedList)}, - { 105, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedMap)}, - { 115, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedUserDefined)}, - { 124, -1, -1, sizeof(::substrait::ParameterizedType_IntegerOption)}, - { 133, -1, -1, sizeof(::substrait::ParameterizedType)}, + { 61, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedIntervalDay)}, + { 70, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedIntervalCompound)}, + { 79, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedPrecisionTimestamp)}, + { 88, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedPrecisionTimestampTZ)}, + { 97, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedStruct)}, + { 106, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedNamedStruct)}, + { 114, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedList)}, + { 123, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedMap)}, + { 133, -1, -1, sizeof(::substrait::ParameterizedType_ParameterizedUserDefined)}, + { 142, -1, -1, sizeof(::substrait::ParameterizedType_IntegerOption)}, + { 151, -1, -1, sizeof(::substrait::ParameterizedType)}, }; static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { @@ -447,6 +498,8 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedVarChar_default_instance_), reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedFixedBinary_default_instance_), reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedDecimal_default_instance_), + reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedIntervalDay_default_instance_), + reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedIntervalCompound_default_instance_), reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedPrecisionTimestamp_default_instance_), reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedPrecisionTimestampTZ_default_instance_), reinterpret_cast(&::substrait::_ParameterizedType_ParameterizedStruct_default_instance_), @@ -460,7 +513,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = const char descriptor_table_protodef_substrait_2fparameterized_5ftypes_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = "\n#substrait/parameterized_types.proto\022\ts" - "ubstrait\032\024substrait/type.proto\"\264\035\n\021Param" + "ubstrait\032\024substrait/type.proto\"\376 \n\021Param" "eterizedType\022\'\n\004bool\030\001 \001(\0132\027.substrait.T" "ype.BooleanH\000\022 \n\002i8\030\002 \001(\0132\022.substrait.Ty" "pe.I8H\000\022\"\n\003i16\030\003 \001(\0132\023.substrait.Type.I1" @@ -474,98 +527,109 @@ const char descriptor_table_protodef_substrait_2fparameterized_5ftypes_2eproto[] "pB\002\030\001H\000\022$\n\004date\030\020 \001(\0132\024.substrait.Type.D" "ateH\000\022$\n\004time\030\021 \001(\0132\024.substrait.Type.Tim" "eH\000\0225\n\rinterval_year\030\023 \001(\0132\034.substrait.T" - "ype.IntervalYearH\000\0223\n\014interval_day\030\024 \001(\013" - "2\033.substrait.Type.IntervalDayH\000\0227\n\014times" - "tamp_tz\030\035 \001(\0132\033.substrait.Type.Timestamp" - "TZB\002\030\001H\000\022$\n\004uuid\030 \001(\0132\024.substrait.Type." - "UUIDH\000\022I\n\nfixed_char\030\025 \001(\01323.substrait.P" - "arameterizedType.ParameterizedFixedCharH" - "\000\022D\n\007varchar\030\026 \001(\01321.substrait.Parameter" - "izedType.ParameterizedVarCharH\000\022M\n\014fixed" - "_binary\030\027 \001(\01325.substrait.ParameterizedT" - "ype.ParameterizedFixedBinaryH\000\022D\n\007decima" - "l\030\030 \001(\01321.substrait.ParameterizedType.Pa" - "rameterizedDecimalH\000\022[\n\023precision_timest" - "amp\030\" \001(\0132<.substrait.ParameterizedType." - "ParameterizedPrecisionTimestampH\000\022`\n\026pre" - "cision_timestamp_tz\030# \001(\0132>.substrait.Pa" - "rameterizedType.ParameterizedPrecisionTi" - "mestampTZH\000\022B\n\006struct\030\031 \001(\01320.substrait." - "ParameterizedType.ParameterizedStructH\000\022" - ">\n\004list\030\033 \001(\0132..substrait.ParameterizedT" - "ype.ParameterizedListH\000\022<\n\003map\030\034 \001(\0132-.s" - "ubstrait.ParameterizedType.Parameterized" - "MapH\000\022M\n\014user_defined\030\036 \001(\01325.substrait." - "ParameterizedType.ParameterizedUserDefin" - "edH\000\022\"\n\024user_defined_pointer\030\037 \001(\rB\002\030\001H\000" - "\022D\n\016type_parameter\030! \001(\0132*.substrait.Par" - "ameterizedType.TypeParameterH\000\032K\n\rTypePa" - "rameter\022\014\n\004name\030\001 \001(\t\022,\n\006bounds\030\002 \003(\0132\034." - "substrait.ParameterizedType\032\270\001\n\020IntegerP" - "arameter\022\014\n\004name\030\001 \001(\t\022K\n\025range_start_in" - "clusive\030\002 \001(\0132,.substrait.ParameterizedT" - "ype.NullableInteger\022I\n\023range_end_exclusi" - "ve\030\003 \001(\0132,.substrait.ParameterizedType.N" - "ullableInteger\032 \n\017NullableInteger\022\r\n\005val" - "ue\030\001 \001(\003\032\241\001\n\026ParameterizedFixedChar\022:\n\006l" - "ength\030\001 \001(\0132*.substrait.ParameterizedTyp" - "e.IntegerOption\022\031\n\021variation_pointer\030\002 \001" - "(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Type" - ".Nullability\032\237\001\n\024ParameterizedVarChar\022:\n" - "\006length\030\001 \001(\0132*.substrait.ParameterizedT" - "ype.IntegerOption\022\031\n\021variation_pointer\030\002" - " \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Ty" - "pe.Nullability\032\243\001\n\030ParameterizedFixedBin" - "ary\022:\n\006length\030\001 \001(\0132*.substrait.Paramete" - "rizedType.IntegerOption\022\031\n\021variation_poi" - "nter\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substr" - "ait.Type.Nullability\032\335\001\n\024ParameterizedDe" - "cimal\0229\n\005scale\030\001 \001(\0132*.substrait.Paramet" - "erizedType.IntegerOption\022=\n\tprecision\030\002 " - "\001(\0132*.substrait.ParameterizedType.Intege" - "rOption\022\031\n\021variation_pointer\030\003 \001(\r\0220\n\013nu" - "llability\030\004 \001(\0162\033.substrait.Type.Nullabi" - "lity\032\255\001\n\037ParameterizedPrecisionTimestamp" - "\022=\n\tprecision\030\001 \001(\0132*.substrait.Paramete" - "rizedType.IntegerOption\022\031\n\021variation_poi" - "nter\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substr" - "ait.Type.Nullability\032\257\001\n!ParameterizedPr" - "ecisionTimestampTZ\022=\n\tprecision\030\001 \001(\0132*." + "ype.IntervalYearH\000\022M\n\014interval_day\030\024 \001(\013" + "25.substrait.ParameterizedType.Parameter" + "izedIntervalDayH\000\022W\n\021interval_compound\030$" + " \001(\0132:.substrait.ParameterizedType.Param" + "eterizedIntervalCompoundH\000\0227\n\014timestamp_" + "tz\030\035 \001(\0132\033.substrait.Type.TimestampTZB\002\030" + "\001H\000\022$\n\004uuid\030 \001(\0132\024.substrait.Type.UUIDH" + "\000\022I\n\nfixed_char\030\025 \001(\01323.substrait.Parame" + "terizedType.ParameterizedFixedCharH\000\022D\n\007" + "varchar\030\026 \001(\01321.substrait.ParameterizedT" + "ype.ParameterizedVarCharH\000\022M\n\014fixed_bina" + "ry\030\027 \001(\01325.substrait.ParameterizedType.P" + "arameterizedFixedBinaryH\000\022D\n\007decimal\030\030 \001" + "(\01321.substrait.ParameterizedType.Paramet" + "erizedDecimalH\000\022[\n\023precision_timestamp\030\"" + " \001(\0132<.substrait.ParameterizedType.Param" + "eterizedPrecisionTimestampH\000\022`\n\026precisio" + "n_timestamp_tz\030# \001(\0132>.substrait.Paramet" + "erizedType.ParameterizedPrecisionTimesta" + "mpTZH\000\022B\n\006struct\030\031 \001(\01320.substrait.Param" + "eterizedType.ParameterizedStructH\000\022>\n\004li" + "st\030\033 \001(\0132..substrait.ParameterizedType.P" + "arameterizedListH\000\022<\n\003map\030\034 \001(\0132-.substr" + "ait.ParameterizedType.ParameterizedMapH\000" + "\022M\n\014user_defined\030\036 \001(\01325.substrait.Param" + "eterizedType.ParameterizedUserDefinedH\000\022" + "\"\n\024user_defined_pointer\030\037 \001(\rB\002\030\001H\000\022D\n\016t" + "ype_parameter\030! \001(\0132*.substrait.Paramete" + "rizedType.TypeParameterH\000\032K\n\rTypeParamet" + "er\022\014\n\004name\030\001 \001(\t\022,\n\006bounds\030\002 \003(\0132\034.subst" + "rait.ParameterizedType\032\270\001\n\020IntegerParame" + "ter\022\014\n\004name\030\001 \001(\t\022K\n\025range_start_inclusi" + "ve\030\002 \001(\0132,.substrait.ParameterizedType.N" + "ullableInteger\022I\n\023range_end_exclusive\030\003 " + "\001(\0132,.substrait.ParameterizedType.Nullab" + "leInteger\032 \n\017NullableInteger\022\r\n\005value\030\001 " + "\001(\003\032\241\001\n\026ParameterizedFixedChar\022:\n\006length" + "\030\001 \001(\0132*.substrait.ParameterizedType.Int" + "egerOption\022\031\n\021variation_pointer\030\002 \001(\r\0220\n" + "\013nullability\030\003 \001(\0162\033.substrait.Type.Null" + "ability\032\237\001\n\024ParameterizedVarChar\022:\n\006leng" + "th\030\001 \001(\0132*.substrait.ParameterizedType.I" + "ntegerOption\022\031\n\021variation_pointer\030\002 \001(\r\022" + "0\n\013nullability\030\003 \001(\0162\033.substrait.Type.Nu" + "llability\032\243\001\n\030ParameterizedFixedBinary\022:" + "\n\006length\030\001 \001(\0132*.substrait.Parameterized" + "Type.IntegerOption\022\031\n\021variation_pointer\030" + "\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.T" + "ype.Nullability\032\335\001\n\024ParameterizedDecimal" + "\0229\n\005scale\030\001 \001(\0132*.substrait.Parameterize" + "dType.IntegerOption\022=\n\tprecision\030\002 \001(\0132*" + ".substrait.ParameterizedType.IntegerOpti" + "on\022\031\n\021variation_pointer\030\003 \001(\r\0220\n\013nullabi" + "lity\030\004 \001(\0162\033.substrait.Type.Nullability\032" + "\246\001\n\030ParameterizedIntervalDay\022=\n\tprecisio" + "n\030\001 \001(\0132*.substrait.ParameterizedType.In" + "tegerOption\022\031\n\021variation_pointer\030\002 \001(\r\0220" + "\n\013nullability\030\003 \001(\0162\033.substrait.Type.Nul" + "lability\032\253\001\n\035ParameterizedIntervalCompou" + "nd\022=\n\tprecision\030\001 \001(\0132*.substrait.Parame" + "terizedType.IntegerOption\022\031\n\021variation_p" + "ointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.subs" + "trait.Type.Nullability\032\255\001\n\037Parameterized" + "PrecisionTimestamp\022=\n\tprecision\030\001 \001(\0132*." "substrait.ParameterizedType.IntegerOptio" "n\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nullabil" - "ity\030\003 \001(\0162\033.substrait.Type.Nullability\032\217" - "\001\n\023ParameterizedStruct\022+\n\005types\030\001 \003(\0132\034." - "substrait.ParameterizedType\022\031\n\021variation" - "_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.su" - "bstrait.Type.Nullability\032k\n\030Parameterize" - "dNamedStruct\022\r\n\005names\030\001 \003(\t\022@\n\006struct\030\002 " - "\001(\01320.substrait.ParameterizedType.Parame" - "terizedStruct\032\214\001\n\021ParameterizedList\022*\n\004t" - "ype\030\001 \001(\0132\034.substrait.ParameterizedType\022" - "\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nullabilit" - "y\030\003 \001(\0162\033.substrait.Type.Nullability\032\267\001\n" - "\020ParameterizedMap\022)\n\003key\030\001 \001(\0132\034.substra" - "it.ParameterizedType\022+\n\005value\030\002 \001(\0132\034.su" - "bstrait.ParameterizedType\022\031\n\021variation_p" - "ointer\030\003 \001(\r\0220\n\013nullability\030\004 \001(\0162\033.subs" - "trait.Type.Nullability\032}\n\030ParameterizedU" - "serDefined\022\024\n\014type_pointer\030\001 \001(\r\022\031\n\021vari" - "ation_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\016" - "2\033.substrait.Type.Nullability\032v\n\rInteger" - "Option\022\021\n\007literal\030\001 \001(\005H\000\022B\n\tparameter\030\002" - " \001(\0132-.substrait.ParameterizedType.Integ" - "erParameterH\000B\016\n\014integer_typeB\006\n\004kindBW\n" - "\022io.substrait.protoP\001Z*github.com/substr" - "ait-io/substrait-go/proto\252\002\022Substrait.Pr" - "otobufb\006proto3" + "ity\030\003 \001(\0162\033.substrait.Type.Nullability\032\257" + "\001\n!ParameterizedPrecisionTimestampTZ\022=\n\t" + "precision\030\001 \001(\0132*.substrait.Parameterize" + "dType.IntegerOption\022\031\n\021variation_pointer" + "\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait." + "Type.Nullability\032\217\001\n\023ParameterizedStruct" + "\022+\n\005types\030\001 \003(\0132\034.substrait.Parameterize" + "dType\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013null" + "ability\030\003 \001(\0162\033.substrait.Type.Nullabili" + "ty\032k\n\030ParameterizedNamedStruct\022\r\n\005names\030" + "\001 \003(\t\022@\n\006struct\030\002 \001(\01320.substrait.Parame" + "terizedType.ParameterizedStruct\032\214\001\n\021Para" + "meterizedList\022*\n\004type\030\001 \001(\0132\034.substrait." + "ParameterizedType\022\031\n\021variation_pointer\030\002" + " \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Ty" + "pe.Nullability\032\267\001\n\020ParameterizedMap\022)\n\003k" + "ey\030\001 \001(\0132\034.substrait.ParameterizedType\022+" + "\n\005value\030\002 \001(\0132\034.substrait.ParameterizedT" + "ype\022\031\n\021variation_pointer\030\003 \001(\r\0220\n\013nullab" + "ility\030\004 \001(\0162\033.substrait.Type.Nullability" + "\032}\n\030ParameterizedUserDefined\022\024\n\014type_poi" + "nter\030\001 \001(\r\022\031\n\021variation_pointer\030\002 \001(\r\0220\n" + "\013nullability\030\003 \001(\0162\033.substrait.Type.Null" + "ability\032v\n\rIntegerOption\022\021\n\007literal\030\001 \001(" + "\005H\000\022B\n\tparameter\030\002 \001(\0132-.substrait.Param" + "eterizedType.IntegerParameterH\000B\016\n\014integ" + "er_typeB\006\n\004kindBW\n\022io.substrait.protoP\001Z" + "*github.com/substrait-io/substrait-go/pr" + "oto\252\002\022Substrait.Protobufb\006proto3" ; static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_substrait_2fparameterized_5ftypes_2eproto_deps[1] = { &::descriptor_table_substrait_2ftype_2eproto, }; static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once; const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_substrait_2fparameterized_5ftypes_2eproto = { - false, false, 3934, descriptor_table_protodef_substrait_2fparameterized_5ftypes_2eproto, "substrait/parameterized_types.proto", - &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, descriptor_table_substrait_2fparameterized_5ftypes_2eproto_deps, 1, 16, + false, false, 4392, descriptor_table_protodef_substrait_2fparameterized_5ftypes_2eproto, "substrait/parameterized_types.proto", + &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, descriptor_table_substrait_2fparameterized_5ftypes_2eproto_deps, 1, 18, schemas, file_default_instances, TableStruct_substrait_2fparameterized_5ftypes_2eproto::offsets, file_level_metadata_substrait_2fparameterized_5ftypes_2eproto, file_level_enum_descriptors_substrait_2fparameterized_5ftypes_2eproto, file_level_service_descriptors_substrait_2fparameterized_5ftypes_2eproto, }; @@ -1795,98 +1859,651 @@ void ParameterizedType_ParameterizedVarChar::InternalSwap(ParameterizedType_Para ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedVarChar::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[4]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[4]); +} + +// =================================================================== + +class ParameterizedType_ParameterizedFixedBinary::_Internal { + public: + static const ::substrait::ParameterizedType_IntegerOption& length(const ParameterizedType_ParameterizedFixedBinary* msg); +}; + +const ::substrait::ParameterizedType_IntegerOption& +ParameterizedType_ParameterizedFixedBinary::_Internal::length(const ParameterizedType_ParameterizedFixedBinary* msg) { + return *msg->length_; +} +ParameterizedType_ParameterizedFixedBinary::ParameterizedType_ParameterizedFixedBinary(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedFixedBinary) +} +ParameterizedType_ParameterizedFixedBinary::ParameterizedType_ParameterizedFixedBinary(const ParameterizedType_ParameterizedFixedBinary& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + if (from._internal_has_length()) { + length_ = new ::substrait::ParameterizedType_IntegerOption(*from.length_); + } else { + length_ = nullptr; + } + ::memcpy(&variation_pointer_, &from.variation_pointer_, + static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedFixedBinary) +} + +inline void ParameterizedType_ParameterizedFixedBinary::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&length_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&length_)) + sizeof(nullability_)); +} + +ParameterizedType_ParameterizedFixedBinary::~ParameterizedType_ParameterizedFixedBinary() { + // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedFixedBinary) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void ParameterizedType_ParameterizedFixedBinary::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (this != internal_default_instance()) delete length_; +} + +void ParameterizedType_ParameterizedFixedBinary::ArenaDtor(void* object) { + ParameterizedType_ParameterizedFixedBinary* _this = reinterpret_cast< ParameterizedType_ParameterizedFixedBinary* >(object); + (void)_this; +} +void ParameterizedType_ParameterizedFixedBinary::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void ParameterizedType_ParameterizedFixedBinary::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void ParameterizedType_ParameterizedFixedBinary::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedFixedBinary) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaForAllocation() == nullptr && length_ != nullptr) { + delete length_; + } + length_ = nullptr; + ::memset(&variation_pointer_, 0, static_cast( + reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* ParameterizedType_ParameterizedFixedBinary::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // .substrait.ParameterizedType.IntegerOption length = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + ptr = ctx->ParseMessage(_internal_mutable_length(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint32 variation_pointer = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Type.Nullability nullability = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* ParameterizedType_ParameterizedFixedBinary::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedFixedBinary) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // .substrait.ParameterizedType.IntegerOption length = 1; + if (this->_internal_has_length()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 1, _Internal::length(this), target, stream); + } + + // uint32 variation_pointer = 2; + if (this->_internal_variation_pointer() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_variation_pointer(), target); + } + + // .substrait.Type.Nullability nullability = 3; + if (this->_internal_nullability() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 3, this->_internal_nullability(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedFixedBinary) + return target; +} + +size_t ParameterizedType_ParameterizedFixedBinary::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedFixedBinary) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .substrait.ParameterizedType.IntegerOption length = 1; + if (this->_internal_has_length()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *length_); + } + + // uint32 variation_pointer = 2; + if (this->_internal_variation_pointer() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); + } + + // .substrait.Type.Nullability nullability = 3; + if (this->_internal_nullability() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedFixedBinary::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + ParameterizedType_ParameterizedFixedBinary::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedFixedBinary::GetClassData() const { return &_class_data_; } + +void ParameterizedType_ParameterizedFixedBinary::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void ParameterizedType_ParameterizedFixedBinary::MergeFrom(const ParameterizedType_ParameterizedFixedBinary& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedFixedBinary) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_has_length()) { + _internal_mutable_length()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_length()); + } + if (from._internal_variation_pointer() != 0) { + _internal_set_variation_pointer(from._internal_variation_pointer()); + } + if (from._internal_nullability() != 0) { + _internal_set_nullability(from._internal_nullability()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void ParameterizedType_ParameterizedFixedBinary::CopyFrom(const ParameterizedType_ParameterizedFixedBinary& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedFixedBinary) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool ParameterizedType_ParameterizedFixedBinary::IsInitialized() const { + return true; +} + +void ParameterizedType_ParameterizedFixedBinary::InternalSwap(ParameterizedType_ParameterizedFixedBinary* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedFixedBinary, nullability_) + + sizeof(ParameterizedType_ParameterizedFixedBinary::nullability_) + - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedFixedBinary, length_)>( + reinterpret_cast(&length_), + reinterpret_cast(&other->length_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedFixedBinary::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[5]); +} + +// =================================================================== + +class ParameterizedType_ParameterizedDecimal::_Internal { + public: + static const ::substrait::ParameterizedType_IntegerOption& scale(const ParameterizedType_ParameterizedDecimal* msg); + static const ::substrait::ParameterizedType_IntegerOption& precision(const ParameterizedType_ParameterizedDecimal* msg); +}; + +const ::substrait::ParameterizedType_IntegerOption& +ParameterizedType_ParameterizedDecimal::_Internal::scale(const ParameterizedType_ParameterizedDecimal* msg) { + return *msg->scale_; +} +const ::substrait::ParameterizedType_IntegerOption& +ParameterizedType_ParameterizedDecimal::_Internal::precision(const ParameterizedType_ParameterizedDecimal* msg) { + return *msg->precision_; +} +ParameterizedType_ParameterizedDecimal::ParameterizedType_ParameterizedDecimal(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedDecimal) +} +ParameterizedType_ParameterizedDecimal::ParameterizedType_ParameterizedDecimal(const ParameterizedType_ParameterizedDecimal& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + if (from._internal_has_scale()) { + scale_ = new ::substrait::ParameterizedType_IntegerOption(*from.scale_); + } else { + scale_ = nullptr; + } + if (from._internal_has_precision()) { + precision_ = new ::substrait::ParameterizedType_IntegerOption(*from.precision_); + } else { + precision_ = nullptr; + } + ::memcpy(&variation_pointer_, &from.variation_pointer_, + static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedDecimal) +} + +inline void ParameterizedType_ParameterizedDecimal::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&scale_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&scale_)) + sizeof(nullability_)); +} + +ParameterizedType_ParameterizedDecimal::~ParameterizedType_ParameterizedDecimal() { + // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedDecimal) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void ParameterizedType_ParameterizedDecimal::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (this != internal_default_instance()) delete scale_; + if (this != internal_default_instance()) delete precision_; +} + +void ParameterizedType_ParameterizedDecimal::ArenaDtor(void* object) { + ParameterizedType_ParameterizedDecimal* _this = reinterpret_cast< ParameterizedType_ParameterizedDecimal* >(object); + (void)_this; +} +void ParameterizedType_ParameterizedDecimal::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void ParameterizedType_ParameterizedDecimal::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void ParameterizedType_ParameterizedDecimal::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedDecimal) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaForAllocation() == nullptr && scale_ != nullptr) { + delete scale_; + } + scale_ = nullptr; + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; + ::memset(&variation_pointer_, 0, static_cast( + reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* ParameterizedType_ParameterizedDecimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // .substrait.ParameterizedType.IntegerOption scale = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + ptr = ctx->ParseMessage(_internal_mutable_scale(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.ParameterizedType.IntegerOption precision = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint32 variation_pointer = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Type.Nullability nullability = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* ParameterizedType_ParameterizedDecimal::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedDecimal) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // .substrait.ParameterizedType.IntegerOption scale = 1; + if (this->_internal_has_scale()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 1, _Internal::scale(this), target, stream); + } + + // .substrait.ParameterizedType.IntegerOption precision = 2; + if (this->_internal_has_precision()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 2, _Internal::precision(this), target, stream); + } + + // uint32 variation_pointer = 3; + if (this->_internal_variation_pointer() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(3, this->_internal_variation_pointer(), target); + } + + // .substrait.Type.Nullability nullability = 4; + if (this->_internal_nullability() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 4, this->_internal_nullability(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedDecimal) + return target; +} + +size_t ParameterizedType_ParameterizedDecimal::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedDecimal) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .substrait.ParameterizedType.IntegerOption scale = 1; + if (this->_internal_has_scale()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *scale_); + } + + // .substrait.ParameterizedType.IntegerOption precision = 2; + if (this->_internal_has_precision()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *precision_); + } + + // uint32 variation_pointer = 3; + if (this->_internal_variation_pointer() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); + } + + // .substrait.Type.Nullability nullability = 4; + if (this->_internal_nullability() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedDecimal::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + ParameterizedType_ParameterizedDecimal::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedDecimal::GetClassData() const { return &_class_data_; } + +void ParameterizedType_ParameterizedDecimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void ParameterizedType_ParameterizedDecimal::MergeFrom(const ParameterizedType_ParameterizedDecimal& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedDecimal) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_has_scale()) { + _internal_mutable_scale()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_scale()); + } + if (from._internal_has_precision()) { + _internal_mutable_precision()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_precision()); + } + if (from._internal_variation_pointer() != 0) { + _internal_set_variation_pointer(from._internal_variation_pointer()); + } + if (from._internal_nullability() != 0) { + _internal_set_nullability(from._internal_nullability()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void ParameterizedType_ParameterizedDecimal::CopyFrom(const ParameterizedType_ParameterizedDecimal& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedDecimal) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool ParameterizedType_ParameterizedDecimal::IsInitialized() const { + return true; +} + +void ParameterizedType_ParameterizedDecimal::InternalSwap(ParameterizedType_ParameterizedDecimal* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedDecimal, nullability_) + + sizeof(ParameterizedType_ParameterizedDecimal::nullability_) + - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedDecimal, scale_)>( + reinterpret_cast(&scale_), + reinterpret_cast(&other->scale_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedDecimal::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[6]); } // =================================================================== -class ParameterizedType_ParameterizedFixedBinary::_Internal { +class ParameterizedType_ParameterizedIntervalDay::_Internal { public: - static const ::substrait::ParameterizedType_IntegerOption& length(const ParameterizedType_ParameterizedFixedBinary* msg); + static const ::substrait::ParameterizedType_IntegerOption& precision(const ParameterizedType_ParameterizedIntervalDay* msg); }; const ::substrait::ParameterizedType_IntegerOption& -ParameterizedType_ParameterizedFixedBinary::_Internal::length(const ParameterizedType_ParameterizedFixedBinary* msg) { - return *msg->length_; +ParameterizedType_ParameterizedIntervalDay::_Internal::precision(const ParameterizedType_ParameterizedIntervalDay* msg) { + return *msg->precision_; } -ParameterizedType_ParameterizedFixedBinary::ParameterizedType_ParameterizedFixedBinary(::PROTOBUF_NAMESPACE_ID::Arena* arena, +ParameterizedType_ParameterizedIntervalDay::ParameterizedType_ParameterizedIntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedFixedBinary) + // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedIntervalDay) } -ParameterizedType_ParameterizedFixedBinary::ParameterizedType_ParameterizedFixedBinary(const ParameterizedType_ParameterizedFixedBinary& from) +ParameterizedType_ParameterizedIntervalDay::ParameterizedType_ParameterizedIntervalDay(const ParameterizedType_ParameterizedIntervalDay& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); - if (from._internal_has_length()) { - length_ = new ::substrait::ParameterizedType_IntegerOption(*from.length_); + if (from._internal_has_precision()) { + precision_ = new ::substrait::ParameterizedType_IntegerOption(*from.precision_); } else { - length_ = nullptr; + precision_ = nullptr; } ::memcpy(&variation_pointer_, &from.variation_pointer_, static_cast(reinterpret_cast(&nullability_) - reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); - // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedFixedBinary) + // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedIntervalDay) } -inline void ParameterizedType_ParameterizedFixedBinary::SharedCtor() { +inline void ParameterizedType_ParameterizedIntervalDay::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( - reinterpret_cast(&length_) - reinterpret_cast(this)), + reinterpret_cast(&precision_) - reinterpret_cast(this)), 0, static_cast(reinterpret_cast(&nullability_) - - reinterpret_cast(&length_)) + sizeof(nullability_)); + reinterpret_cast(&precision_)) + sizeof(nullability_)); } -ParameterizedType_ParameterizedFixedBinary::~ParameterizedType_ParameterizedFixedBinary() { - // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedFixedBinary) +ParameterizedType_ParameterizedIntervalDay::~ParameterizedType_ParameterizedIntervalDay() { + // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedIntervalDay) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void ParameterizedType_ParameterizedFixedBinary::SharedDtor() { +inline void ParameterizedType_ParameterizedIntervalDay::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); - if (this != internal_default_instance()) delete length_; + if (this != internal_default_instance()) delete precision_; } -void ParameterizedType_ParameterizedFixedBinary::ArenaDtor(void* object) { - ParameterizedType_ParameterizedFixedBinary* _this = reinterpret_cast< ParameterizedType_ParameterizedFixedBinary* >(object); +void ParameterizedType_ParameterizedIntervalDay::ArenaDtor(void* object) { + ParameterizedType_ParameterizedIntervalDay* _this = reinterpret_cast< ParameterizedType_ParameterizedIntervalDay* >(object); (void)_this; } -void ParameterizedType_ParameterizedFixedBinary::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void ParameterizedType_ParameterizedIntervalDay::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void ParameterizedType_ParameterizedFixedBinary::SetCachedSize(int size) const { +void ParameterizedType_ParameterizedIntervalDay::SetCachedSize(int size) const { _cached_size_.Set(size); } -void ParameterizedType_ParameterizedFixedBinary::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedFixedBinary) +void ParameterizedType_ParameterizedIntervalDay::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedIntervalDay) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - if (GetArenaForAllocation() == nullptr && length_ != nullptr) { - delete length_; + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; } - length_ = nullptr; + precision_ = nullptr; ::memset(&variation_pointer_, 0, static_cast( reinterpret_cast(&nullability_) - reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* ParameterizedType_ParameterizedFixedBinary::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* ParameterizedType_ParameterizedIntervalDay::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); switch (tag >> 3) { - // .substrait.ParameterizedType.IntegerOption length = 1; + // .substrait.ParameterizedType.IntegerOption precision = 1; case 1: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { - ptr = ctx->ParseMessage(_internal_mutable_length(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); CHK_(ptr); } else goto handle_unusual; @@ -1931,18 +2548,18 @@ const char* ParameterizedType_ParameterizedFixedBinary::_InternalParse(const cha #undef CHK_ } -uint8_t* ParameterizedType_ParameterizedFixedBinary::_InternalSerialize( +uint8_t* ParameterizedType_ParameterizedIntervalDay::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedFixedBinary) + // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedIntervalDay) uint32_t cached_has_bits = 0; (void) cached_has_bits; - // .substrait.ParameterizedType.IntegerOption length = 1; - if (this->_internal_has_length()) { + // .substrait.ParameterizedType.IntegerOption precision = 1; + if (this->_internal_has_precision()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 1, _Internal::length(this), target, stream); + 1, _Internal::precision(this), target, stream); } // uint32 variation_pointer = 2; @@ -1962,23 +2579,23 @@ uint8_t* ParameterizedType_ParameterizedFixedBinary::_InternalSerialize( target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedFixedBinary) + // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedIntervalDay) return target; } -size_t ParameterizedType_ParameterizedFixedBinary::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedFixedBinary) +size_t ParameterizedType_ParameterizedIntervalDay::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedIntervalDay) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // .substrait.ParameterizedType.IntegerOption length = 1; - if (this->_internal_has_length()) { + // .substrait.ParameterizedType.IntegerOption precision = 1; + if (this->_internal_has_precision()) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *length_); + *precision_); } // uint32 variation_pointer = 2; @@ -1995,27 +2612,27 @@ size_t ParameterizedType_ParameterizedFixedBinary::ByteSizeLong() const { return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedFixedBinary::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedIntervalDay::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - ParameterizedType_ParameterizedFixedBinary::MergeImpl + ParameterizedType_ParameterizedIntervalDay::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedFixedBinary::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedIntervalDay::GetClassData() const { return &_class_data_; } -void ParameterizedType_ParameterizedFixedBinary::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void ParameterizedType_ParameterizedIntervalDay::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void ParameterizedType_ParameterizedFixedBinary::MergeFrom(const ParameterizedType_ParameterizedFixedBinary& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedFixedBinary) +void ParameterizedType_ParameterizedIntervalDay::MergeFrom(const ParameterizedType_ParameterizedIntervalDay& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedIntervalDay) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; - if (from._internal_has_length()) { - _internal_mutable_length()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_length()); + if (from._internal_has_precision()) { + _internal_mutable_precision()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_precision()); } if (from._internal_variation_pointer() != 0) { _internal_set_variation_pointer(from._internal_variation_pointer()); @@ -2026,67 +2643,57 @@ void ParameterizedType_ParameterizedFixedBinary::MergeFrom(const ParameterizedTy _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void ParameterizedType_ParameterizedFixedBinary::CopyFrom(const ParameterizedType_ParameterizedFixedBinary& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedFixedBinary) +void ParameterizedType_ParameterizedIntervalDay::CopyFrom(const ParameterizedType_ParameterizedIntervalDay& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedIntervalDay) if (&from == this) return; Clear(); MergeFrom(from); } -bool ParameterizedType_ParameterizedFixedBinary::IsInitialized() const { +bool ParameterizedType_ParameterizedIntervalDay::IsInitialized() const { return true; } -void ParameterizedType_ParameterizedFixedBinary::InternalSwap(ParameterizedType_ParameterizedFixedBinary* other) { +void ParameterizedType_ParameterizedIntervalDay::InternalSwap(ParameterizedType_ParameterizedIntervalDay* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedFixedBinary, nullability_) - + sizeof(ParameterizedType_ParameterizedFixedBinary::nullability_) - - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedFixedBinary, length_)>( - reinterpret_cast(&length_), - reinterpret_cast(&other->length_)); + PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedIntervalDay, nullability_) + + sizeof(ParameterizedType_ParameterizedIntervalDay::nullability_) + - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedIntervalDay, precision_)>( + reinterpret_cast(&precision_), + reinterpret_cast(&other->precision_)); } -::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedFixedBinary::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedIntervalDay::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[5]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[7]); } // =================================================================== -class ParameterizedType_ParameterizedDecimal::_Internal { +class ParameterizedType_ParameterizedIntervalCompound::_Internal { public: - static const ::substrait::ParameterizedType_IntegerOption& scale(const ParameterizedType_ParameterizedDecimal* msg); - static const ::substrait::ParameterizedType_IntegerOption& precision(const ParameterizedType_ParameterizedDecimal* msg); + static const ::substrait::ParameterizedType_IntegerOption& precision(const ParameterizedType_ParameterizedIntervalCompound* msg); }; const ::substrait::ParameterizedType_IntegerOption& -ParameterizedType_ParameterizedDecimal::_Internal::scale(const ParameterizedType_ParameterizedDecimal* msg) { - return *msg->scale_; -} -const ::substrait::ParameterizedType_IntegerOption& -ParameterizedType_ParameterizedDecimal::_Internal::precision(const ParameterizedType_ParameterizedDecimal* msg) { +ParameterizedType_ParameterizedIntervalCompound::_Internal::precision(const ParameterizedType_ParameterizedIntervalCompound* msg) { return *msg->precision_; } -ParameterizedType_ParameterizedDecimal::ParameterizedType_ParameterizedDecimal(::PROTOBUF_NAMESPACE_ID::Arena* arena, +ParameterizedType_ParameterizedIntervalCompound::ParameterizedType_ParameterizedIntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedDecimal) + // @@protoc_insertion_point(arena_constructor:substrait.ParameterizedType.ParameterizedIntervalCompound) } -ParameterizedType_ParameterizedDecimal::ParameterizedType_ParameterizedDecimal(const ParameterizedType_ParameterizedDecimal& from) +ParameterizedType_ParameterizedIntervalCompound::ParameterizedType_ParameterizedIntervalCompound(const ParameterizedType_ParameterizedIntervalCompound& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); - if (from._internal_has_scale()) { - scale_ = new ::substrait::ParameterizedType_IntegerOption(*from.scale_); - } else { - scale_ = nullptr; - } if (from._internal_has_precision()) { precision_ = new ::substrait::ParameterizedType_IntegerOption(*from.precision_); } else { @@ -2095,49 +2702,44 @@ ParameterizedType_ParameterizedDecimal::ParameterizedType_ParameterizedDecimal(c ::memcpy(&variation_pointer_, &from.variation_pointer_, static_cast(reinterpret_cast(&nullability_) - reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); - // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedDecimal) + // @@protoc_insertion_point(copy_constructor:substrait.ParameterizedType.ParameterizedIntervalCompound) } -inline void ParameterizedType_ParameterizedDecimal::SharedCtor() { +inline void ParameterizedType_ParameterizedIntervalCompound::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( - reinterpret_cast(&scale_) - reinterpret_cast(this)), + reinterpret_cast(&precision_) - reinterpret_cast(this)), 0, static_cast(reinterpret_cast(&nullability_) - - reinterpret_cast(&scale_)) + sizeof(nullability_)); + reinterpret_cast(&precision_)) + sizeof(nullability_)); } -ParameterizedType_ParameterizedDecimal::~ParameterizedType_ParameterizedDecimal() { - // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedDecimal) +ParameterizedType_ParameterizedIntervalCompound::~ParameterizedType_ParameterizedIntervalCompound() { + // @@protoc_insertion_point(destructor:substrait.ParameterizedType.ParameterizedIntervalCompound) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void ParameterizedType_ParameterizedDecimal::SharedDtor() { +inline void ParameterizedType_ParameterizedIntervalCompound::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); - if (this != internal_default_instance()) delete scale_; if (this != internal_default_instance()) delete precision_; } -void ParameterizedType_ParameterizedDecimal::ArenaDtor(void* object) { - ParameterizedType_ParameterizedDecimal* _this = reinterpret_cast< ParameterizedType_ParameterizedDecimal* >(object); +void ParameterizedType_ParameterizedIntervalCompound::ArenaDtor(void* object) { + ParameterizedType_ParameterizedIntervalCompound* _this = reinterpret_cast< ParameterizedType_ParameterizedIntervalCompound* >(object); (void)_this; } -void ParameterizedType_ParameterizedDecimal::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void ParameterizedType_ParameterizedIntervalCompound::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void ParameterizedType_ParameterizedDecimal::SetCachedSize(int size) const { +void ParameterizedType_ParameterizedIntervalCompound::SetCachedSize(int size) const { _cached_size_.Set(size); } -void ParameterizedType_ParameterizedDecimal::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedDecimal) +void ParameterizedType_ParameterizedIntervalCompound::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.ParameterizedType.ParameterizedIntervalCompound) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - if (GetArenaForAllocation() == nullptr && scale_ != nullptr) { - delete scale_; - } - scale_ = nullptr; if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { delete precision_; } @@ -2148,39 +2750,31 @@ void ParameterizedType_ParameterizedDecimal::Clear() { _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* ParameterizedType_ParameterizedDecimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* ParameterizedType_ParameterizedIntervalCompound::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); switch (tag >> 3) { - // .substrait.ParameterizedType.IntegerOption scale = 1; + // .substrait.ParameterizedType.IntegerOption precision = 1; case 1: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { - ptr = ctx->ParseMessage(_internal_mutable_scale(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); CHK_(ptr); } else goto handle_unusual; continue; - // .substrait.ParameterizedType.IntegerOption precision = 2; + // uint32 variation_pointer = 2; case 2: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { - ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); CHK_(ptr); } else goto handle_unusual; continue; - // uint32 variation_pointer = 3; + // .substrait.Type.Nullability nullability = 3; case 3: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { - variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // .substrait.Type.Nullability nullability = 4; - case 4: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); @@ -2210,77 +2804,62 @@ const char* ParameterizedType_ParameterizedDecimal::_InternalParse(const char* p #undef CHK_ } -uint8_t* ParameterizedType_ParameterizedDecimal::_InternalSerialize( +uint8_t* ParameterizedType_ParameterizedIntervalCompound::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedDecimal) + // @@protoc_insertion_point(serialize_to_array_start:substrait.ParameterizedType.ParameterizedIntervalCompound) uint32_t cached_has_bits = 0; (void) cached_has_bits; - // .substrait.ParameterizedType.IntegerOption scale = 1; - if (this->_internal_has_scale()) { - target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 1, _Internal::scale(this), target, stream); - } - - // .substrait.ParameterizedType.IntegerOption precision = 2; + // .substrait.ParameterizedType.IntegerOption precision = 1; if (this->_internal_has_precision()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 2, _Internal::precision(this), target, stream); + 1, _Internal::precision(this), target, stream); } - // uint32 variation_pointer = 3; + // uint32 variation_pointer = 2; if (this->_internal_variation_pointer() != 0) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(3, this->_internal_variation_pointer(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_variation_pointer(), target); } - // .substrait.Type.Nullability nullability = 4; + // .substrait.Type.Nullability nullability = 3; if (this->_internal_nullability() != 0) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( - 4, this->_internal_nullability(), target); + 3, this->_internal_nullability(), target); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedDecimal) + // @@protoc_insertion_point(serialize_to_array_end:substrait.ParameterizedType.ParameterizedIntervalCompound) return target; } -size_t ParameterizedType_ParameterizedDecimal::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedDecimal) +size_t ParameterizedType_ParameterizedIntervalCompound::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.ParameterizedType.ParameterizedIntervalCompound) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // .substrait.ParameterizedType.IntegerOption scale = 1; - if (this->_internal_has_scale()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *scale_); - } - - // .substrait.ParameterizedType.IntegerOption precision = 2; + // .substrait.ParameterizedType.IntegerOption precision = 1; if (this->_internal_has_precision()) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( *precision_); } - // uint32 variation_pointer = 3; + // uint32 variation_pointer = 2; if (this->_internal_variation_pointer() != 0) { total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); } - // .substrait.Type.Nullability nullability = 4; + // .substrait.Type.Nullability nullability = 3; if (this->_internal_nullability() != 0) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); @@ -2289,28 +2868,25 @@ size_t ParameterizedType_ParameterizedDecimal::ByteSizeLong() const { return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedDecimal::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData ParameterizedType_ParameterizedIntervalCompound::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - ParameterizedType_ParameterizedDecimal::MergeImpl + ParameterizedType_ParameterizedIntervalCompound::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedDecimal::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*ParameterizedType_ParameterizedIntervalCompound::GetClassData() const { return &_class_data_; } -void ParameterizedType_ParameterizedDecimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void ParameterizedType_ParameterizedIntervalCompound::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void ParameterizedType_ParameterizedDecimal::MergeFrom(const ParameterizedType_ParameterizedDecimal& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedDecimal) +void ParameterizedType_ParameterizedIntervalCompound::MergeFrom(const ParameterizedType_ParameterizedIntervalCompound& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.ParameterizedType.ParameterizedIntervalCompound) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; - if (from._internal_has_scale()) { - _internal_mutable_scale()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_scale()); - } if (from._internal_has_precision()) { _internal_mutable_precision()->::substrait::ParameterizedType_IntegerOption::MergeFrom(from._internal_precision()); } @@ -2323,32 +2899,32 @@ void ParameterizedType_ParameterizedDecimal::MergeFrom(const ParameterizedType_P _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void ParameterizedType_ParameterizedDecimal::CopyFrom(const ParameterizedType_ParameterizedDecimal& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedDecimal) +void ParameterizedType_ParameterizedIntervalCompound::CopyFrom(const ParameterizedType_ParameterizedIntervalCompound& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.ParameterizedType.ParameterizedIntervalCompound) if (&from == this) return; Clear(); MergeFrom(from); } -bool ParameterizedType_ParameterizedDecimal::IsInitialized() const { +bool ParameterizedType_ParameterizedIntervalCompound::IsInitialized() const { return true; } -void ParameterizedType_ParameterizedDecimal::InternalSwap(ParameterizedType_ParameterizedDecimal* other) { +void ParameterizedType_ParameterizedIntervalCompound::InternalSwap(ParameterizedType_ParameterizedIntervalCompound* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedDecimal, nullability_) - + sizeof(ParameterizedType_ParameterizedDecimal::nullability_) - - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedDecimal, scale_)>( - reinterpret_cast(&scale_), - reinterpret_cast(&other->scale_)); + PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedIntervalCompound, nullability_) + + sizeof(ParameterizedType_ParameterizedIntervalCompound::nullability_) + - PROTOBUF_FIELD_OFFSET(ParameterizedType_ParameterizedIntervalCompound, precision_)>( + reinterpret_cast(&precision_), + reinterpret_cast(&other->precision_)); } -::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedDecimal::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedIntervalCompound::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[6]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[8]); } // =================================================================== @@ -2604,7 +3180,7 @@ void ParameterizedType_ParameterizedPrecisionTimestamp::InternalSwap(Parameteriz ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedPrecisionTimestamp::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[7]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[9]); } // =================================================================== @@ -2860,7 +3436,7 @@ void ParameterizedType_ParameterizedPrecisionTimestampTZ::InternalSwap(Parameter ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedPrecisionTimestampTZ::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[8]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[10]); } // =================================================================== @@ -3108,7 +3684,7 @@ void ParameterizedType_ParameterizedStruct::InternalSwap(ParameterizedType_Param ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedStruct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[9]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[11]); } // =================================================================== @@ -3341,7 +3917,7 @@ void ParameterizedType_ParameterizedNamedStruct::InternalSwap(ParameterizedType_ ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedNamedStruct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[10]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[12]); } // =================================================================== @@ -3597,7 +4173,7 @@ void ParameterizedType_ParameterizedList::InternalSwap(ParameterizedType_Paramet ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedList::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[11]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[13]); } // =================================================================== @@ -3894,7 +4470,7 @@ void ParameterizedType_ParameterizedMap::InternalSwap(ParameterizedType_Paramete ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedMap::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[12]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[14]); } // =================================================================== @@ -4131,7 +4707,7 @@ void ParameterizedType_ParameterizedUserDefined::InternalSwap(ParameterizedType_ ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_ParameterizedUserDefined::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[13]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[15]); } // =================================================================== @@ -4405,7 +4981,7 @@ void ParameterizedType_IntegerOption::InternalSwap(ParameterizedType_IntegerOpti ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType_IntegerOption::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[14]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[16]); } // =================================================================== @@ -4425,7 +5001,8 @@ class ParameterizedType::_Internal { static const ::substrait::Type_Date& date(const ParameterizedType* msg); static const ::substrait::Type_Time& time(const ParameterizedType* msg); static const ::substrait::Type_IntervalYear& interval_year(const ParameterizedType* msg); - static const ::substrait::Type_IntervalDay& interval_day(const ParameterizedType* msg); + static const ::substrait::ParameterizedType_ParameterizedIntervalDay& interval_day(const ParameterizedType* msg); + static const ::substrait::ParameterizedType_ParameterizedIntervalCompound& interval_compound(const ParameterizedType* msg); static const ::substrait::Type_TimestampTZ& timestamp_tz(const ParameterizedType* msg); static const ::substrait::Type_UUID& uuid(const ParameterizedType* msg); static const ::substrait::ParameterizedType_ParameterizedFixedChar& fixed_char(const ParameterizedType* msg); @@ -4493,10 +5070,14 @@ const ::substrait::Type_IntervalYear& ParameterizedType::_Internal::interval_year(const ParameterizedType* msg) { return *msg->kind_.interval_year_; } -const ::substrait::Type_IntervalDay& +const ::substrait::ParameterizedType_ParameterizedIntervalDay& ParameterizedType::_Internal::interval_day(const ParameterizedType* msg) { return *msg->kind_.interval_day_; } +const ::substrait::ParameterizedType_ParameterizedIntervalCompound& +ParameterizedType::_Internal::interval_compound(const ParameterizedType* msg) { + return *msg->kind_.interval_compound_; +} const ::substrait::Type_TimestampTZ& ParameterizedType::_Internal::timestamp_tz(const ParameterizedType* msg) { return *msg->kind_.timestamp_tz_; @@ -4874,14 +5455,12 @@ void ParameterizedType::clear_interval_year() { clear_has_kind(); } } -void ParameterizedType::set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day) { +void ParameterizedType::set_allocated_interval_day(::substrait::ParameterizedType_ParameterizedIntervalDay* interval_day) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_kind(); if (interval_day) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper< - ::PROTOBUF_NAMESPACE_ID::MessageLite>::GetOwningArena( - reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(interval_day)); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ParameterizedType_ParameterizedIntervalDay>::GetOwningArena(interval_day); if (message_arena != submessage_arena) { interval_day = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( message_arena, interval_day, submessage_arena); @@ -4891,13 +5470,20 @@ void ParameterizedType::set_allocated_interval_day(::substrait::Type_IntervalDay } // @@protoc_insertion_point(field_set_allocated:substrait.ParameterizedType.interval_day) } -void ParameterizedType::clear_interval_day() { - if (_internal_has_interval_day()) { - if (GetArenaForAllocation() == nullptr) { - delete kind_.interval_day_; +void ParameterizedType::set_allocated_interval_compound(::substrait::ParameterizedType_ParameterizedIntervalCompound* interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_kind(); + if (interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ParameterizedType_ParameterizedIntervalCompound>::GetOwningArena(interval_compound); + if (message_arena != submessage_arena) { + interval_compound = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_compound, submessage_arena); } - clear_has_kind(); + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; } + // @@protoc_insertion_point(field_set_allocated:substrait.ParameterizedType.interval_compound) } void ParameterizedType::set_allocated_timestamp_tz(::substrait::Type_TimestampTZ* timestamp_tz) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); @@ -5181,7 +5767,11 @@ ParameterizedType::ParameterizedType(const ParameterizedType& from) break; } case kIntervalDay: { - _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); + _internal_mutable_interval_day()->::substrait::ParameterizedType_ParameterizedIntervalDay::MergeFrom(from._internal_interval_day()); + break; + } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::ParameterizedType_ParameterizedIntervalCompound::MergeFrom(from._internal_interval_compound()); break; } case kTimestampTz: { @@ -5362,6 +5952,12 @@ void ParameterizedType::clear_kind() { } break; } + case kIntervalCompound: { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + break; + } case kTimestampTz: { if (GetArenaForAllocation() == nullptr) { delete kind_.timestamp_tz_; @@ -5572,7 +6168,7 @@ const char* ParameterizedType::_InternalParse(const char* ptr, ::PROTOBUF_NAMESP } else goto handle_unusual; continue; - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.ParameterizedType.ParameterizedIntervalDay interval_day = 20; case 20: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 162)) { ptr = ctx->ParseMessage(_internal_mutable_interval_day(), ptr); @@ -5692,6 +6288,14 @@ const char* ParameterizedType::_InternalParse(const char* ptr, ::PROTOBUF_NAMESP } else goto handle_unusual; continue; + // .substrait.ParameterizedType.ParameterizedIntervalCompound interval_compound = 36; + case 36: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 34)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_compound(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; default: goto handle_unusual; } // switch @@ -5825,7 +6429,7 @@ uint8_t* ParameterizedType::_InternalSerialize( 19, _Internal::interval_year(this), target, stream); } - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.ParameterizedType.ParameterizedIntervalDay interval_day = 20; if (_internal_has_interval_day()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: @@ -5943,6 +6547,14 @@ uint8_t* ParameterizedType::_InternalSerialize( 35, _Internal::precision_timestamp_tz(this), target, stream); } + // .substrait.ParameterizedType.ParameterizedIntervalCompound interval_compound = 36; + if (_internal_has_interval_compound()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 36, _Internal::interval_compound(this), target, stream); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -6051,13 +6663,20 @@ size_t ParameterizedType::ByteSizeLong() const { *kind_.interval_year_); break; } - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.ParameterizedType.ParameterizedIntervalDay interval_day = 20; case kIntervalDay: { total_size += 2 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( *kind_.interval_day_); break; } + // .substrait.ParameterizedType.ParameterizedIntervalCompound interval_compound = 36; + case kIntervalCompound: { + total_size += 2 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *kind_.interval_compound_); + break; + } // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; case kTimestampTz: { total_size += 2 + @@ -6236,7 +6855,11 @@ void ParameterizedType::MergeFrom(const ParameterizedType& from) { break; } case kIntervalDay: { - _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); + _internal_mutable_interval_day()->::substrait::ParameterizedType_ParameterizedIntervalDay::MergeFrom(from._internal_interval_day()); + break; + } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::ParameterizedType_ParameterizedIntervalCompound::MergeFrom(from._internal_interval_compound()); break; } case kTimestampTz: { @@ -6323,7 +6946,7 @@ void ParameterizedType::InternalSwap(ParameterizedType* other) { ::PROTOBUF_NAMESPACE_ID::Metadata ParameterizedType::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_getter, &descriptor_table_substrait_2fparameterized_5ftypes_2eproto_once, - file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[15]); + file_level_metadata_substrait_2fparameterized_5ftypes_2eproto[17]); } // @@protoc_insertion_point(namespace_scope) @@ -6350,6 +6973,12 @@ template<> PROTOBUF_NOINLINE ::substrait::ParameterizedType_ParameterizedFixedBi template<> PROTOBUF_NOINLINE ::substrait::ParameterizedType_ParameterizedDecimal* Arena::CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedDecimal >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::ParameterizedType_ParameterizedDecimal >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::ParameterizedType_ParameterizedIntervalDay* Arena::CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedIntervalDay >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::ParameterizedType_ParameterizedIntervalDay >(arena); +} +template<> PROTOBUF_NOINLINE ::substrait::ParameterizedType_ParameterizedIntervalCompound* Arena::CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedIntervalCompound >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::ParameterizedType_ParameterizedIntervalCompound >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::ParameterizedType_ParameterizedPrecisionTimestamp* Arena::CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedPrecisionTimestamp >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::ParameterizedType_ParameterizedPrecisionTimestamp >(arena); } diff --git a/third_party/substrait/substrait/parameterized_types.pb.h b/third_party/substrait/substrait/parameterized_types.pb.h index bceb1c1..5e3c9ba 100644 --- a/third_party/substrait/substrait/parameterized_types.pb.h +++ b/third_party/substrait/substrait/parameterized_types.pb.h @@ -47,7 +47,7 @@ struct TableStruct_substrait_2fparameterized_5ftypes_2eproto { PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::AuxiliaryParseTableField aux[] PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[16] + static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[18] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[]; static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[]; @@ -76,6 +76,12 @@ extern ParameterizedType_ParameterizedFixedBinaryDefaultTypeInternal _Parameteri class ParameterizedType_ParameterizedFixedChar; struct ParameterizedType_ParameterizedFixedCharDefaultTypeInternal; extern ParameterizedType_ParameterizedFixedCharDefaultTypeInternal _ParameterizedType_ParameterizedFixedChar_default_instance_; +class ParameterizedType_ParameterizedIntervalCompound; +struct ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal; +extern ParameterizedType_ParameterizedIntervalCompoundDefaultTypeInternal _ParameterizedType_ParameterizedIntervalCompound_default_instance_; +class ParameterizedType_ParameterizedIntervalDay; +struct ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal; +extern ParameterizedType_ParameterizedIntervalDayDefaultTypeInternal _ParameterizedType_ParameterizedIntervalDay_default_instance_; class ParameterizedType_ParameterizedList; struct ParameterizedType_ParameterizedListDefaultTypeInternal; extern ParameterizedType_ParameterizedListDefaultTypeInternal _ParameterizedType_ParameterizedList_default_instance_; @@ -112,6 +118,8 @@ template<> ::substrait::ParameterizedType_NullableInteger* Arena::CreateMaybeMes template<> ::substrait::ParameterizedType_ParameterizedDecimal* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedDecimal>(Arena*); template<> ::substrait::ParameterizedType_ParameterizedFixedBinary* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedFixedBinary>(Arena*); template<> ::substrait::ParameterizedType_ParameterizedFixedChar* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedFixedChar>(Arena*); +template<> ::substrait::ParameterizedType_ParameterizedIntervalCompound* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedIntervalCompound>(Arena*); +template<> ::substrait::ParameterizedType_ParameterizedIntervalDay* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedIntervalDay>(Arena*); template<> ::substrait::ParameterizedType_ParameterizedList* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedList>(Arena*); template<> ::substrait::ParameterizedType_ParameterizedMap* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedMap>(Arena*); template<> ::substrait::ParameterizedType_ParameterizedNamedStruct* Arena::CreateMaybeMessage<::substrait::ParameterizedType_ParameterizedNamedStruct>(Arena*); @@ -1362,6 +1370,360 @@ class ParameterizedType_ParameterizedDecimal final : }; // ------------------------------------------------------------------- +class ParameterizedType_ParameterizedIntervalDay final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ParameterizedType.ParameterizedIntervalDay) */ { + public: + inline ParameterizedType_ParameterizedIntervalDay() : ParameterizedType_ParameterizedIntervalDay(nullptr) {} + ~ParameterizedType_ParameterizedIntervalDay() override; + explicit constexpr ParameterizedType_ParameterizedIntervalDay(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + ParameterizedType_ParameterizedIntervalDay(const ParameterizedType_ParameterizedIntervalDay& from); + ParameterizedType_ParameterizedIntervalDay(ParameterizedType_ParameterizedIntervalDay&& from) noexcept + : ParameterizedType_ParameterizedIntervalDay() { + *this = ::std::move(from); + } + + inline ParameterizedType_ParameterizedIntervalDay& operator=(const ParameterizedType_ParameterizedIntervalDay& from) { + CopyFrom(from); + return *this; + } + inline ParameterizedType_ParameterizedIntervalDay& operator=(ParameterizedType_ParameterizedIntervalDay&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const ParameterizedType_ParameterizedIntervalDay& default_instance() { + return *internal_default_instance(); + } + static inline const ParameterizedType_ParameterizedIntervalDay* internal_default_instance() { + return reinterpret_cast( + &_ParameterizedType_ParameterizedIntervalDay_default_instance_); + } + static constexpr int kIndexInFileMessages = + 7; + + friend void swap(ParameterizedType_ParameterizedIntervalDay& a, ParameterizedType_ParameterizedIntervalDay& b) { + a.Swap(&b); + } + inline void Swap(ParameterizedType_ParameterizedIntervalDay* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(ParameterizedType_ParameterizedIntervalDay* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + ParameterizedType_ParameterizedIntervalDay* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const ParameterizedType_ParameterizedIntervalDay& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const ParameterizedType_ParameterizedIntervalDay& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(ParameterizedType_ParameterizedIntervalDay* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.ParameterizedType.ParameterizedIntervalDay"; + } + protected: + explicit ParameterizedType_ParameterizedIntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kPrecisionFieldNumber = 1, + kVariationPointerFieldNumber = 2, + kNullabilityFieldNumber = 3, + }; + // .substrait.ParameterizedType.IntegerOption precision = 1; + bool has_precision() const; + private: + bool _internal_has_precision() const; + public: + void clear_precision(); + const ::substrait::ParameterizedType_IntegerOption& precision() const; + PROTOBUF_NODISCARD ::substrait::ParameterizedType_IntegerOption* release_precision(); + ::substrait::ParameterizedType_IntegerOption* mutable_precision(); + void set_allocated_precision(::substrait::ParameterizedType_IntegerOption* precision); + private: + const ::substrait::ParameterizedType_IntegerOption& _internal_precision() const; + ::substrait::ParameterizedType_IntegerOption* _internal_mutable_precision(); + public: + void unsafe_arena_set_allocated_precision( + ::substrait::ParameterizedType_IntegerOption* precision); + ::substrait::ParameterizedType_IntegerOption* unsafe_arena_release_precision(); + + // uint32 variation_pointer = 2; + void clear_variation_pointer(); + uint32_t variation_pointer() const; + void set_variation_pointer(uint32_t value); + private: + uint32_t _internal_variation_pointer() const; + void _internal_set_variation_pointer(uint32_t value); + public: + + // .substrait.Type.Nullability nullability = 3; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); + private: + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); + public: + + // @@protoc_insertion_point(class_scope:substrait.ParameterizedType.ParameterizedIntervalDay) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::substrait::ParameterizedType_IntegerOption* precision_; + uint32_t variation_pointer_; + int nullability_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2fparameterized_5ftypes_2eproto; +}; +// ------------------------------------------------------------------- + +class ParameterizedType_ParameterizedIntervalCompound final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ParameterizedType.ParameterizedIntervalCompound) */ { + public: + inline ParameterizedType_ParameterizedIntervalCompound() : ParameterizedType_ParameterizedIntervalCompound(nullptr) {} + ~ParameterizedType_ParameterizedIntervalCompound() override; + explicit constexpr ParameterizedType_ParameterizedIntervalCompound(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + ParameterizedType_ParameterizedIntervalCompound(const ParameterizedType_ParameterizedIntervalCompound& from); + ParameterizedType_ParameterizedIntervalCompound(ParameterizedType_ParameterizedIntervalCompound&& from) noexcept + : ParameterizedType_ParameterizedIntervalCompound() { + *this = ::std::move(from); + } + + inline ParameterizedType_ParameterizedIntervalCompound& operator=(const ParameterizedType_ParameterizedIntervalCompound& from) { + CopyFrom(from); + return *this; + } + inline ParameterizedType_ParameterizedIntervalCompound& operator=(ParameterizedType_ParameterizedIntervalCompound&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const ParameterizedType_ParameterizedIntervalCompound& default_instance() { + return *internal_default_instance(); + } + static inline const ParameterizedType_ParameterizedIntervalCompound* internal_default_instance() { + return reinterpret_cast( + &_ParameterizedType_ParameterizedIntervalCompound_default_instance_); + } + static constexpr int kIndexInFileMessages = + 8; + + friend void swap(ParameterizedType_ParameterizedIntervalCompound& a, ParameterizedType_ParameterizedIntervalCompound& b) { + a.Swap(&b); + } + inline void Swap(ParameterizedType_ParameterizedIntervalCompound* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(ParameterizedType_ParameterizedIntervalCompound* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + ParameterizedType_ParameterizedIntervalCompound* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const ParameterizedType_ParameterizedIntervalCompound& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const ParameterizedType_ParameterizedIntervalCompound& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(ParameterizedType_ParameterizedIntervalCompound* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.ParameterizedType.ParameterizedIntervalCompound"; + } + protected: + explicit ParameterizedType_ParameterizedIntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kPrecisionFieldNumber = 1, + kVariationPointerFieldNumber = 2, + kNullabilityFieldNumber = 3, + }; + // .substrait.ParameterizedType.IntegerOption precision = 1; + bool has_precision() const; + private: + bool _internal_has_precision() const; + public: + void clear_precision(); + const ::substrait::ParameterizedType_IntegerOption& precision() const; + PROTOBUF_NODISCARD ::substrait::ParameterizedType_IntegerOption* release_precision(); + ::substrait::ParameterizedType_IntegerOption* mutable_precision(); + void set_allocated_precision(::substrait::ParameterizedType_IntegerOption* precision); + private: + const ::substrait::ParameterizedType_IntegerOption& _internal_precision() const; + ::substrait::ParameterizedType_IntegerOption* _internal_mutable_precision(); + public: + void unsafe_arena_set_allocated_precision( + ::substrait::ParameterizedType_IntegerOption* precision); + ::substrait::ParameterizedType_IntegerOption* unsafe_arena_release_precision(); + + // uint32 variation_pointer = 2; + void clear_variation_pointer(); + uint32_t variation_pointer() const; + void set_variation_pointer(uint32_t value); + private: + uint32_t _internal_variation_pointer() const; + void _internal_set_variation_pointer(uint32_t value); + public: + + // .substrait.Type.Nullability nullability = 3; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); + private: + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); + public: + + // @@protoc_insertion_point(class_scope:substrait.ParameterizedType.ParameterizedIntervalCompound) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::substrait::ParameterizedType_IntegerOption* precision_; + uint32_t variation_pointer_; + int nullability_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2fparameterized_5ftypes_2eproto; +}; +// ------------------------------------------------------------------- + class ParameterizedType_ParameterizedPrecisionTimestamp final : public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.ParameterizedType.ParameterizedPrecisionTimestamp) */ { public: @@ -1410,7 +1772,7 @@ class ParameterizedType_ParameterizedPrecisionTimestamp final : &_ParameterizedType_ParameterizedPrecisionTimestamp_default_instance_); } static constexpr int kIndexInFileMessages = - 7; + 9; friend void swap(ParameterizedType_ParameterizedPrecisionTimestamp& a, ParameterizedType_ParameterizedPrecisionTimestamp& b) { a.Swap(&b); @@ -1587,7 +1949,7 @@ class ParameterizedType_ParameterizedPrecisionTimestampTZ final : &_ParameterizedType_ParameterizedPrecisionTimestampTZ_default_instance_); } static constexpr int kIndexInFileMessages = - 8; + 10; friend void swap(ParameterizedType_ParameterizedPrecisionTimestampTZ& a, ParameterizedType_ParameterizedPrecisionTimestampTZ& b) { a.Swap(&b); @@ -1764,7 +2126,7 @@ class ParameterizedType_ParameterizedStruct final : &_ParameterizedType_ParameterizedStruct_default_instance_); } static constexpr int kIndexInFileMessages = - 9; + 11; friend void swap(ParameterizedType_ParameterizedStruct& a, ParameterizedType_ParameterizedStruct& b) { a.Swap(&b); @@ -1941,7 +2303,7 @@ class ParameterizedType_ParameterizedNamedStruct final : &_ParameterizedType_ParameterizedNamedStruct_default_instance_); } static constexpr int kIndexInFileMessages = - 10; + 12; friend void swap(ParameterizedType_ParameterizedNamedStruct& a, ParameterizedType_ParameterizedNamedStruct& b) { a.Swap(&b); @@ -2122,7 +2484,7 @@ class ParameterizedType_ParameterizedList final : &_ParameterizedType_ParameterizedList_default_instance_); } static constexpr int kIndexInFileMessages = - 11; + 13; friend void swap(ParameterizedType_ParameterizedList& a, ParameterizedType_ParameterizedList& b) { a.Swap(&b); @@ -2299,7 +2661,7 @@ class ParameterizedType_ParameterizedMap final : &_ParameterizedType_ParameterizedMap_default_instance_); } static constexpr int kIndexInFileMessages = - 12; + 14; friend void swap(ParameterizedType_ParameterizedMap& a, ParameterizedType_ParameterizedMap& b) { a.Swap(&b); @@ -2496,7 +2858,7 @@ class ParameterizedType_ParameterizedUserDefined final : &_ParameterizedType_ParameterizedUserDefined_default_instance_); } static constexpr int kIndexInFileMessages = - 13; + 15; friend void swap(ParameterizedType_ParameterizedUserDefined& a, ParameterizedType_ParameterizedUserDefined& b) { a.Swap(&b); @@ -2670,7 +3032,7 @@ class ParameterizedType_IntegerOption final : &_ParameterizedType_IntegerOption_default_instance_); } static constexpr int kIndexInFileMessages = - 14; + 16; friend void swap(ParameterizedType_IntegerOption& a, ParameterizedType_IntegerOption& b) { a.Swap(&b); @@ -2863,6 +3225,7 @@ class ParameterizedType final : kTime = 17, kIntervalYear = 19, kIntervalDay = 20, + kIntervalCompound = 36, kTimestampTz = 29, kUuid = 32, kFixedChar = 21, @@ -2885,7 +3248,7 @@ class ParameterizedType final : &_ParameterizedType_default_instance_); } static constexpr int kIndexInFileMessages = - 15; + 17; friend void swap(ParameterizedType& a, ParameterizedType& b) { a.Swap(&b); @@ -2963,6 +3326,8 @@ class ParameterizedType final : typedef ParameterizedType_ParameterizedVarChar ParameterizedVarChar; typedef ParameterizedType_ParameterizedFixedBinary ParameterizedFixedBinary; typedef ParameterizedType_ParameterizedDecimal ParameterizedDecimal; + typedef ParameterizedType_ParameterizedIntervalDay ParameterizedIntervalDay; + typedef ParameterizedType_ParameterizedIntervalCompound ParameterizedIntervalCompound; typedef ParameterizedType_ParameterizedPrecisionTimestamp ParameterizedPrecisionTimestamp; typedef ParameterizedType_ParameterizedPrecisionTimestampTZ ParameterizedPrecisionTimestampTZ; typedef ParameterizedType_ParameterizedStruct ParameterizedStruct; @@ -2989,6 +3354,7 @@ class ParameterizedType final : kTimeFieldNumber = 17, kIntervalYearFieldNumber = 19, kIntervalDayFieldNumber = 20, + kIntervalCompoundFieldNumber = 36, kTimestampTzFieldNumber = 29, kUuidFieldNumber = 32, kFixedCharFieldNumber = 21, @@ -3238,23 +3604,41 @@ class ParameterizedType final : ::substrait::Type_IntervalYear* interval_year); ::substrait::Type_IntervalYear* unsafe_arena_release_interval_year(); - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.ParameterizedType.ParameterizedIntervalDay interval_day = 20; bool has_interval_day() const; private: bool _internal_has_interval_day() const; public: void clear_interval_day(); - const ::substrait::Type_IntervalDay& interval_day() const; - PROTOBUF_NODISCARD ::substrait::Type_IntervalDay* release_interval_day(); - ::substrait::Type_IntervalDay* mutable_interval_day(); - void set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day); + const ::substrait::ParameterizedType_ParameterizedIntervalDay& interval_day() const; + PROTOBUF_NODISCARD ::substrait::ParameterizedType_ParameterizedIntervalDay* release_interval_day(); + ::substrait::ParameterizedType_ParameterizedIntervalDay* mutable_interval_day(); + void set_allocated_interval_day(::substrait::ParameterizedType_ParameterizedIntervalDay* interval_day); private: - const ::substrait::Type_IntervalDay& _internal_interval_day() const; - ::substrait::Type_IntervalDay* _internal_mutable_interval_day(); + const ::substrait::ParameterizedType_ParameterizedIntervalDay& _internal_interval_day() const; + ::substrait::ParameterizedType_ParameterizedIntervalDay* _internal_mutable_interval_day(); public: void unsafe_arena_set_allocated_interval_day( - ::substrait::Type_IntervalDay* interval_day); - ::substrait::Type_IntervalDay* unsafe_arena_release_interval_day(); + ::substrait::ParameterizedType_ParameterizedIntervalDay* interval_day); + ::substrait::ParameterizedType_ParameterizedIntervalDay* unsafe_arena_release_interval_day(); + + // .substrait.ParameterizedType.ParameterizedIntervalCompound interval_compound = 36; + bool has_interval_compound() const; + private: + bool _internal_has_interval_compound() const; + public: + void clear_interval_compound(); + const ::substrait::ParameterizedType_ParameterizedIntervalCompound& interval_compound() const; + PROTOBUF_NODISCARD ::substrait::ParameterizedType_ParameterizedIntervalCompound* release_interval_compound(); + ::substrait::ParameterizedType_ParameterizedIntervalCompound* mutable_interval_compound(); + void set_allocated_interval_compound(::substrait::ParameterizedType_ParameterizedIntervalCompound* interval_compound); + private: + const ::substrait::ParameterizedType_ParameterizedIntervalCompound& _internal_interval_compound() const; + ::substrait::ParameterizedType_ParameterizedIntervalCompound* _internal_mutable_interval_compound(); + public: + void unsafe_arena_set_allocated_interval_compound( + ::substrait::ParameterizedType_ParameterizedIntervalCompound* interval_compound); + ::substrait::ParameterizedType_ParameterizedIntervalCompound* unsafe_arena_release_interval_compound(); // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; PROTOBUF_DEPRECATED bool has_timestamp_tz() const; @@ -3522,6 +3906,7 @@ class ParameterizedType final : void set_has_time(); void set_has_interval_year(); void set_has_interval_day(); + void set_has_interval_compound(); void set_has_timestamp_tz(); void set_has_uuid(); void set_has_fixed_char(); @@ -3559,7 +3944,8 @@ class ParameterizedType final : ::substrait::Type_Date* date_; ::substrait::Type_Time* time_; ::substrait::Type_IntervalYear* interval_year_; - ::substrait::Type_IntervalDay* interval_day_; + ::substrait::ParameterizedType_ParameterizedIntervalDay* interval_day_; + ::substrait::ParameterizedType_ParameterizedIntervalCompound* interval_compound_; ::substrait::Type_TimestampTZ* timestamp_tz_; ::substrait::Type_UUID* uuid_; ::substrait::ParameterizedType_ParameterizedFixedChar* fixed_char_; @@ -4569,6 +4955,274 @@ inline void ParameterizedType_ParameterizedDecimal::set_nullability(::substrait: // ------------------------------------------------------------------- +// ParameterizedType_ParameterizedIntervalDay + +// .substrait.ParameterizedType.IntegerOption precision = 1; +inline bool ParameterizedType_ParameterizedIntervalDay::_internal_has_precision() const { + return this != internal_default_instance() && precision_ != nullptr; +} +inline bool ParameterizedType_ParameterizedIntervalDay::has_precision() const { + return _internal_has_precision(); +} +inline void ParameterizedType_ParameterizedIntervalDay::clear_precision() { + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; +} +inline const ::substrait::ParameterizedType_IntegerOption& ParameterizedType_ParameterizedIntervalDay::_internal_precision() const { + const ::substrait::ParameterizedType_IntegerOption* p = precision_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_ParameterizedType_IntegerOption_default_instance_); +} +inline const ::substrait::ParameterizedType_IntegerOption& ParameterizedType_ParameterizedIntervalDay::precision() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalDay.precision) + return _internal_precision(); +} +inline void ParameterizedType_ParameterizedIntervalDay::unsafe_arena_set_allocated_precision( + ::substrait::ParameterizedType_IntegerOption* precision) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); + } + precision_ = precision; + if (precision) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.ParameterizedType.ParameterizedIntervalDay.precision) +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalDay::release_precision() { + + ::substrait::ParameterizedType_IntegerOption* temp = precision_; + precision_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalDay::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.ParameterizedType.ParameterizedIntervalDay.precision) + + ::substrait::ParameterizedType_IntegerOption* temp = precision_; + precision_ = nullptr; + return temp; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalDay::_internal_mutable_precision() { + + if (precision_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::ParameterizedType_IntegerOption>(GetArenaForAllocation()); + precision_ = p; + } + return precision_; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalDay::mutable_precision() { + ::substrait::ParameterizedType_IntegerOption* _msg = _internal_mutable_precision(); + // @@protoc_insertion_point(field_mutable:substrait.ParameterizedType.ParameterizedIntervalDay.precision) + return _msg; +} +inline void ParameterizedType_ParameterizedIntervalDay::set_allocated_precision(::substrait::ParameterizedType_IntegerOption* precision) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete precision_; + } + if (precision) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ParameterizedType_IntegerOption>::GetOwningArena(precision); + if (message_arena != submessage_arena) { + precision = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision, submessage_arena); + } + + } else { + + } + precision_ = precision; + // @@protoc_insertion_point(field_set_allocated:substrait.ParameterizedType.ParameterizedIntervalDay.precision) +} + +// uint32 variation_pointer = 2; +inline void ParameterizedType_ParameterizedIntervalDay::clear_variation_pointer() { + variation_pointer_ = 0u; +} +inline uint32_t ParameterizedType_ParameterizedIntervalDay::_internal_variation_pointer() const { + return variation_pointer_; +} +inline uint32_t ParameterizedType_ParameterizedIntervalDay::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalDay.variation_pointer) + return _internal_variation_pointer(); +} +inline void ParameterizedType_ParameterizedIntervalDay::_internal_set_variation_pointer(uint32_t value) { + + variation_pointer_ = value; +} +inline void ParameterizedType_ParameterizedIntervalDay::set_variation_pointer(uint32_t value) { + _internal_set_variation_pointer(value); + // @@protoc_insertion_point(field_set:substrait.ParameterizedType.ParameterizedIntervalDay.variation_pointer) +} + +// .substrait.Type.Nullability nullability = 3; +inline void ParameterizedType_ParameterizedIntervalDay::clear_nullability() { + nullability_ = 0; +} +inline ::substrait::Type_Nullability ParameterizedType_ParameterizedIntervalDay::_internal_nullability() const { + return static_cast< ::substrait::Type_Nullability >(nullability_); +} +inline ::substrait::Type_Nullability ParameterizedType_ParameterizedIntervalDay::nullability() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalDay.nullability) + return _internal_nullability(); +} +inline void ParameterizedType_ParameterizedIntervalDay::_internal_set_nullability(::substrait::Type_Nullability value) { + + nullability_ = value; +} +inline void ParameterizedType_ParameterizedIntervalDay::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.ParameterizedType.ParameterizedIntervalDay.nullability) +} + +// ------------------------------------------------------------------- + +// ParameterizedType_ParameterizedIntervalCompound + +// .substrait.ParameterizedType.IntegerOption precision = 1; +inline bool ParameterizedType_ParameterizedIntervalCompound::_internal_has_precision() const { + return this != internal_default_instance() && precision_ != nullptr; +} +inline bool ParameterizedType_ParameterizedIntervalCompound::has_precision() const { + return _internal_has_precision(); +} +inline void ParameterizedType_ParameterizedIntervalCompound::clear_precision() { + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; +} +inline const ::substrait::ParameterizedType_IntegerOption& ParameterizedType_ParameterizedIntervalCompound::_internal_precision() const { + const ::substrait::ParameterizedType_IntegerOption* p = precision_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_ParameterizedType_IntegerOption_default_instance_); +} +inline const ::substrait::ParameterizedType_IntegerOption& ParameterizedType_ParameterizedIntervalCompound::precision() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalCompound.precision) + return _internal_precision(); +} +inline void ParameterizedType_ParameterizedIntervalCompound::unsafe_arena_set_allocated_precision( + ::substrait::ParameterizedType_IntegerOption* precision) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); + } + precision_ = precision; + if (precision) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.ParameterizedType.ParameterizedIntervalCompound.precision) +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalCompound::release_precision() { + + ::substrait::ParameterizedType_IntegerOption* temp = precision_; + precision_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalCompound::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.ParameterizedType.ParameterizedIntervalCompound.precision) + + ::substrait::ParameterizedType_IntegerOption* temp = precision_; + precision_ = nullptr; + return temp; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalCompound::_internal_mutable_precision() { + + if (precision_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::ParameterizedType_IntegerOption>(GetArenaForAllocation()); + precision_ = p; + } + return precision_; +} +inline ::substrait::ParameterizedType_IntegerOption* ParameterizedType_ParameterizedIntervalCompound::mutable_precision() { + ::substrait::ParameterizedType_IntegerOption* _msg = _internal_mutable_precision(); + // @@protoc_insertion_point(field_mutable:substrait.ParameterizedType.ParameterizedIntervalCompound.precision) + return _msg; +} +inline void ParameterizedType_ParameterizedIntervalCompound::set_allocated_precision(::substrait::ParameterizedType_IntegerOption* precision) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete precision_; + } + if (precision) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::ParameterizedType_IntegerOption>::GetOwningArena(precision); + if (message_arena != submessage_arena) { + precision = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision, submessage_arena); + } + + } else { + + } + precision_ = precision; + // @@protoc_insertion_point(field_set_allocated:substrait.ParameterizedType.ParameterizedIntervalCompound.precision) +} + +// uint32 variation_pointer = 2; +inline void ParameterizedType_ParameterizedIntervalCompound::clear_variation_pointer() { + variation_pointer_ = 0u; +} +inline uint32_t ParameterizedType_ParameterizedIntervalCompound::_internal_variation_pointer() const { + return variation_pointer_; +} +inline uint32_t ParameterizedType_ParameterizedIntervalCompound::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalCompound.variation_pointer) + return _internal_variation_pointer(); +} +inline void ParameterizedType_ParameterizedIntervalCompound::_internal_set_variation_pointer(uint32_t value) { + + variation_pointer_ = value; +} +inline void ParameterizedType_ParameterizedIntervalCompound::set_variation_pointer(uint32_t value) { + _internal_set_variation_pointer(value); + // @@protoc_insertion_point(field_set:substrait.ParameterizedType.ParameterizedIntervalCompound.variation_pointer) +} + +// .substrait.Type.Nullability nullability = 3; +inline void ParameterizedType_ParameterizedIntervalCompound::clear_nullability() { + nullability_ = 0; +} +inline ::substrait::Type_Nullability ParameterizedType_ParameterizedIntervalCompound::_internal_nullability() const { + return static_cast< ::substrait::Type_Nullability >(nullability_); +} +inline ::substrait::Type_Nullability ParameterizedType_ParameterizedIntervalCompound::nullability() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.ParameterizedIntervalCompound.nullability) + return _internal_nullability(); +} +inline void ParameterizedType_ParameterizedIntervalCompound::_internal_set_nullability(::substrait::Type_Nullability value) { + + nullability_ = value; +} +inline void ParameterizedType_ParameterizedIntervalCompound::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.ParameterizedType.ParameterizedIntervalCompound.nullability) +} + +// ------------------------------------------------------------------- + // ParameterizedType_ParameterizedPrecisionTimestamp // .substrait.ParameterizedType.IntegerOption precision = 1; @@ -6497,7 +7151,7 @@ inline ::substrait::Type_IntervalYear* ParameterizedType::mutable_interval_year( return _msg; } -// .substrait.Type.IntervalDay interval_day = 20; +// .substrait.ParameterizedType.ParameterizedIntervalDay interval_day = 20; inline bool ParameterizedType::_internal_has_interval_day() const { return kind_case() == kIntervalDay; } @@ -6507,11 +7161,19 @@ inline bool ParameterizedType::has_interval_day() const { inline void ParameterizedType::set_has_interval_day() { _oneof_case_[0] = kIntervalDay; } -inline ::substrait::Type_IntervalDay* ParameterizedType::release_interval_day() { +inline void ParameterizedType::clear_interval_day() { + if (_internal_has_interval_day()) { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_day_; + } + clear_has_kind(); + } +} +inline ::substrait::ParameterizedType_ParameterizedIntervalDay* ParameterizedType::release_interval_day() { // @@protoc_insertion_point(field_release:substrait.ParameterizedType.interval_day) if (_internal_has_interval_day()) { clear_has_kind(); - ::substrait::Type_IntervalDay* temp = kind_.interval_day_; + ::substrait::ParameterizedType_ParameterizedIntervalDay* temp = kind_.interval_day_; if (GetArenaForAllocation() != nullptr) { temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); } @@ -6521,27 +7183,27 @@ inline ::substrait::Type_IntervalDay* ParameterizedType::release_interval_day() return nullptr; } } -inline const ::substrait::Type_IntervalDay& ParameterizedType::_internal_interval_day() const { +inline const ::substrait::ParameterizedType_ParameterizedIntervalDay& ParameterizedType::_internal_interval_day() const { return _internal_has_interval_day() ? *kind_.interval_day_ - : reinterpret_cast< ::substrait::Type_IntervalDay&>(::substrait::_Type_IntervalDay_default_instance_); + : reinterpret_cast< ::substrait::ParameterizedType_ParameterizedIntervalDay&>(::substrait::_ParameterizedType_ParameterizedIntervalDay_default_instance_); } -inline const ::substrait::Type_IntervalDay& ParameterizedType::interval_day() const { +inline const ::substrait::ParameterizedType_ParameterizedIntervalDay& ParameterizedType::interval_day() const { // @@protoc_insertion_point(field_get:substrait.ParameterizedType.interval_day) return _internal_interval_day(); } -inline ::substrait::Type_IntervalDay* ParameterizedType::unsafe_arena_release_interval_day() { +inline ::substrait::ParameterizedType_ParameterizedIntervalDay* ParameterizedType::unsafe_arena_release_interval_day() { // @@protoc_insertion_point(field_unsafe_arena_release:substrait.ParameterizedType.interval_day) if (_internal_has_interval_day()) { clear_has_kind(); - ::substrait::Type_IntervalDay* temp = kind_.interval_day_; + ::substrait::ParameterizedType_ParameterizedIntervalDay* temp = kind_.interval_day_; kind_.interval_day_ = nullptr; return temp; } else { return nullptr; } } -inline void ParameterizedType::unsafe_arena_set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day) { +inline void ParameterizedType::unsafe_arena_set_allocated_interval_day(::substrait::ParameterizedType_ParameterizedIntervalDay* interval_day) { clear_kind(); if (interval_day) { set_has_interval_day(); @@ -6549,20 +7211,94 @@ inline void ParameterizedType::unsafe_arena_set_allocated_interval_day(::substra } // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.ParameterizedType.interval_day) } -inline ::substrait::Type_IntervalDay* ParameterizedType::_internal_mutable_interval_day() { +inline ::substrait::ParameterizedType_ParameterizedIntervalDay* ParameterizedType::_internal_mutable_interval_day() { if (!_internal_has_interval_day()) { clear_kind(); set_has_interval_day(); - kind_.interval_day_ = CreateMaybeMessage< ::substrait::Type_IntervalDay >(GetArenaForAllocation()); + kind_.interval_day_ = CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedIntervalDay >(GetArenaForAllocation()); } return kind_.interval_day_; } -inline ::substrait::Type_IntervalDay* ParameterizedType::mutable_interval_day() { - ::substrait::Type_IntervalDay* _msg = _internal_mutable_interval_day(); +inline ::substrait::ParameterizedType_ParameterizedIntervalDay* ParameterizedType::mutable_interval_day() { + ::substrait::ParameterizedType_ParameterizedIntervalDay* _msg = _internal_mutable_interval_day(); // @@protoc_insertion_point(field_mutable:substrait.ParameterizedType.interval_day) return _msg; } +// .substrait.ParameterizedType.ParameterizedIntervalCompound interval_compound = 36; +inline bool ParameterizedType::_internal_has_interval_compound() const { + return kind_case() == kIntervalCompound; +} +inline bool ParameterizedType::has_interval_compound() const { + return _internal_has_interval_compound(); +} +inline void ParameterizedType::set_has_interval_compound() { + _oneof_case_[0] = kIntervalCompound; +} +inline void ParameterizedType::clear_interval_compound() { + if (_internal_has_interval_compound()) { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + clear_has_kind(); + } +} +inline ::substrait::ParameterizedType_ParameterizedIntervalCompound* ParameterizedType::release_interval_compound() { + // @@protoc_insertion_point(field_release:substrait.ParameterizedType.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::ParameterizedType_ParameterizedIntervalCompound* temp = kind_.interval_compound_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::ParameterizedType_ParameterizedIntervalCompound& ParameterizedType::_internal_interval_compound() const { + return _internal_has_interval_compound() + ? *kind_.interval_compound_ + : reinterpret_cast< ::substrait::ParameterizedType_ParameterizedIntervalCompound&>(::substrait::_ParameterizedType_ParameterizedIntervalCompound_default_instance_); +} +inline const ::substrait::ParameterizedType_ParameterizedIntervalCompound& ParameterizedType::interval_compound() const { + // @@protoc_insertion_point(field_get:substrait.ParameterizedType.interval_compound) + return _internal_interval_compound(); +} +inline ::substrait::ParameterizedType_ParameterizedIntervalCompound* ParameterizedType::unsafe_arena_release_interval_compound() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.ParameterizedType.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::ParameterizedType_ParameterizedIntervalCompound* temp = kind_.interval_compound_; + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void ParameterizedType::unsafe_arena_set_allocated_interval_compound(::substrait::ParameterizedType_ParameterizedIntervalCompound* interval_compound) { + clear_kind(); + if (interval_compound) { + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.ParameterizedType.interval_compound) +} +inline ::substrait::ParameterizedType_ParameterizedIntervalCompound* ParameterizedType::_internal_mutable_interval_compound() { + if (!_internal_has_interval_compound()) { + clear_kind(); + set_has_interval_compound(); + kind_.interval_compound_ = CreateMaybeMessage< ::substrait::ParameterizedType_ParameterizedIntervalCompound >(GetArenaForAllocation()); + } + return kind_.interval_compound_; +} +inline ::substrait::ParameterizedType_ParameterizedIntervalCompound* ParameterizedType::mutable_interval_compound() { + ::substrait::ParameterizedType_ParameterizedIntervalCompound* _msg = _internal_mutable_interval_compound(); + // @@protoc_insertion_point(field_mutable:substrait.ParameterizedType.interval_compound) + return _msg; +} + // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; inline bool ParameterizedType::_internal_has_timestamp_tz() const { return kind_case() == kTimestampTz; @@ -7589,6 +8325,10 @@ inline ParameterizedType::KindCase ParameterizedType::kind_case() const { // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) diff --git a/third_party/substrait/substrait/type.pb.cc b/third_party/substrait/substrait/type.pb.cc index 865608e..80a6124 100644 --- a/third_party/substrait/substrait/type.pb.cc +++ b/third_party/substrait/substrait/type.pb.cc @@ -217,7 +217,8 @@ constexpr Type_IntervalDay::Type_IntervalDay( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : type_variation_reference_(0u) , nullability_(0) -{} + + , precision_(0){} struct Type_IntervalDayDefaultTypeInternal { constexpr Type_IntervalDayDefaultTypeInternal() : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} @@ -227,6 +228,21 @@ struct Type_IntervalDayDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Type_IntervalDayDefaultTypeInternal _Type_IntervalDay_default_instance_; +constexpr Type_IntervalCompound::Type_IntervalCompound( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : type_variation_reference_(0u) + , nullability_(0) + + , precision_(0){} +struct Type_IntervalCompoundDefaultTypeInternal { + constexpr Type_IntervalCompoundDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~Type_IntervalCompoundDefaultTypeInternal() {} + union { + Type_IntervalCompound _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT Type_IntervalCompoundDefaultTypeInternal _Type_IntervalCompound_default_instance_; constexpr Type_UUID::Type_UUID( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : type_variation_reference_(0u) @@ -432,7 +448,7 @@ struct NamedStructDefaultTypeInternal { }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT NamedStructDefaultTypeInternal _NamedStruct_default_instance_; } // namespace substrait -static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2ftype_2eproto[29]; +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2ftype_2eproto[30]; static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_substrait_2ftype_2eproto[1]; static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_substrait_2ftype_2eproto = nullptr; @@ -549,7 +565,7 @@ const uint32_t TableStruct_substrait_2ftype_2eproto::offsets[] PROTOBUF_SECTION_ ~0u, // no _inlined_string_donated_ PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalYear, type_variation_reference_), PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalYear, nullability_), - ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalDay, _has_bits_), PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalDay, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ @@ -557,6 +573,19 @@ const uint32_t TableStruct_substrait_2ftype_2eproto::offsets[] PROTOBUF_SECTION_ ~0u, // no _inlined_string_donated_ PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalDay, type_variation_reference_), PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalDay, nullability_), + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalDay, precision_), + ~0u, + ~0u, + 0, + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalCompound, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalCompound, type_variation_reference_), + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalCompound, nullability_), + PROTOBUF_FIELD_OFFSET(::substrait::Type_IntervalCompound, precision_), ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::Type_UUID, _internal_metadata_), ~0u, // no _extensions_ @@ -704,6 +733,7 @@ const uint32_t TableStruct_substrait_2ftype_2eproto::offsets[] PROTOBUF_SECTION_ ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, PROTOBUF_FIELD_OFFSET(::substrait::Type, kind_), ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::NamedStruct, _internal_metadata_), @@ -729,21 +759,22 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOB { 88, -1, -1, sizeof(::substrait::Type_Time)}, { 96, -1, -1, sizeof(::substrait::Type_TimestampTZ)}, { 104, -1, -1, sizeof(::substrait::Type_IntervalYear)}, - { 112, -1, -1, sizeof(::substrait::Type_IntervalDay)}, - { 120, -1, -1, sizeof(::substrait::Type_UUID)}, - { 128, -1, -1, sizeof(::substrait::Type_FixedChar)}, - { 137, -1, -1, sizeof(::substrait::Type_VarChar)}, - { 146, -1, -1, sizeof(::substrait::Type_FixedBinary)}, - { 155, -1, -1, sizeof(::substrait::Type_Decimal)}, - { 165, -1, -1, sizeof(::substrait::Type_PrecisionTimestamp)}, - { 174, -1, -1, sizeof(::substrait::Type_PrecisionTimestampTZ)}, - { 183, -1, -1, sizeof(::substrait::Type_Struct)}, - { 192, -1, -1, sizeof(::substrait::Type_List)}, - { 201, -1, -1, sizeof(::substrait::Type_Map)}, - { 211, -1, -1, sizeof(::substrait::Type_UserDefined)}, - { 221, -1, -1, sizeof(::substrait::Type_Parameter)}, - { 234, -1, -1, sizeof(::substrait::Type)}, - { 268, -1, -1, sizeof(::substrait::NamedStruct)}, + { 112, 121, -1, sizeof(::substrait::Type_IntervalDay)}, + { 124, -1, -1, sizeof(::substrait::Type_IntervalCompound)}, + { 133, -1, -1, sizeof(::substrait::Type_UUID)}, + { 141, -1, -1, sizeof(::substrait::Type_FixedChar)}, + { 150, -1, -1, sizeof(::substrait::Type_VarChar)}, + { 159, -1, -1, sizeof(::substrait::Type_FixedBinary)}, + { 168, -1, -1, sizeof(::substrait::Type_Decimal)}, + { 178, -1, -1, sizeof(::substrait::Type_PrecisionTimestamp)}, + { 187, -1, -1, sizeof(::substrait::Type_PrecisionTimestampTZ)}, + { 196, -1, -1, sizeof(::substrait::Type_Struct)}, + { 205, -1, -1, sizeof(::substrait::Type_List)}, + { 214, -1, -1, sizeof(::substrait::Type_Map)}, + { 224, -1, -1, sizeof(::substrait::Type_UserDefined)}, + { 234, -1, -1, sizeof(::substrait::Type_Parameter)}, + { 247, -1, -1, sizeof(::substrait::Type)}, + { 282, -1, -1, sizeof(::substrait::NamedStruct)}, }; static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { @@ -762,6 +793,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = reinterpret_cast(&::substrait::_Type_TimestampTZ_default_instance_), reinterpret_cast(&::substrait::_Type_IntervalYear_default_instance_), reinterpret_cast(&::substrait::_Type_IntervalDay_default_instance_), + reinterpret_cast(&::substrait::_Type_IntervalCompound_default_instance_), reinterpret_cast(&::substrait::_Type_UUID_default_instance_), reinterpret_cast(&::substrait::_Type_FixedChar_default_instance_), reinterpret_cast(&::substrait::_Type_VarChar_default_instance_), @@ -780,7 +812,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = const char descriptor_table_protodef_substrait_2ftype_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = "\n\024substrait/type.proto\022\tsubstrait\032\033googl" - "e/protobuf/empty.proto\"\316!\n\004Type\022\'\n\004bool\030" + "e/protobuf/empty.proto\"\257#\n\004Type\022\'\n\004bool\030" "\001 \001(\0132\027.substrait.Type.BooleanH\000\022 \n\002i8\030\002" " \001(\0132\022.substrait.Type.I8H\000\022\"\n\003i16\030\003 \001(\0132" "\023.substrait.Type.I16H\000\022\"\n\003i32\030\005 \001(\0132\023.su" @@ -795,112 +827,117 @@ const char descriptor_table_protodef_substrait_2ftype_2eproto[] PROTOBUF_SECTION ".substrait.Type.TimeH\000\0225\n\rinterval_year\030" "\023 \001(\0132\034.substrait.Type.IntervalYearH\000\0223\n" "\014interval_day\030\024 \001(\0132\033.substrait.Type.Int" - "ervalDayH\000\0227\n\014timestamp_tz\030\035 \001(\0132\033.subst" - "rait.Type.TimestampTZB\002\030\001H\000\022$\n\004uuid\030 \001(" - "\0132\024.substrait.Type.UUIDH\000\022/\n\nfixed_char\030" - "\025 \001(\0132\031.substrait.Type.FixedCharH\000\022*\n\007va" - "rchar\030\026 \001(\0132\027.substrait.Type.VarCharH\000\0223" - "\n\014fixed_binary\030\027 \001(\0132\033.substrait.Type.Fi" - "xedBinaryH\000\022*\n\007decimal\030\030 \001(\0132\027.substrait" - ".Type.DecimalH\000\022A\n\023precision_timestamp\030!" - " \001(\0132\".substrait.Type.PrecisionTimestamp" - "H\000\022F\n\026precision_timestamp_tz\030\" \001(\0132$.sub" - "strait.Type.PrecisionTimestampTZH\000\022(\n\006st" - "ruct\030\031 \001(\0132\026.substrait.Type.StructH\000\022$\n\004" - "list\030\033 \001(\0132\024.substrait.Type.ListH\000\022\"\n\003ma" - "p\030\034 \001(\0132\023.substrait.Type.MapH\000\0223\n\014user_d" - "efined\030\036 \001(\0132\033.substrait.Type.UserDefine" - "dH\000\022)\n\033user_defined_type_reference\030\037 \001(\r" - "B\002\030\001H\000\032]\n\007Boolean\022 \n\030type_variation_refe" - "rence\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.subst" - "rait.Type.Nullability\032X\n\002I8\022 \n\030type_vari" + "ervalDayH\000\022=\n\021interval_compound\030# \001(\0132 ." + "substrait.Type.IntervalCompoundH\000\0227\n\014tim" + "estamp_tz\030\035 \001(\0132\033.substrait.Type.Timesta" + "mpTZB\002\030\001H\000\022$\n\004uuid\030 \001(\0132\024.substrait.Typ" + "e.UUIDH\000\022/\n\nfixed_char\030\025 \001(\0132\031.substrait" + ".Type.FixedCharH\000\022*\n\007varchar\030\026 \001(\0132\027.sub" + "strait.Type.VarCharH\000\0223\n\014fixed_binary\030\027 " + "\001(\0132\033.substrait.Type.FixedBinaryH\000\022*\n\007de" + "cimal\030\030 \001(\0132\027.substrait.Type.DecimalH\000\022A" + "\n\023precision_timestamp\030! \001(\0132\".substrait." + "Type.PrecisionTimestampH\000\022F\n\026precision_t" + "imestamp_tz\030\" \001(\0132$.substrait.Type.Preci" + "sionTimestampTZH\000\022(\n\006struct\030\031 \001(\0132\026.subs" + "trait.Type.StructH\000\022$\n\004list\030\033 \001(\0132\024.subs" + "trait.Type.ListH\000\022\"\n\003map\030\034 \001(\0132\023.substra" + "it.Type.MapH\000\0223\n\014user_defined\030\036 \001(\0132\033.su" + "bstrait.Type.UserDefinedH\000\022)\n\033user_defin" + "ed_type_reference\030\037 \001(\rB\002\030\001H\000\032]\n\007Boolean" + "\022 \n\030type_variation_reference\030\001 \001(\r\0220\n\013nu" + "llability\030\002 \001(\0162\033.substrait.Type.Nullabi" + "lity\032X\n\002I8\022 \n\030type_variation_reference\030\001" + " \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait.Ty" + "pe.Nullability\032Y\n\003I16\022 \n\030type_variation_" + "reference\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.s" + "ubstrait.Type.Nullability\032Y\n\003I32\022 \n\030type" + "_variation_reference\030\001 \001(\r\0220\n\013nullabilit" + "y\030\002 \001(\0162\033.substrait.Type.Nullability\032Y\n\003" + "I64\022 \n\030type_variation_reference\030\001 \001(\r\0220\n" + "\013nullability\030\002 \001(\0162\033.substrait.Type.Null" + "ability\032Z\n\004FP32\022 \n\030type_variation_refere" + "nce\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substra" + "it.Type.Nullability\032Z\n\004FP64\022 \n\030type_vari" "ation_reference\030\001 \001(\r\0220\n\013nullability\030\002 \001" - "(\0162\033.substrait.Type.Nullability\032Y\n\003I16\022 " - "\n\030type_variation_reference\030\001 \001(\r\0220\n\013null" - "ability\030\002 \001(\0162\033.substrait.Type.Nullabili" - "ty\032Y\n\003I32\022 \n\030type_variation_reference\030\001 " - "\001(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait.Typ" - "e.Nullability\032Y\n\003I64\022 \n\030type_variation_r" - "eference\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.su" - "bstrait.Type.Nullability\032Z\n\004FP32\022 \n\030type" + "(\0162\033.substrait.Type.Nullability\032\\\n\006Strin" + "g\022 \n\030type_variation_reference\030\001 \001(\r\0220\n\013n" + "ullability\030\002 \001(\0162\033.substrait.Type.Nullab" + "ility\032\\\n\006Binary\022 \n\030type_variation_refere" + "nce\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substra" + "it.Type.Nullability\032_\n\tTimestamp\022 \n\030type" "_variation_reference\030\001 \001(\r\0220\n\013nullabilit" "y\030\002 \001(\0162\033.substrait.Type.Nullability\032Z\n\004" - "FP64\022 \n\030type_variation_reference\030\001 \001(\r\0220" + "Date\022 \n\030type_variation_reference\030\001 \001(\r\0220" "\n\013nullability\030\002 \001(\0162\033.substrait.Type.Nul" - "lability\032\\\n\006String\022 \n\030type_variation_ref" - "erence\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.subs" - "trait.Type.Nullability\032\\\n\006Binary\022 \n\030type" - "_variation_reference\030\001 \001(\r\0220\n\013nullabilit" - "y\030\002 \001(\0162\033.substrait.Type.Nullability\032_\n\t" - "Timestamp\022 \n\030type_variation_reference\030\001 " - "\001(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait.Typ" - "e.Nullability\032Z\n\004Date\022 \n\030type_variation_" - "reference\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.s" - "ubstrait.Type.Nullability\032Z\n\004Time\022 \n\030typ" - "e_variation_reference\030\001 \001(\r\0220\n\013nullabili" - "ty\030\002 \001(\0162\033.substrait.Type.Nullability\032a\n" - "\013TimestampTZ\022 \n\030type_variation_reference" - "\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait." - "Type.Nullability\032b\n\014IntervalYear\022 \n\030type" - "_variation_reference\030\001 \001(\r\0220\n\013nullabilit" - "y\030\002 \001(\0162\033.substrait.Type.Nullability\032a\n\013" - "IntervalDay\022 \n\030type_variation_reference\030" - "\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait.T" - "ype.Nullability\032Z\n\004UUID\022 \n\030type_variatio" - "n_reference\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033" - ".substrait.Type.Nullability\032o\n\tFixedChar" - "\022\016\n\006length\030\001 \001(\005\022 \n\030type_variation_refer" - "ence\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substr" - "ait.Type.Nullability\032m\n\007VarChar\022\016\n\006lengt" - "h\030\001 \001(\005\022 \n\030type_variation_reference\030\002 \001(" - "\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Type." - "Nullability\032q\n\013FixedBinary\022\016\n\006length\030\001 \001" - "(\005\022 \n\030type_variation_reference\030\002 \001(\r\0220\n\013" - "nullability\030\003 \001(\0162\033.substrait.Type.Nulla" - "bility\032\177\n\007Decimal\022\r\n\005scale\030\001 \001(\005\022\021\n\tprec" - "ision\030\002 \001(\005\022 \n\030type_variation_reference\030" - "\003 \001(\r\0220\n\013nullability\030\004 \001(\0162\033.substrait.T" - "ype.Nullability\032{\n\022PrecisionTimestamp\022\021\n" - "\tprecision\030\001 \001(\005\022 \n\030type_variation_refer" - "ence\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substr" - "ait.Type.Nullability\032}\n\024PrecisionTimesta" - "mpTZ\022\021\n\tprecision\030\001 \001(\005\022 \n\030type_variatio" - "n_reference\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033" - ".substrait.Type.Nullability\032|\n\006Struct\022\036\n" - "\005types\030\001 \003(\0132\017.substrait.Type\022 \n\030type_va" - "riation_reference\030\002 \001(\r\0220\n\013nullability\030\003" - " \001(\0162\033.substrait.Type.Nullability\032y\n\004Lis" - "t\022\035\n\004type\030\001 \001(\0132\017.substrait.Type\022 \n\030type" + "lability\032Z\n\004Time\022 \n\030type_variation_refer" + "ence\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substr" + "ait.Type.Nullability\032a\n\013TimestampTZ\022 \n\030t" + "ype_variation_reference\030\001 \001(\r\0220\n\013nullabi" + "lity\030\002 \001(\0162\033.substrait.Type.Nullability\032" + "b\n\014IntervalYear\022 \n\030type_variation_refere" + "nce\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.substra" + "it.Type.Nullability\032\207\001\n\013IntervalDay\022 \n\030t" + "ype_variation_reference\030\001 \001(\r\0220\n\013nullabi" + "lity\030\002 \001(\0162\033.substrait.Type.Nullability\022" + "\026\n\tprecision\030\003 \001(\005H\000\210\001\001B\014\n\n_precision\032y\n" + "\020IntervalCompound\022 \n\030type_variation_refe" + "rence\030\001 \001(\r\0220\n\013nullability\030\002 \001(\0162\033.subst" + "rait.Type.Nullability\022\021\n\tprecision\030\003 \001(\005" + "\032Z\n\004UUID\022 \n\030type_variation_reference\030\001 \001" + "(\r\0220\n\013nullability\030\002 \001(\0162\033.substrait.Type" + ".Nullability\032o\n\tFixedChar\022\016\n\006length\030\001 \001(" + "\005\022 \n\030type_variation_reference\030\002 \001(\r\0220\n\013n" + "ullability\030\003 \001(\0162\033.substrait.Type.Nullab" + "ility\032m\n\007VarChar\022\016\n\006length\030\001 \001(\005\022 \n\030type" "_variation_reference\030\002 \001(\r\0220\n\013nullabilit" - "y\030\003 \001(\0162\033.substrait.Type.Nullability\032\227\001\n" - "\003Map\022\034\n\003key\030\001 \001(\0132\017.substrait.Type\022\036\n\005va" - "lue\030\002 \001(\0132\017.substrait.Type\022 \n\030type_varia" - "tion_reference\030\003 \001(\r\0220\n\013nullability\030\004 \001(" - "\0162\033.substrait.Type.Nullability\032\255\001\n\013UserD" - "efined\022\026\n\016type_reference\030\001 \001(\r\022 \n\030type_v" - "ariation_reference\030\002 \001(\r\0220\n\013nullability\030" - "\003 \001(\0162\033.substrait.Type.Nullability\0222\n\017ty" - "pe_parameters\030\004 \003(\0132\031.substrait.Type.Par" - "ameter\032\256\001\n\tParameter\022&\n\004null\030\001 \001(\0132\026.goo" - "gle.protobuf.EmptyH\000\022$\n\tdata_type\030\002 \001(\0132" - "\017.substrait.TypeH\000\022\021\n\007boolean\030\003 \001(\010H\000\022\021\n" - "\007integer\030\004 \001(\003H\000\022\016\n\004enum\030\005 \001(\tH\000\022\020\n\006stri" - "ng\030\006 \001(\tH\000B\013\n\tparameter\"^\n\013Nullability\022\033" - "\n\027NULLABILITY_UNSPECIFIED\020\000\022\030\n\024NULLABILI" - "TY_NULLABLE\020\001\022\030\n\024NULLABILITY_REQUIRED\020\002B" - "\006\n\004kind\"D\n\013NamedStruct\022\r\n\005names\030\001 \003(\t\022&\n" - "\006struct\030\002 \001(\0132\026.substrait.Type.StructBW\n" - "\022io.substrait.protoP\001Z*github.com/substr" - "ait-io/substrait-go/proto\252\002\022Substrait.Pr" - "otobufb\006proto3" + "y\030\003 \001(\0162\033.substrait.Type.Nullability\032q\n\013" + "FixedBinary\022\016\n\006length\030\001 \001(\005\022 \n\030type_vari" + "ation_reference\030\002 \001(\r\0220\n\013nullability\030\003 \001" + "(\0162\033.substrait.Type.Nullability\032\177\n\007Decim" + "al\022\r\n\005scale\030\001 \001(\005\022\021\n\tprecision\030\002 \001(\005\022 \n\030" + "type_variation_reference\030\003 \001(\r\0220\n\013nullab" + "ility\030\004 \001(\0162\033.substrait.Type.Nullability" + "\032{\n\022PrecisionTimestamp\022\021\n\tprecision\030\001 \001(" + "\005\022 \n\030type_variation_reference\030\002 \001(\r\0220\n\013n" + "ullability\030\003 \001(\0162\033.substrait.Type.Nullab" + "ility\032}\n\024PrecisionTimestampTZ\022\021\n\tprecisi" + "on\030\001 \001(\005\022 \n\030type_variation_reference\030\002 \001" + "(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Type" + ".Nullability\032|\n\006Struct\022\036\n\005types\030\001 \003(\0132\017." + "substrait.Type\022 \n\030type_variation_referen" + "ce\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrai" + "t.Type.Nullability\032y\n\004List\022\035\n\004type\030\001 \001(\013" + "2\017.substrait.Type\022 \n\030type_variation_refe" + "rence\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.subst" + "rait.Type.Nullability\032\227\001\n\003Map\022\034\n\003key\030\001 \001" + "(\0132\017.substrait.Type\022\036\n\005value\030\002 \001(\0132\017.sub" + "strait.Type\022 \n\030type_variation_reference\030" + "\003 \001(\r\0220\n\013nullability\030\004 \001(\0162\033.substrait.T" + "ype.Nullability\032\255\001\n\013UserDefined\022\026\n\016type_" + "reference\030\001 \001(\r\022 \n\030type_variation_refere" + "nce\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substra" + "it.Type.Nullability\0222\n\017type_parameters\030\004" + " \003(\0132\031.substrait.Type.Parameter\032\256\001\n\tPara" + "meter\022&\n\004null\030\001 \001(\0132\026.google.protobuf.Em" + "ptyH\000\022$\n\tdata_type\030\002 \001(\0132\017.substrait.Typ" + "eH\000\022\021\n\007boolean\030\003 \001(\010H\000\022\021\n\007integer\030\004 \001(\003H" + "\000\022\016\n\004enum\030\005 \001(\tH\000\022\020\n\006string\030\006 \001(\tH\000B\013\n\tp" + "arameter\"^\n\013Nullability\022\033\n\027NULLABILITY_U" + "NSPECIFIED\020\000\022\030\n\024NULLABILITY_NULLABLE\020\001\022\030" + "\n\024NULLABILITY_REQUIRED\020\002B\006\n\004kind\"D\n\013Name" + "dStruct\022\r\n\005names\030\001 \003(\t\022&\n\006struct\030\002 \001(\0132\026" + ".substrait.Type.StructBW\n\022io.substrait.p" + "rotoP\001Z*github.com/substrait-io/substrai" + "t-go/proto\252\002\022Substrait.Protobufb\006proto3" ; static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_substrait_2ftype_2eproto_deps[1] = { &::descriptor_table_google_2fprotobuf_2fempty_2eproto, }; static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_substrait_2ftype_2eproto_once; const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_substrait_2ftype_2eproto = { - false, false, 4534, descriptor_table_protodef_substrait_2ftype_2eproto, "substrait/type.proto", - &descriptor_table_substrait_2ftype_2eproto_once, descriptor_table_substrait_2ftype_2eproto_deps, 1, 29, + false, false, 4759, descriptor_table_protodef_substrait_2ftype_2eproto, "substrait/type.proto", + &descriptor_table_substrait_2ftype_2eproto_once, descriptor_table_substrait_2ftype_2eproto_deps, 1, 30, schemas, file_default_instances, TableStruct_substrait_2ftype_2eproto::offsets, file_level_metadata_substrait_2ftype_2eproto, file_level_enum_descriptors_substrait_2ftype_2eproto, file_level_service_descriptors_substrait_2ftype_2eproto, }; @@ -3949,6 +3986,10 @@ ::PROTOBUF_NAMESPACE_ID::Metadata Type_IntervalYear::GetMetadata() const { class Type_IntervalDay::_Internal { public: + using HasBits = decltype(std::declval()._has_bits_); + static void set_has_precision(HasBits* has_bits) { + (*has_bits)[0] |= 1u; + } }; Type_IntervalDay::Type_IntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, @@ -3961,19 +4002,20 @@ Type_IntervalDay::Type_IntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, // @@protoc_insertion_point(arena_constructor:substrait.Type.IntervalDay) } Type_IntervalDay::Type_IntervalDay(const Type_IntervalDay& from) - : ::PROTOBUF_NAMESPACE_ID::Message() { + : ::PROTOBUF_NAMESPACE_ID::Message(), + _has_bits_(from._has_bits_) { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); ::memcpy(&type_variation_reference_, &from.type_variation_reference_, - static_cast(reinterpret_cast(&nullability_) - - reinterpret_cast(&type_variation_reference_)) + sizeof(nullability_)); + static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&type_variation_reference_)) + sizeof(precision_)); // @@protoc_insertion_point(copy_constructor:substrait.Type.IntervalDay) } inline void Type_IntervalDay::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&type_variation_reference_) - reinterpret_cast(this)), - 0, static_cast(reinterpret_cast(&nullability_) - - reinterpret_cast(&type_variation_reference_)) + sizeof(nullability_)); + 0, static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&type_variation_reference_)) + sizeof(precision_)); } Type_IntervalDay::~Type_IntervalDay() { @@ -4006,11 +4048,14 @@ void Type_IntervalDay::Clear() { ::memset(&type_variation_reference_, 0, static_cast( reinterpret_cast(&nullability_) - reinterpret_cast(&type_variation_reference_)) + sizeof(nullability_)); + precision_ = 0; + _has_bits_.Clear(); _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } const char* Type_IntervalDay::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + _Internal::HasBits has_bits{}; while (!ctx->Done(&ptr)) { uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); @@ -4032,6 +4077,15 @@ const char* Type_IntervalDay::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPA } else goto handle_unusual; continue; + // optional int32 precision = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + _Internal::set_has_precision(&has_bits); + precision_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; default: goto handle_unusual; } // switch @@ -4048,6 +4102,7 @@ const char* Type_IntervalDay::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPA CHK_(ptr != nullptr); } // while message_done: + _has_bits_.Or(has_bits); return ptr; failure: ptr = nullptr; @@ -4074,6 +4129,12 @@ uint8_t* Type_IntervalDay::_InternalSerialize( 2, this->_internal_nullability(), target); } + // optional int32 precision = 3; + if (_internal_has_precision()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(3, this->_internal_precision(), target); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -4101,6 +4162,12 @@ size_t Type_IntervalDay::ByteSizeLong() const { ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); } + // optional int32 precision = 3; + cached_has_bits = _has_bits_[0]; + if (cached_has_bits & 0x00000001u) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_precision()); + } + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } @@ -4129,6 +4196,9 @@ void Type_IntervalDay::MergeFrom(const Type_IntervalDay& from) { if (from._internal_nullability() != 0) { _internal_set_nullability(from._internal_nullability()); } + if (from._internal_has_precision()) { + _internal_set_precision(from._internal_precision()); + } _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } @@ -4146,9 +4216,10 @@ bool Type_IntervalDay::IsInitialized() const { void Type_IntervalDay::InternalSwap(Type_IntervalDay* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); + swap(_has_bits_[0], other->_has_bits_[0]); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(Type_IntervalDay, nullability_) - + sizeof(Type_IntervalDay::nullability_) + PROTOBUF_FIELD_OFFSET(Type_IntervalDay, precision_) + + sizeof(Type_IntervalDay::precision_) - PROTOBUF_FIELD_OFFSET(Type_IntervalDay, type_variation_reference_)>( reinterpret_cast(&type_variation_reference_), reinterpret_cast(&other->type_variation_reference_)); @@ -4162,6 +4233,243 @@ ::PROTOBUF_NAMESPACE_ID::Metadata Type_IntervalDay::GetMetadata() const { // =================================================================== +class Type_IntervalCompound::_Internal { + public: +}; + +Type_IntervalCompound::Type_IntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.Type.IntervalCompound) +} +Type_IntervalCompound::Type_IntervalCompound(const Type_IntervalCompound& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + ::memcpy(&type_variation_reference_, &from.type_variation_reference_, + static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&type_variation_reference_)) + sizeof(precision_)); + // @@protoc_insertion_point(copy_constructor:substrait.Type.IntervalCompound) +} + +inline void Type_IntervalCompound::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&type_variation_reference_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&precision_) - + reinterpret_cast(&type_variation_reference_)) + sizeof(precision_)); +} + +Type_IntervalCompound::~Type_IntervalCompound() { + // @@protoc_insertion_point(destructor:substrait.Type.IntervalCompound) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void Type_IntervalCompound::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); +} + +void Type_IntervalCompound::ArenaDtor(void* object) { + Type_IntervalCompound* _this = reinterpret_cast< Type_IntervalCompound* >(object); + (void)_this; +} +void Type_IntervalCompound::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void Type_IntervalCompound::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void Type_IntervalCompound::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.Type.IntervalCompound) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + ::memset(&type_variation_reference_, 0, static_cast( + reinterpret_cast(&precision_) - + reinterpret_cast(&type_variation_reference_)) + sizeof(precision_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* Type_IntervalCompound::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // uint32 type_variation_reference = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 8)) { + type_variation_reference_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Type.Nullability nullability = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); + } else + goto handle_unusual; + continue; + // int32 precision = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + precision_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* Type_IntervalCompound::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.Type.IntervalCompound) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // uint32 type_variation_reference = 1; + if (this->_internal_type_variation_reference() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(1, this->_internal_type_variation_reference(), target); + } + + // .substrait.Type.Nullability nullability = 2; + if (this->_internal_nullability() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 2, this->_internal_nullability(), target); + } + + // int32 precision = 3; + if (this->_internal_precision() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteInt32ToArray(3, this->_internal_precision(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.Type.IntervalCompound) + return target; +} + +size_t Type_IntervalCompound::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.Type.IntervalCompound) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // uint32 type_variation_reference = 1; + if (this->_internal_type_variation_reference() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_type_variation_reference()); + } + + // .substrait.Type.Nullability nullability = 2; + if (this->_internal_nullability() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); + } + + // int32 precision = 3; + if (this->_internal_precision() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::Int32SizePlusOne(this->_internal_precision()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData Type_IntervalCompound::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + Type_IntervalCompound::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*Type_IntervalCompound::GetClassData() const { return &_class_data_; } + +void Type_IntervalCompound::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void Type_IntervalCompound::MergeFrom(const Type_IntervalCompound& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.Type.IntervalCompound) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_type_variation_reference() != 0) { + _internal_set_type_variation_reference(from._internal_type_variation_reference()); + } + if (from._internal_nullability() != 0) { + _internal_set_nullability(from._internal_nullability()); + } + if (from._internal_precision() != 0) { + _internal_set_precision(from._internal_precision()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void Type_IntervalCompound::CopyFrom(const Type_IntervalCompound& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.Type.IntervalCompound) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool Type_IntervalCompound::IsInitialized() const { + return true; +} + +void Type_IntervalCompound::InternalSwap(Type_IntervalCompound* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(Type_IntervalCompound, precision_) + + sizeof(Type_IntervalCompound::precision_) + - PROTOBUF_FIELD_OFFSET(Type_IntervalCompound, type_variation_reference_)>( + reinterpret_cast(&type_variation_reference_), + reinterpret_cast(&other->type_variation_reference_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata Type_IntervalCompound::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, + file_level_metadata_substrait_2ftype_2eproto[15]); +} + +// =================================================================== + class Type_UUID::_Internal { public: }; @@ -4372,7 +4680,7 @@ void Type_UUID::InternalSwap(Type_UUID* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_UUID::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[15]); + file_level_metadata_substrait_2ftype_2eproto[16]); } // =================================================================== @@ -4609,7 +4917,7 @@ void Type_FixedChar::InternalSwap(Type_FixedChar* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_FixedChar::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[16]); + file_level_metadata_substrait_2ftype_2eproto[17]); } // =================================================================== @@ -4846,7 +5154,7 @@ void Type_VarChar::InternalSwap(Type_VarChar* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_VarChar::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[17]); + file_level_metadata_substrait_2ftype_2eproto[18]); } // =================================================================== @@ -5083,7 +5391,7 @@ void Type_FixedBinary::InternalSwap(Type_FixedBinary* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_FixedBinary::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[18]); + file_level_metadata_substrait_2ftype_2eproto[19]); } // =================================================================== @@ -5342,7 +5650,7 @@ void Type_Decimal::InternalSwap(Type_Decimal* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_Decimal::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[19]); + file_level_metadata_substrait_2ftype_2eproto[20]); } // =================================================================== @@ -5579,7 +5887,7 @@ void Type_PrecisionTimestamp::InternalSwap(Type_PrecisionTimestamp* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_PrecisionTimestamp::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[20]); + file_level_metadata_substrait_2ftype_2eproto[21]); } // =================================================================== @@ -5816,7 +6124,7 @@ void Type_PrecisionTimestampTZ::InternalSwap(Type_PrecisionTimestampTZ* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_PrecisionTimestampTZ::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[21]); + file_level_metadata_substrait_2ftype_2eproto[22]); } // =================================================================== @@ -6064,7 +6372,7 @@ void Type_Struct::InternalSwap(Type_Struct* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_Struct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[22]); + file_level_metadata_substrait_2ftype_2eproto[23]); } // =================================================================== @@ -6320,7 +6628,7 @@ void Type_List::InternalSwap(Type_List* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_List::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[23]); + file_level_metadata_substrait_2ftype_2eproto[24]); } // =================================================================== @@ -6617,7 +6925,7 @@ void Type_Map::InternalSwap(Type_Map* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_Map::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[24]); + file_level_metadata_substrait_2ftype_2eproto[25]); } // =================================================================== @@ -6887,7 +7195,7 @@ void Type_UserDefined::InternalSwap(Type_UserDefined* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_UserDefined::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[25]); + file_level_metadata_substrait_2ftype_2eproto[26]); } // =================================================================== @@ -7337,7 +7645,7 @@ void Type_Parameter::InternalSwap(Type_Parameter* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type_Parameter::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[26]); + file_level_metadata_substrait_2ftype_2eproto[27]); } // =================================================================== @@ -7358,6 +7666,7 @@ class Type::_Internal { static const ::substrait::Type_Time& time(const Type* msg); static const ::substrait::Type_IntervalYear& interval_year(const Type* msg); static const ::substrait::Type_IntervalDay& interval_day(const Type* msg); + static const ::substrait::Type_IntervalCompound& interval_compound(const Type* msg); static const ::substrait::Type_TimestampTZ& timestamp_tz(const Type* msg); static const ::substrait::Type_UUID& uuid(const Type* msg); static const ::substrait::Type_FixedChar& fixed_char(const Type* msg); @@ -7428,6 +7737,10 @@ const ::substrait::Type_IntervalDay& Type::_Internal::interval_day(const Type* msg) { return *msg->kind_.interval_day_; } +const ::substrait::Type_IntervalCompound& +Type::_Internal::interval_compound(const Type* msg) { + return *msg->kind_.interval_compound_; +} const ::substrait::Type_TimestampTZ& Type::_Internal::timestamp_tz(const Type* msg) { return *msg->kind_.timestamp_tz_; @@ -7686,6 +7999,21 @@ void Type::set_allocated_interval_day(::substrait::Type_IntervalDay* interval_da } // @@protoc_insertion_point(field_set_allocated:substrait.Type.interval_day) } +void Type::set_allocated_interval_compound(::substrait::Type_IntervalCompound* interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_kind(); + if (interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::Type_IntervalCompound>::GetOwningArena(interval_compound); + if (message_arena != submessage_arena) { + interval_compound = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_compound, submessage_arena); + } + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_set_allocated:substrait.Type.interval_compound) +} void Type::set_allocated_timestamp_tz(::substrait::Type_TimestampTZ* timestamp_tz) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_kind(); @@ -7936,6 +8264,10 @@ Type::Type(const Type& from) _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); break; } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::Type_IntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kTimestampTz: { _internal_mutable_timestamp_tz()->::substrait::Type_TimestampTZ::MergeFrom(from._internal_timestamp_tz()); break; @@ -8110,6 +8442,12 @@ void Type::clear_kind() { } break; } + case kIntervalCompound: { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + break; + } case kTimestampTz: { if (GetArenaForAllocation() == nullptr) { delete kind_.timestamp_tz_; @@ -8426,6 +8764,14 @@ const char* Type::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::inter } else goto handle_unusual; continue; + // .substrait.Type.IntervalCompound interval_compound = 35; + case 35: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 26)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_compound(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; default: goto handle_unusual; } // switch @@ -8669,6 +9015,14 @@ uint8_t* Type::_InternalSerialize( 34, _Internal::precision_timestamp_tz(this), target, stream); } + // .substrait.Type.IntervalCompound interval_compound = 35; + if (_internal_has_interval_compound()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 35, _Internal::interval_compound(this), target, stream); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -8784,6 +9138,13 @@ size_t Type::ByteSizeLong() const { *kind_.interval_day_); break; } + // .substrait.Type.IntervalCompound interval_compound = 35; + case kIntervalCompound: { + total_size += 2 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *kind_.interval_compound_); + break; + } // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; case kTimestampTz: { total_size += 2 + @@ -8958,6 +9319,10 @@ void Type::MergeFrom(const Type& from) { _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); break; } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::Type_IntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kTimestampTz: { _internal_mutable_timestamp_tz()->::substrait::Type_TimestampTZ::MergeFrom(from._internal_timestamp_tz()); break; @@ -9038,7 +9403,7 @@ void Type::InternalSwap(Type* other) { ::PROTOBUF_NAMESPACE_ID::Metadata Type::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[27]); + file_level_metadata_substrait_2ftype_2eproto[28]); } // =================================================================== @@ -9271,7 +9636,7 @@ void NamedStruct::InternalSwap(NamedStruct* other) { ::PROTOBUF_NAMESPACE_ID::Metadata NamedStruct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_2eproto_getter, &descriptor_table_substrait_2ftype_2eproto_once, - file_level_metadata_substrait_2ftype_2eproto[28]); + file_level_metadata_substrait_2ftype_2eproto[29]); } // @@protoc_insertion_point(namespace_scope) @@ -9322,6 +9687,9 @@ template<> PROTOBUF_NOINLINE ::substrait::Type_IntervalYear* Arena::CreateMaybeM template<> PROTOBUF_NOINLINE ::substrait::Type_IntervalDay* Arena::CreateMaybeMessage< ::substrait::Type_IntervalDay >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Type_IntervalDay >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::Type_IntervalCompound* Arena::CreateMaybeMessage< ::substrait::Type_IntervalCompound >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::Type_IntervalCompound >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::Type_UUID* Arena::CreateMaybeMessage< ::substrait::Type_UUID >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::Type_UUID >(arena); } diff --git a/third_party/substrait/substrait/type.pb.h b/third_party/substrait/substrait/type.pb.h index a8d0dfd..025ed98 100644 --- a/third_party/substrait/substrait/type.pb.h +++ b/third_party/substrait/substrait/type.pb.h @@ -48,7 +48,7 @@ struct TableStruct_substrait_2ftype_2eproto { PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::AuxiliaryParseTableField aux[] PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[29] + static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[30] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[]; static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[]; @@ -98,6 +98,9 @@ extern Type_I64DefaultTypeInternal _Type_I64_default_instance_; class Type_I8; struct Type_I8DefaultTypeInternal; extern Type_I8DefaultTypeInternal _Type_I8_default_instance_; +class Type_IntervalCompound; +struct Type_IntervalCompoundDefaultTypeInternal; +extern Type_IntervalCompoundDefaultTypeInternal _Type_IntervalCompound_default_instance_; class Type_IntervalDay; struct Type_IntervalDayDefaultTypeInternal; extern Type_IntervalDayDefaultTypeInternal _Type_IntervalDay_default_instance_; @@ -159,6 +162,7 @@ template<> ::substrait::Type_I16* Arena::CreateMaybeMessage<::substrait::Type_I1 template<> ::substrait::Type_I32* Arena::CreateMaybeMessage<::substrait::Type_I32>(Arena*); template<> ::substrait::Type_I64* Arena::CreateMaybeMessage<::substrait::Type_I64>(Arena*); template<> ::substrait::Type_I8* Arena::CreateMaybeMessage<::substrait::Type_I8>(Arena*); +template<> ::substrait::Type_IntervalCompound* Arena::CreateMaybeMessage<::substrait::Type_IntervalCompound>(Arena*); template<> ::substrait::Type_IntervalDay* Arena::CreateMaybeMessage<::substrait::Type_IntervalDay>(Arena*); template<> ::substrait::Type_IntervalYear* Arena::CreateMaybeMessage<::substrait::Type_IntervalYear>(Arena*); template<> ::substrait::Type_List* Arena::CreateMaybeMessage<::substrait::Type_List>(Arena*); @@ -2527,6 +2531,7 @@ class Type_IntervalDay final : enum : int { kTypeVariationReferenceFieldNumber = 1, kNullabilityFieldNumber = 2, + kPrecisionFieldNumber = 3, }; // uint32 type_variation_reference = 1; void clear_type_variation_reference(); @@ -2546,15 +2551,198 @@ class Type_IntervalDay final : void _internal_set_nullability(::substrait::Type_Nullability value); public: + // optional int32 precision = 3; + bool has_precision() const; + private: + bool _internal_has_precision() const; + public: + void clear_precision(); + int32_t precision() const; + void set_precision(int32_t value); + private: + int32_t _internal_precision() const; + void _internal_set_precision(int32_t value); + public: + // @@protoc_insertion_point(class_scope:substrait.Type.IntervalDay) private: class _Internal; + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::PROTOBUF_NAMESPACE_ID::internal::HasBits<1> _has_bits_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + uint32_t type_variation_reference_; + int nullability_; + int32_t precision_; + friend struct ::TableStruct_substrait_2ftype_2eproto; +}; +// ------------------------------------------------------------------- + +class Type_IntervalCompound final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.Type.IntervalCompound) */ { + public: + inline Type_IntervalCompound() : Type_IntervalCompound(nullptr) {} + ~Type_IntervalCompound() override; + explicit constexpr Type_IntervalCompound(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + Type_IntervalCompound(const Type_IntervalCompound& from); + Type_IntervalCompound(Type_IntervalCompound&& from) noexcept + : Type_IntervalCompound() { + *this = ::std::move(from); + } + + inline Type_IntervalCompound& operator=(const Type_IntervalCompound& from) { + CopyFrom(from); + return *this; + } + inline Type_IntervalCompound& operator=(Type_IntervalCompound&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const Type_IntervalCompound& default_instance() { + return *internal_default_instance(); + } + static inline const Type_IntervalCompound* internal_default_instance() { + return reinterpret_cast( + &_Type_IntervalCompound_default_instance_); + } + static constexpr int kIndexInFileMessages = + 15; + + friend void swap(Type_IntervalCompound& a, Type_IntervalCompound& b) { + a.Swap(&b); + } + inline void Swap(Type_IntervalCompound* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(Type_IntervalCompound* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + Type_IntervalCompound* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const Type_IntervalCompound& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const Type_IntervalCompound& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(Type_IntervalCompound* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.Type.IntervalCompound"; + } + protected: + explicit Type_IntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kTypeVariationReferenceFieldNumber = 1, + kNullabilityFieldNumber = 2, + kPrecisionFieldNumber = 3, + }; + // uint32 type_variation_reference = 1; + void clear_type_variation_reference(); + uint32_t type_variation_reference() const; + void set_type_variation_reference(uint32_t value); + private: + uint32_t _internal_type_variation_reference() const; + void _internal_set_type_variation_reference(uint32_t value); + public: + + // .substrait.Type.Nullability nullability = 2; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); + private: + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); + public: + + // int32 precision = 3; + void clear_precision(); + int32_t precision() const; + void set_precision(int32_t value); + private: + int32_t _internal_precision() const; + void _internal_set_precision(int32_t value); + public: + + // @@protoc_insertion_point(class_scope:substrait.Type.IntervalCompound) + private: + class _Internal; + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; uint32_t type_variation_reference_; int nullability_; + int32_t precision_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_2eproto; }; @@ -2608,7 +2796,7 @@ class Type_UUID final : &_Type_UUID_default_instance_); } static constexpr int kIndexInFileMessages = - 15; + 16; friend void swap(Type_UUID& a, Type_UUID& b) { a.Swap(&b); @@ -2765,7 +2953,7 @@ class Type_FixedChar final : &_Type_FixedChar_default_instance_); } static constexpr int kIndexInFileMessages = - 16; + 17; friend void swap(Type_FixedChar& a, Type_FixedChar& b) { a.Swap(&b); @@ -2933,7 +3121,7 @@ class Type_VarChar final : &_Type_VarChar_default_instance_); } static constexpr int kIndexInFileMessages = - 17; + 18; friend void swap(Type_VarChar& a, Type_VarChar& b) { a.Swap(&b); @@ -3101,7 +3289,7 @@ class Type_FixedBinary final : &_Type_FixedBinary_default_instance_); } static constexpr int kIndexInFileMessages = - 18; + 19; friend void swap(Type_FixedBinary& a, Type_FixedBinary& b) { a.Swap(&b); @@ -3269,7 +3457,7 @@ class Type_Decimal final : &_Type_Decimal_default_instance_); } static constexpr int kIndexInFileMessages = - 19; + 20; friend void swap(Type_Decimal& a, Type_Decimal& b) { a.Swap(&b); @@ -3448,7 +3636,7 @@ class Type_PrecisionTimestamp final : &_Type_PrecisionTimestamp_default_instance_); } static constexpr int kIndexInFileMessages = - 20; + 21; friend void swap(Type_PrecisionTimestamp& a, Type_PrecisionTimestamp& b) { a.Swap(&b); @@ -3616,7 +3804,7 @@ class Type_PrecisionTimestampTZ final : &_Type_PrecisionTimestampTZ_default_instance_); } static constexpr int kIndexInFileMessages = - 21; + 22; friend void swap(Type_PrecisionTimestampTZ& a, Type_PrecisionTimestampTZ& b) { a.Swap(&b); @@ -3784,7 +3972,7 @@ class Type_Struct final : &_Type_Struct_default_instance_); } static constexpr int kIndexInFileMessages = - 22; + 23; friend void swap(Type_Struct& a, Type_Struct& b) { a.Swap(&b); @@ -3961,7 +4149,7 @@ class Type_List final : &_Type_List_default_instance_); } static constexpr int kIndexInFileMessages = - 23; + 24; friend void swap(Type_List& a, Type_List& b) { a.Swap(&b); @@ -4138,7 +4326,7 @@ class Type_Map final : &_Type_Map_default_instance_); } static constexpr int kIndexInFileMessages = - 24; + 25; friend void swap(Type_Map& a, Type_Map& b) { a.Swap(&b); @@ -4335,7 +4523,7 @@ class Type_UserDefined final : &_Type_UserDefined_default_instance_); } static constexpr int kIndexInFileMessages = - 25; + 26; friend void swap(Type_UserDefined& a, Type_UserDefined& b) { a.Swap(&b); @@ -4533,7 +4721,7 @@ class Type_Parameter final : &_Type_Parameter_default_instance_); } static constexpr int kIndexInFileMessages = - 26; + 27; friend void swap(Type_Parameter& a, Type_Parameter& b) { a.Swap(&b); @@ -4805,6 +4993,7 @@ class Type final : kTime = 17, kIntervalYear = 19, kIntervalDay = 20, + kIntervalCompound = 35, kTimestampTz = 29, kUuid = 32, kFixedChar = 21, @@ -4826,7 +5015,7 @@ class Type final : &_Type_default_instance_); } static constexpr int kIndexInFileMessages = - 27; + 28; friend void swap(Type& a, Type& b) { a.Swap(&b); @@ -4912,6 +5101,7 @@ class Type final : typedef Type_TimestampTZ TimestampTZ; typedef Type_IntervalYear IntervalYear; typedef Type_IntervalDay IntervalDay; + typedef Type_IntervalCompound IntervalCompound; typedef Type_UUID UUID; typedef Type_FixedChar FixedChar; typedef Type_VarChar VarChar; @@ -4974,6 +5164,7 @@ class Type final : kTimeFieldNumber = 17, kIntervalYearFieldNumber = 19, kIntervalDayFieldNumber = 20, + kIntervalCompoundFieldNumber = 35, kTimestampTzFieldNumber = 29, kUuidFieldNumber = 32, kFixedCharFieldNumber = 21, @@ -5240,6 +5431,24 @@ class Type final : ::substrait::Type_IntervalDay* interval_day); ::substrait::Type_IntervalDay* unsafe_arena_release_interval_day(); + // .substrait.Type.IntervalCompound interval_compound = 35; + bool has_interval_compound() const; + private: + bool _internal_has_interval_compound() const; + public: + void clear_interval_compound(); + const ::substrait::Type_IntervalCompound& interval_compound() const; + PROTOBUF_NODISCARD ::substrait::Type_IntervalCompound* release_interval_compound(); + ::substrait::Type_IntervalCompound* mutable_interval_compound(); + void set_allocated_interval_compound(::substrait::Type_IntervalCompound* interval_compound); + private: + const ::substrait::Type_IntervalCompound& _internal_interval_compound() const; + ::substrait::Type_IntervalCompound* _internal_mutable_interval_compound(); + public: + void unsafe_arena_set_allocated_interval_compound( + ::substrait::Type_IntervalCompound* interval_compound); + ::substrait::Type_IntervalCompound* unsafe_arena_release_interval_compound(); + // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; PROTOBUF_DEPRECATED bool has_timestamp_tz() const; private: @@ -5488,6 +5697,7 @@ class Type final : void set_has_time(); void set_has_interval_year(); void set_has_interval_day(); + void set_has_interval_compound(); void set_has_timestamp_tz(); void set_has_uuid(); void set_has_fixed_char(); @@ -5525,6 +5735,7 @@ class Type final : ::substrait::Type_Time* time_; ::substrait::Type_IntervalYear* interval_year_; ::substrait::Type_IntervalDay* interval_day_; + ::substrait::Type_IntervalCompound* interval_compound_; ::substrait::Type_TimestampTZ* timestamp_tz_; ::substrait::Type_UUID* uuid_; ::substrait::Type_FixedChar* fixed_char_; @@ -5594,7 +5805,7 @@ class NamedStruct final : &_NamedStruct_default_instance_); } static constexpr int kIndexInFileMessages = - 28; + 29; friend void swap(NamedStruct& a, NamedStruct& b) { a.Swap(&b); @@ -6392,6 +6603,98 @@ inline void Type_IntervalDay::set_nullability(::substrait::Type_Nullability valu // @@protoc_insertion_point(field_set:substrait.Type.IntervalDay.nullability) } +// optional int32 precision = 3; +inline bool Type_IntervalDay::_internal_has_precision() const { + bool value = (_has_bits_[0] & 0x00000001u) != 0; + return value; +} +inline bool Type_IntervalDay::has_precision() const { + return _internal_has_precision(); +} +inline void Type_IntervalDay::clear_precision() { + precision_ = 0; + _has_bits_[0] &= ~0x00000001u; +} +inline int32_t Type_IntervalDay::_internal_precision() const { + return precision_; +} +inline int32_t Type_IntervalDay::precision() const { + // @@protoc_insertion_point(field_get:substrait.Type.IntervalDay.precision) + return _internal_precision(); +} +inline void Type_IntervalDay::_internal_set_precision(int32_t value) { + _has_bits_[0] |= 0x00000001u; + precision_ = value; +} +inline void Type_IntervalDay::set_precision(int32_t value) { + _internal_set_precision(value); + // @@protoc_insertion_point(field_set:substrait.Type.IntervalDay.precision) +} + +// ------------------------------------------------------------------- + +// Type_IntervalCompound + +// uint32 type_variation_reference = 1; +inline void Type_IntervalCompound::clear_type_variation_reference() { + type_variation_reference_ = 0u; +} +inline uint32_t Type_IntervalCompound::_internal_type_variation_reference() const { + return type_variation_reference_; +} +inline uint32_t Type_IntervalCompound::type_variation_reference() const { + // @@protoc_insertion_point(field_get:substrait.Type.IntervalCompound.type_variation_reference) + return _internal_type_variation_reference(); +} +inline void Type_IntervalCompound::_internal_set_type_variation_reference(uint32_t value) { + + type_variation_reference_ = value; +} +inline void Type_IntervalCompound::set_type_variation_reference(uint32_t value) { + _internal_set_type_variation_reference(value); + // @@protoc_insertion_point(field_set:substrait.Type.IntervalCompound.type_variation_reference) +} + +// .substrait.Type.Nullability nullability = 2; +inline void Type_IntervalCompound::clear_nullability() { + nullability_ = 0; +} +inline ::substrait::Type_Nullability Type_IntervalCompound::_internal_nullability() const { + return static_cast< ::substrait::Type_Nullability >(nullability_); +} +inline ::substrait::Type_Nullability Type_IntervalCompound::nullability() const { + // @@protoc_insertion_point(field_get:substrait.Type.IntervalCompound.nullability) + return _internal_nullability(); +} +inline void Type_IntervalCompound::_internal_set_nullability(::substrait::Type_Nullability value) { + + nullability_ = value; +} +inline void Type_IntervalCompound::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.Type.IntervalCompound.nullability) +} + +// int32 precision = 3; +inline void Type_IntervalCompound::clear_precision() { + precision_ = 0; +} +inline int32_t Type_IntervalCompound::_internal_precision() const { + return precision_; +} +inline int32_t Type_IntervalCompound::precision() const { + // @@protoc_insertion_point(field_get:substrait.Type.IntervalCompound.precision) + return _internal_precision(); +} +inline void Type_IntervalCompound::_internal_set_precision(int32_t value) { + + precision_ = value; +} +inline void Type_IntervalCompound::set_precision(int32_t value) { + _internal_set_precision(value); + // @@protoc_insertion_point(field_set:substrait.Type.IntervalCompound.precision) +} + // ------------------------------------------------------------------- // Type_UUID @@ -8819,6 +9122,80 @@ inline ::substrait::Type_IntervalDay* Type::mutable_interval_day() { return _msg; } +// .substrait.Type.IntervalCompound interval_compound = 35; +inline bool Type::_internal_has_interval_compound() const { + return kind_case() == kIntervalCompound; +} +inline bool Type::has_interval_compound() const { + return _internal_has_interval_compound(); +} +inline void Type::set_has_interval_compound() { + _oneof_case_[0] = kIntervalCompound; +} +inline void Type::clear_interval_compound() { + if (_internal_has_interval_compound()) { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + clear_has_kind(); + } +} +inline ::substrait::Type_IntervalCompound* Type::release_interval_compound() { + // @@protoc_insertion_point(field_release:substrait.Type.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::Type_IntervalCompound* temp = kind_.interval_compound_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::Type_IntervalCompound& Type::_internal_interval_compound() const { + return _internal_has_interval_compound() + ? *kind_.interval_compound_ + : reinterpret_cast< ::substrait::Type_IntervalCompound&>(::substrait::_Type_IntervalCompound_default_instance_); +} +inline const ::substrait::Type_IntervalCompound& Type::interval_compound() const { + // @@protoc_insertion_point(field_get:substrait.Type.interval_compound) + return _internal_interval_compound(); +} +inline ::substrait::Type_IntervalCompound* Type::unsafe_arena_release_interval_compound() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.Type.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::Type_IntervalCompound* temp = kind_.interval_compound_; + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void Type::unsafe_arena_set_allocated_interval_compound(::substrait::Type_IntervalCompound* interval_compound) { + clear_kind(); + if (interval_compound) { + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.Type.interval_compound) +} +inline ::substrait::Type_IntervalCompound* Type::_internal_mutable_interval_compound() { + if (!_internal_has_interval_compound()) { + clear_kind(); + set_has_interval_compound(); + kind_.interval_compound_ = CreateMaybeMessage< ::substrait::Type_IntervalCompound >(GetArenaForAllocation()); + } + return kind_.interval_compound_; +} +inline ::substrait::Type_IntervalCompound* Type::mutable_interval_compound() { + ::substrait::Type_IntervalCompound* _msg = _internal_mutable_interval_compound(); + // @@protoc_insertion_point(field_mutable:substrait.Type.interval_compound) + return _msg; +} + // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; inline bool Type::_internal_has_timestamp_tz() const { return kind_case() == kTimestampTz; @@ -9982,6 +10359,8 @@ inline void NamedStruct::set_allocated_struct_(::substrait::Type_Struct* struct_ // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope) diff --git a/third_party/substrait/substrait/type_expressions.pb.cc b/third_party/substrait/substrait/type_expressions.pb.cc index 8eb7196..40fb98c 100644 --- a/third_party/substrait/substrait/type_expressions.pb.cc +++ b/third_party/substrait/substrait/type_expressions.pb.cc @@ -93,6 +93,36 @@ struct DerivationExpression_ExpressionPrecisionTimestampDefaultTypeInternal { }; }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DerivationExpression_ExpressionPrecisionTimestampDefaultTypeInternal _DerivationExpression_ExpressionPrecisionTimestamp_default_instance_; +constexpr DerivationExpression_ExpressionIntervalDay::DerivationExpression_ExpressionIntervalDay( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : precision_(nullptr) + , variation_pointer_(0u) + , nullability_(0) +{} +struct DerivationExpression_ExpressionIntervalDayDefaultTypeInternal { + constexpr DerivationExpression_ExpressionIntervalDayDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~DerivationExpression_ExpressionIntervalDayDefaultTypeInternal() {} + union { + DerivationExpression_ExpressionIntervalDay _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DerivationExpression_ExpressionIntervalDayDefaultTypeInternal _DerivationExpression_ExpressionIntervalDay_default_instance_; +constexpr DerivationExpression_ExpressionIntervalCompound::DerivationExpression_ExpressionIntervalCompound( + ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) + : precision_(nullptr) + , variation_pointer_(0u) + , nullability_(0) +{} +struct DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal { + constexpr DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal() + : _instance(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized{}) {} + ~DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal() {} + union { + DerivationExpression_ExpressionIntervalCompound _instance; + }; +}; +PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal _DerivationExpression_ExpressionIntervalCompound_default_instance_; constexpr DerivationExpression_ExpressionPrecisionTimestampTZ::DerivationExpression_ExpressionPrecisionTimestampTZ( ::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized) : precision_(nullptr) @@ -264,7 +294,7 @@ struct DerivationExpressionDefaultTypeInternal { }; PROTOBUF_ATTRIBUTE_NO_DESTROY PROTOBUF_CONSTINIT DerivationExpressionDefaultTypeInternal _DerivationExpression_default_instance_; } // namespace substrait -static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2ftype_5fexpressions_2eproto[17]; +static ::PROTOBUF_NAMESPACE_ID::Metadata file_level_metadata_substrait_2ftype_5fexpressions_2eproto[19]; static const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* file_level_enum_descriptors_substrait_2ftype_5fexpressions_2eproto[2]; static constexpr ::PROTOBUF_NAMESPACE_ID::ServiceDescriptor const** file_level_service_descriptors_substrait_2ftype_5fexpressions_2eproto = nullptr; @@ -316,6 +346,24 @@ const uint32_t TableStruct_substrait_2ftype_5fexpressions_2eproto::offsets[] PRO PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionPrecisionTimestamp, variation_pointer_), PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionPrecisionTimestamp, nullability_), ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalDay, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalDay, precision_), + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalDay, variation_pointer_), + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalDay, nullability_), + ~0u, // no _has_bits_ + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalCompound, _internal_metadata_), + ~0u, // no _extensions_ + ~0u, // no _oneof_case_ + ~0u, // no _weak_field_map_ + ~0u, // no _inlined_string_donated_ + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalCompound, precision_), + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalCompound, variation_pointer_), + PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionIntervalCompound, nullability_), + ~0u, // no _has_bits_ PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ, _internal_metadata_), ~0u, // no _extensions_ ~0u, // no _oneof_case_ @@ -451,6 +499,7 @@ const uint32_t TableStruct_substrait_2ftype_5fexpressions_2eproto::offsets[] PRO ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, + ::PROTOBUF_NAMESPACE_ID::internal::kInvalidFieldOffsetTag, PROTOBUF_FIELD_OFFSET(::substrait::DerivationExpression, kind_), }; static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = { @@ -459,18 +508,20 @@ static const ::PROTOBUF_NAMESPACE_ID::internal::MigrationSchema schemas[] PROTOB { 18, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionFixedBinary)}, { 27, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionDecimal)}, { 37, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionPrecisionTimestamp)}, - { 46, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ)}, - { 55, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionStruct)}, - { 64, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionNamedStruct)}, - { 72, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionList)}, - { 81, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionMap)}, - { 91, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionUserDefined)}, - { 100, -1, -1, sizeof(::substrait::DerivationExpression_IfElse)}, - { 109, -1, -1, sizeof(::substrait::DerivationExpression_UnaryOp)}, - { 117, -1, -1, sizeof(::substrait::DerivationExpression_BinaryOp)}, - { 126, -1, -1, sizeof(::substrait::DerivationExpression_ReturnProgram_Assignment)}, - { 134, -1, -1, sizeof(::substrait::DerivationExpression_ReturnProgram)}, - { 142, -1, -1, sizeof(::substrait::DerivationExpression)}, + { 46, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionIntervalDay)}, + { 55, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionIntervalCompound)}, + { 64, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ)}, + { 73, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionStruct)}, + { 82, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionNamedStruct)}, + { 90, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionList)}, + { 99, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionMap)}, + { 109, -1, -1, sizeof(::substrait::DerivationExpression_ExpressionUserDefined)}, + { 118, -1, -1, sizeof(::substrait::DerivationExpression_IfElse)}, + { 127, -1, -1, sizeof(::substrait::DerivationExpression_UnaryOp)}, + { 135, -1, -1, sizeof(::substrait::DerivationExpression_BinaryOp)}, + { 144, -1, -1, sizeof(::substrait::DerivationExpression_ReturnProgram_Assignment)}, + { 152, -1, -1, sizeof(::substrait::DerivationExpression_ReturnProgram)}, + { 160, -1, -1, sizeof(::substrait::DerivationExpression)}, }; static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = { @@ -479,6 +530,8 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = reinterpret_cast(&::substrait::_DerivationExpression_ExpressionFixedBinary_default_instance_), reinterpret_cast(&::substrait::_DerivationExpression_ExpressionDecimal_default_instance_), reinterpret_cast(&::substrait::_DerivationExpression_ExpressionPrecisionTimestamp_default_instance_), + reinterpret_cast(&::substrait::_DerivationExpression_ExpressionIntervalDay_default_instance_), + reinterpret_cast(&::substrait::_DerivationExpression_ExpressionIntervalCompound_default_instance_), reinterpret_cast(&::substrait::_DerivationExpression_ExpressionPrecisionTimestampTZ_default_instance_), reinterpret_cast(&::substrait::_DerivationExpression_ExpressionStruct_default_instance_), reinterpret_cast(&::substrait::_DerivationExpression_ExpressionNamedStruct_default_instance_), @@ -495,7 +548,7 @@ static ::PROTOBUF_NAMESPACE_ID::Message const * const file_default_instances[] = const char descriptor_table_protodef_substrait_2ftype_5fexpressions_2eproto[] PROTOBUF_SECTION_VARIABLE(protodesc_cold) = "\n substrait/type_expressions.proto\022\tsubs" - "trait\032\024substrait/type.proto\"\322$\n\024Derivati" + "trait\032\024substrait/type.proto\"\200(\n\024Derivati" "onExpression\022\'\n\004bool\030\001 \001(\0132\027.substrait.T" "ype.BooleanH\000\022 \n\002i8\030\002 \001(\0132\022.substrait.Ty" "pe.I8H\000\022\"\n\003i16\030\003 \001(\0132\023.substrait.Type.I1" @@ -509,121 +562,132 @@ const char descriptor_table_protodef_substrait_2ftype_5fexpressions_2eproto[] PR "pB\002\030\001H\000\022$\n\004date\030\020 \001(\0132\024.substrait.Type.D" "ateH\000\022$\n\004time\030\021 \001(\0132\024.substrait.Type.Tim" "eH\000\0225\n\rinterval_year\030\023 \001(\0132\034.substrait.T" - "ype.IntervalYearH\000\0223\n\014interval_day\030\024 \001(\013" - "2\033.substrait.Type.IntervalDayH\000\0227\n\014times" - "tamp_tz\030\035 \001(\0132\033.substrait.Type.Timestamp" - "TZB\002\030\001H\000\022$\n\004uuid\030 \001(\0132\024.substrait.Type." - "UUIDH\000\022I\n\nfixed_char\030\025 \001(\01323.substrait.D" - "erivationExpression.ExpressionFixedCharH" - "\000\022D\n\007varchar\030\026 \001(\01321.substrait.Derivatio" - "nExpression.ExpressionVarCharH\000\022M\n\014fixed" - "_binary\030\027 \001(\01325.substrait.DerivationExpr" - "ession.ExpressionFixedBinaryH\000\022D\n\007decima" - "l\030\030 \001(\01321.substrait.DerivationExpression" - ".ExpressionDecimalH\000\022[\n\023precision_timest" - "amp\030( \001(\0132<.substrait.DerivationExpressi" - "on.ExpressionPrecisionTimestampH\000\022`\n\026pre" - "cision_timestamp_tz\030) \001(\0132>.substrait.De" - "rivationExpression.ExpressionPrecisionTi" - "mestampTZH\000\022B\n\006struct\030\031 \001(\01320.substrait." - "DerivationExpression.ExpressionStructH\000\022" - ">\n\004list\030\033 \001(\0132..substrait.DerivationExpr" - "ession.ExpressionListH\000\022<\n\003map\030\034 \001(\0132-.s" - "ubstrait.DerivationExpression.Expression" - "MapH\000\022M\n\014user_defined\030\036 \001(\01325.substrait." - "DerivationExpression.ExpressionUserDefin" - "edH\000\022\"\n\024user_defined_pointer\030\037 \001(\rB\002\030\001H\000" - "\022\035\n\023type_parameter_name\030! \001(\tH\000\022 \n\026integ" - "er_parameter_name\030\" \001(\tH\000\022\031\n\017integer_lit" - "eral\030# \001(\005H\000\022;\n\010unary_op\030$ \001(\0132\'.substra" - "it.DerivationExpression.UnaryOpH\000\022=\n\tbin" - "ary_op\030% \001(\0132(.substrait.DerivationExpre" - "ssion.BinaryOpH\000\0229\n\007if_else\030& \001(\0132&.subs" - "trait.DerivationExpression.IfElseH\000\022G\n\016r" - "eturn_program\030\' \001(\0132-.substrait.Derivati" - "onExpression.ReturnProgramH\000\032\223\001\n\023Express" - "ionFixedChar\022/\n\006length\030\001 \001(\0132\037.substrait" - ".DerivationExpression\022\031\n\021variation_point" - "er\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrai" - "t.Type.Nullability\032\221\001\n\021ExpressionVarChar" - "\022/\n\006length\030\001 \001(\0132\037.substrait.DerivationE" - "xpression\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013" - "nullability\030\003 \001(\0162\033.substrait.Type.Nulla" - "bility\032\225\001\n\025ExpressionFixedBinary\022/\n\006leng" - "th\030\001 \001(\0132\037.substrait.DerivationExpressio" - "n\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nullabil" - "ity\030\003 \001(\0162\033.substrait.Type.Nullability\032\304" - "\001\n\021ExpressionDecimal\022.\n\005scale\030\001 \001(\0132\037.su" - "bstrait.DerivationExpression\0222\n\tprecisio" - "n\030\002 \001(\0132\037.substrait.DerivationExpression" - "\022\031\n\021variation_pointer\030\003 \001(\r\0220\n\013nullabili" - "ty\030\004 \001(\0162\033.substrait.Type.Nullability\032\237\001" - "\n\034ExpressionPrecisionTimestamp\0222\n\tprecis" - "ion\030\001 \001(\0132\037.substrait.DerivationExpressi" - "on\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nullabi" - "lity\030\003 \001(\0162\033.substrait.Type.Nullability\032" - "\241\001\n\036ExpressionPrecisionTimestampTZ\0222\n\tpr" - "ecision\030\001 \001(\0132\037.substrait.DerivationExpr" - "ession\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nul" - "lability\030\003 \001(\0162\033.substrait.Type.Nullabil" - "ity\032\217\001\n\020ExpressionStruct\022.\n\005types\030\001 \003(\0132" + "ype.IntervalYearH\000\0227\n\014timestamp_tz\030\035 \001(\013" + "2\033.substrait.Type.TimestampTZB\002\030\001H\000\022$\n\004u" + "uid\030 \001(\0132\024.substrait.Type.UUIDH\000\022M\n\014int" + "erval_day\030\024 \001(\01325.substrait.DerivationEx" + "pression.ExpressionIntervalDayH\000\022W\n\021inte" + "rval_compound\030* \001(\0132:.substrait.Derivati" + "onExpression.ExpressionIntervalCompoundH" + "\000\022I\n\nfixed_char\030\025 \001(\01323.substrait.Deriva" + "tionExpression.ExpressionFixedCharH\000\022D\n\007" + "varchar\030\026 \001(\01321.substrait.DerivationExpr" + "ession.ExpressionVarCharH\000\022M\n\014fixed_bina" + "ry\030\027 \001(\01325.substrait.DerivationExpressio" + "n.ExpressionFixedBinaryH\000\022D\n\007decimal\030\030 \001" + "(\01321.substrait.DerivationExpression.Expr" + "essionDecimalH\000\022[\n\023precision_timestamp\030(" + " \001(\0132<.substrait.DerivationExpression.Ex" + "pressionPrecisionTimestampH\000\022`\n\026precisio" + "n_timestamp_tz\030) \001(\0132>.substrait.Derivat" + "ionExpression.ExpressionPrecisionTimesta" + "mpTZH\000\022B\n\006struct\030\031 \001(\01320.substrait.Deriv" + "ationExpression.ExpressionStructH\000\022>\n\004li" + "st\030\033 \001(\0132..substrait.DerivationExpressio" + "n.ExpressionListH\000\022<\n\003map\030\034 \001(\0132-.substr" + "ait.DerivationExpression.ExpressionMapH\000" + "\022M\n\014user_defined\030\036 \001(\01325.substrait.Deriv" + "ationExpression.ExpressionUserDefinedH\000\022" + "\"\n\024user_defined_pointer\030\037 \001(\rB\002\030\001H\000\022\035\n\023t" + "ype_parameter_name\030! \001(\tH\000\022 \n\026integer_pa" + "rameter_name\030\" \001(\tH\000\022\031\n\017integer_literal\030" + "# \001(\005H\000\022;\n\010unary_op\030$ \001(\0132\'.substrait.De" + "rivationExpression.UnaryOpH\000\022=\n\tbinary_o" + "p\030% \001(\0132(.substrait.DerivationExpression" + ".BinaryOpH\000\0229\n\007if_else\030& \001(\0132&.substrait" + ".DerivationExpression.IfElseH\000\022G\n\016return" + "_program\030\' \001(\0132-.substrait.DerivationExp" + "ression.ReturnProgramH\000\032\223\001\n\023ExpressionFi" + "xedChar\022/\n\006length\030\001 \001(\0132\037.substrait.Deri" + "vationExpression\022\031\n\021variation_pointer\030\002 " + "\001(\r\0220\n\013nullability\030\003 \001(\0162\033.substrait.Typ" + "e.Nullability\032\221\001\n\021ExpressionVarChar\022/\n\006l" + "ength\030\001 \001(\0132\037.substrait.DerivationExpres" + "sion\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nulla" + "bility\030\003 \001(\0162\033.substrait.Type.Nullabilit" + "y\032\225\001\n\025ExpressionFixedBinary\022/\n\006length\030\001 " + "\001(\0132\037.substrait.DerivationExpression\022\031\n\021" + "variation_pointer\030\002 \001(\r\0220\n\013nullability\030\003" + " \001(\0162\033.substrait.Type.Nullability\032\304\001\n\021Ex" + "pressionDecimal\022.\n\005scale\030\001 \001(\0132\037.substra" + "it.DerivationExpression\0222\n\tprecision\030\002 \001" + "(\0132\037.substrait.DerivationExpression\022\031\n\021v" + "ariation_pointer\030\003 \001(\r\0220\n\013nullability\030\004 " + "\001(\0162\033.substrait.Type.Nullability\032\237\001\n\034Exp" + "ressionPrecisionTimestamp\0222\n\tprecision\030\001" + " \001(\0132\037.substrait.DerivationExpression\022\031\n" + "\021variation_pointer\030\002 \001(\r\0220\n\013nullability\030" + "\003 \001(\0162\033.substrait.Type.Nullability\032\230\001\n\025E" + "xpressionIntervalDay\0222\n\tprecision\030\001 \001(\0132" + "\037.substrait.DerivationExpression\022\031\n\021vari" + "ation_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\016" + "2\033.substrait.Type.Nullability\032\235\001\n\032Expres" + "sionIntervalCompound\0222\n\tprecision\030\001 \001(\0132" "\037.substrait.DerivationExpression\022\031\n\021vari" "ation_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\016" - "2\033.substrait.Type.Nullability\032h\n\025Express" - "ionNamedStruct\022\r\n\005names\030\001 \003(\t\022@\n\006struct\030" - "\002 \001(\01320.substrait.DerivationExpression.E" - "xpressionStruct\032\214\001\n\016ExpressionList\022-\n\004ty" - "pe\030\001 \001(\0132\037.substrait.DerivationExpressio" - "n\022\031\n\021variation_pointer\030\002 \001(\r\0220\n\013nullabil" - "ity\030\003 \001(\0162\033.substrait.Type.Nullability\032\272" - "\001\n\rExpressionMap\022,\n\003key\030\001 \001(\0132\037.substrai" - "t.DerivationExpression\022.\n\005value\030\002 \001(\0132\037." - "substrait.DerivationExpression\022\031\n\021variat" - "ion_pointer\030\003 \001(\r\0220\n\013nullability\030\004 \001(\0162\033" - ".substrait.Type.Nullability\032z\n\025Expressio" - "nUserDefined\022\024\n\014type_pointer\030\001 \001(\r\022\031\n\021va" - "riation_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001" - "(\0162\033.substrait.Type.Nullability\032\251\001\n\006IfEl" - "se\0225\n\014if_condition\030\001 \001(\0132\037.substrait.Der" - "ivationExpression\0222\n\tif_return\030\002 \001(\0132\037.s" - "ubstrait.DerivationExpression\0224\n\013else_re" - "turn\030\003 \001(\0132\037.substrait.DerivationExpress" - "ion\032\312\001\n\007UnaryOp\022D\n\007op_type\030\001 \001(\01623.subst" - "rait.DerivationExpression.UnaryOp.UnaryO" - "pType\022,\n\003arg\030\002 \001(\0132\037.substrait.Derivatio" - "nExpression\"K\n\013UnaryOpType\022\035\n\031UNARY_OP_T" - "YPE_UNSPECIFIED\020\000\022\035\n\031UNARY_OP_TYPE_BOOLE" - "AN_NOT\020\001\032\240\004\n\010BinaryOp\022F\n\007op_type\030\001 \001(\01625" - ".substrait.DerivationExpression.BinaryOp" - ".BinaryOpType\022-\n\004arg1\030\002 \001(\0132\037.substrait." - "DerivationExpression\022-\n\004arg2\030\003 \001(\0132\037.sub" - "strait.DerivationExpression\"\355\002\n\014BinaryOp" - "Type\022\036\n\032BINARY_OP_TYPE_UNSPECIFIED\020\000\022\027\n\023" - "BINARY_OP_TYPE_PLUS\020\001\022\030\n\024BINARY_OP_TYPE_" - "MINUS\020\002\022\033\n\027BINARY_OP_TYPE_MULTIPLY\020\003\022\031\n\025" - "BINARY_OP_TYPE_DIVIDE\020\004\022\026\n\022BINARY_OP_TYP" - "E_MIN\020\005\022\026\n\022BINARY_OP_TYPE_MAX\020\006\022\037\n\033BINAR" - "Y_OP_TYPE_GREATER_THAN\020\007\022\034\n\030BINARY_OP_TY" - "PE_LESS_THAN\020\010\022\026\n\022BINARY_OP_TYPE_AND\020\t\022\025" - "\n\021BINARY_OP_TYPE_OR\020\n\022\031\n\025BINARY_OP_TYPE_" - "EQUALS\020\013\022\031\n\025BINARY_OP_TYPE_COVERS\020\014\032\352\001\n\r" - "ReturnProgram\022M\n\013assignments\030\001 \003(\01328.sub" - "strait.DerivationExpression.ReturnProgra" - "m.Assignment\0229\n\020final_expression\030\002 \001(\0132\037" - ".substrait.DerivationExpression\032O\n\nAssig" - "nment\022\014\n\004name\030\001 \001(\t\0223\n\nexpression\030\002 \001(\0132" - "\037.substrait.DerivationExpressionB\006\n\004kind" - "BW\n\022io.substrait.protoP\001Z*github.com/sub" - "strait-io/substrait-go/proto\252\002\022Substrait" - ".Protobufb\006proto3" + "2\033.substrait.Type.Nullability\032\241\001\n\036Expres" + "sionPrecisionTimestampTZ\0222\n\tprecision\030\001 " + "\001(\0132\037.substrait.DerivationExpression\022\031\n\021" + "variation_pointer\030\002 \001(\r\0220\n\013nullability\030\003" + " \001(\0162\033.substrait.Type.Nullability\032\217\001\n\020Ex" + "pressionStruct\022.\n\005types\030\001 \003(\0132\037.substrai" + "t.DerivationExpression\022\031\n\021variation_poin" + "ter\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.substra" + "it.Type.Nullability\032h\n\025ExpressionNamedSt" + "ruct\022\r\n\005names\030\001 \003(\t\022@\n\006struct\030\002 \001(\01320.su" + "bstrait.DerivationExpression.ExpressionS" + "truct\032\214\001\n\016ExpressionList\022-\n\004type\030\001 \001(\0132\037" + ".substrait.DerivationExpression\022\031\n\021varia" + "tion_pointer\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162" + "\033.substrait.Type.Nullability\032\272\001\n\rExpress" + "ionMap\022,\n\003key\030\001 \001(\0132\037.substrait.Derivati" + "onExpression\022.\n\005value\030\002 \001(\0132\037.substrait." + "DerivationExpression\022\031\n\021variation_pointe" + "r\030\003 \001(\r\0220\n\013nullability\030\004 \001(\0162\033.substrait" + ".Type.Nullability\032z\n\025ExpressionUserDefin" + "ed\022\024\n\014type_pointer\030\001 \001(\r\022\031\n\021variation_po" + "inter\030\002 \001(\r\0220\n\013nullability\030\003 \001(\0162\033.subst" + "rait.Type.Nullability\032\251\001\n\006IfElse\0225\n\014if_c" + "ondition\030\001 \001(\0132\037.substrait.DerivationExp" + "ression\0222\n\tif_return\030\002 \001(\0132\037.substrait.D" + "erivationExpression\0224\n\013else_return\030\003 \001(\013" + "2\037.substrait.DerivationExpression\032\312\001\n\007Un" + "aryOp\022D\n\007op_type\030\001 \001(\01623.substrait.Deriv" + "ationExpression.UnaryOp.UnaryOpType\022,\n\003a" + "rg\030\002 \001(\0132\037.substrait.DerivationExpressio" + "n\"K\n\013UnaryOpType\022\035\n\031UNARY_OP_TYPE_UNSPEC" + "IFIED\020\000\022\035\n\031UNARY_OP_TYPE_BOOLEAN_NOT\020\001\032\240" + "\004\n\010BinaryOp\022F\n\007op_type\030\001 \001(\01625.substrait" + ".DerivationExpression.BinaryOp.BinaryOpT" + "ype\022-\n\004arg1\030\002 \001(\0132\037.substrait.Derivation" + "Expression\022-\n\004arg2\030\003 \001(\0132\037.substrait.Der" + "ivationExpression\"\355\002\n\014BinaryOpType\022\036\n\032BI" + "NARY_OP_TYPE_UNSPECIFIED\020\000\022\027\n\023BINARY_OP_" + "TYPE_PLUS\020\001\022\030\n\024BINARY_OP_TYPE_MINUS\020\002\022\033\n" + "\027BINARY_OP_TYPE_MULTIPLY\020\003\022\031\n\025BINARY_OP_" + "TYPE_DIVIDE\020\004\022\026\n\022BINARY_OP_TYPE_MIN\020\005\022\026\n" + "\022BINARY_OP_TYPE_MAX\020\006\022\037\n\033BINARY_OP_TYPE_" + "GREATER_THAN\020\007\022\034\n\030BINARY_OP_TYPE_LESS_TH" + "AN\020\010\022\026\n\022BINARY_OP_TYPE_AND\020\t\022\025\n\021BINARY_O" + "P_TYPE_OR\020\n\022\031\n\025BINARY_OP_TYPE_EQUALS\020\013\022\031" + "\n\025BINARY_OP_TYPE_COVERS\020\014\032\352\001\n\rReturnProg" + "ram\022M\n\013assignments\030\001 \003(\01328.substrait.Der" + "ivationExpression.ReturnProgram.Assignme" + "nt\0229\n\020final_expression\030\002 \001(\0132\037.substrait" + ".DerivationExpression\032O\n\nAssignment\022\014\n\004n" + "ame\030\001 \001(\t\0223\n\nexpression\030\002 \001(\0132\037.substrai" + "t.DerivationExpressionB\006\n\004kindBW\n\022io.sub" + "strait.protoP\001Z*github.com/substrait-io/" + "substrait-go/proto\252\002\022Substrait.Protobufb" + "\006proto3" ; static const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable*const descriptor_table_substrait_2ftype_5fexpressions_2eproto_deps[1] = { &::descriptor_table_substrait_2ftype_2eproto, }; static ::PROTOBUF_NAMESPACE_ID::internal::once_flag descriptor_table_substrait_2ftype_5fexpressions_2eproto_once; const ::PROTOBUF_NAMESPACE_ID::internal::DescriptorTable descriptor_table_substrait_2ftype_5fexpressions_2eproto = { - false, false, 4857, descriptor_table_protodef_substrait_2ftype_5fexpressions_2eproto, "substrait/type_expressions.proto", - &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, descriptor_table_substrait_2ftype_5fexpressions_2eproto_deps, 1, 17, + false, false, 5287, descriptor_table_protodef_substrait_2ftype_5fexpressions_2eproto, "substrait/type_expressions.proto", + &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, descriptor_table_substrait_2ftype_5fexpressions_2eproto_deps, 1, 19, schemas, file_default_instances, TableStruct_substrait_2ftype_5fexpressions_2eproto::offsets, file_level_metadata_substrait_2ftype_5fexpressions_2eproto, file_level_enum_descriptors_substrait_2ftype_5fexpressions_2eproto, file_level_service_descriptors_substrait_2ftype_5fexpressions_2eproto, }; @@ -1561,39 +1625,569 @@ void DerivationExpression_ExpressionDecimal::Clear() { _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* DerivationExpression_ExpressionDecimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* DerivationExpression_ExpressionDecimal::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // .substrait.DerivationExpression scale = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + ptr = ctx->ParseMessage(_internal_mutable_scale(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.DerivationExpression precision = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint32 variation_pointer = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Type.Nullability nullability = 4; + case 4: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* DerivationExpression_ExpressionDecimal::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionDecimal) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // .substrait.DerivationExpression scale = 1; + if (this->_internal_has_scale()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 1, _Internal::scale(this), target, stream); + } + + // .substrait.DerivationExpression precision = 2; + if (this->_internal_has_precision()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 2, _Internal::precision(this), target, stream); + } + + // uint32 variation_pointer = 3; + if (this->_internal_variation_pointer() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(3, this->_internal_variation_pointer(), target); + } + + // .substrait.Type.Nullability nullability = 4; + if (this->_internal_nullability() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 4, this->_internal_nullability(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionDecimal) + return target; +} + +size_t DerivationExpression_ExpressionDecimal::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionDecimal) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .substrait.DerivationExpression scale = 1; + if (this->_internal_has_scale()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *scale_); + } + + // .substrait.DerivationExpression precision = 2; + if (this->_internal_has_precision()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *precision_); + } + + // uint32 variation_pointer = 3; + if (this->_internal_variation_pointer() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); + } + + // .substrait.Type.Nullability nullability = 4; + if (this->_internal_nullability() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionDecimal::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + DerivationExpression_ExpressionDecimal::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionDecimal::GetClassData() const { return &_class_data_; } + +void DerivationExpression_ExpressionDecimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void DerivationExpression_ExpressionDecimal::MergeFrom(const DerivationExpression_ExpressionDecimal& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionDecimal) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_has_scale()) { + _internal_mutable_scale()->::substrait::DerivationExpression::MergeFrom(from._internal_scale()); + } + if (from._internal_has_precision()) { + _internal_mutable_precision()->::substrait::DerivationExpression::MergeFrom(from._internal_precision()); + } + if (from._internal_variation_pointer() != 0) { + _internal_set_variation_pointer(from._internal_variation_pointer()); + } + if (from._internal_nullability() != 0) { + _internal_set_nullability(from._internal_nullability()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void DerivationExpression_ExpressionDecimal::CopyFrom(const DerivationExpression_ExpressionDecimal& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionDecimal) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool DerivationExpression_ExpressionDecimal::IsInitialized() const { + return true; +} + +void DerivationExpression_ExpressionDecimal::InternalSwap(DerivationExpression_ExpressionDecimal* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionDecimal, nullability_) + + sizeof(DerivationExpression_ExpressionDecimal::nullability_) + - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionDecimal, scale_)>( + reinterpret_cast(&scale_), + reinterpret_cast(&other->scale_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionDecimal::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[3]); +} + +// =================================================================== + +class DerivationExpression_ExpressionPrecisionTimestamp::_Internal { + public: + static const ::substrait::DerivationExpression& precision(const DerivationExpression_ExpressionPrecisionTimestamp* msg); +}; + +const ::substrait::DerivationExpression& +DerivationExpression_ExpressionPrecisionTimestamp::_Internal::precision(const DerivationExpression_ExpressionPrecisionTimestamp* msg) { + return *msg->precision_; +} +DerivationExpression_ExpressionPrecisionTimestamp::DerivationExpression_ExpressionPrecisionTimestamp(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +} +DerivationExpression_ExpressionPrecisionTimestamp::DerivationExpression_ExpressionPrecisionTimestamp(const DerivationExpression_ExpressionPrecisionTimestamp& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + if (from._internal_has_precision()) { + precision_ = new ::substrait::DerivationExpression(*from.precision_); + } else { + precision_ = nullptr; + } + ::memcpy(&variation_pointer_, &from.variation_pointer_, + static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + // @@protoc_insertion_point(copy_constructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +} + +inline void DerivationExpression_ExpressionPrecisionTimestamp::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&precision_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&precision_)) + sizeof(nullability_)); +} + +DerivationExpression_ExpressionPrecisionTimestamp::~DerivationExpression_ExpressionPrecisionTimestamp() { + // @@protoc_insertion_point(destructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void DerivationExpression_ExpressionPrecisionTimestamp::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (this != internal_default_instance()) delete precision_; +} + +void DerivationExpression_ExpressionPrecisionTimestamp::ArenaDtor(void* object) { + DerivationExpression_ExpressionPrecisionTimestamp* _this = reinterpret_cast< DerivationExpression_ExpressionPrecisionTimestamp* >(object); + (void)_this; +} +void DerivationExpression_ExpressionPrecisionTimestamp::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void DerivationExpression_ExpressionPrecisionTimestamp::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void DerivationExpression_ExpressionPrecisionTimestamp::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; + ::memset(&variation_pointer_, 0, static_cast( + reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* DerivationExpression_ExpressionPrecisionTimestamp::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +#define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure + while (!ctx->Done(&ptr)) { + uint32_t tag; + ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); + switch (tag >> 3) { + // .substrait.DerivationExpression precision = 1; + case 1: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // uint32 variation_pointer = 2; + case 2: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; + // .substrait.Type.Nullability nullability = 3; + case 3: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { + uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); + CHK_(ptr); + _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); + } else + goto handle_unusual; + continue; + default: + goto handle_unusual; + } // switch + handle_unusual: + if ((tag == 0) || ((tag & 7) == 4)) { + CHK_(ptr); + ctx->SetLastTag(tag); + goto message_done; + } + ptr = UnknownFieldParse( + tag, + _internal_metadata_.mutable_unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(), + ptr, ctx); + CHK_(ptr != nullptr); + } // while +message_done: + return ptr; +failure: + ptr = nullptr; + goto message_done; +#undef CHK_ +} + +uint8_t* DerivationExpression_ExpressionPrecisionTimestamp::_InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { + // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + // .substrait.DerivationExpression precision = 1; + if (this->_internal_has_precision()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 1, _Internal::precision(this), target, stream); + } + + // uint32 variation_pointer = 2; + if (this->_internal_variation_pointer() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_variation_pointer(), target); + } + + // .substrait.Type.Nullability nullability = 3; + if (this->_internal_nullability() != 0) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( + 3, this->_internal_nullability(), target); + } + + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( + _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); + } + // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + return target; +} + +size_t DerivationExpression_ExpressionPrecisionTimestamp::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + size_t total_size = 0; + + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + // .substrait.DerivationExpression precision = 1; + if (this->_internal_has_precision()) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *precision_); + } + + // uint32 variation_pointer = 2; + if (this->_internal_variation_pointer() != 0) { + total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); + } + + // .substrait.Type.Nullability nullability = 3; + if (this->_internal_nullability() != 0) { + total_size += 1 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); + } + + return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); +} + +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionPrecisionTimestamp::_class_data_ = { + ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, + DerivationExpression_ExpressionPrecisionTimestamp::MergeImpl +}; +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionPrecisionTimestamp::GetClassData() const { return &_class_data_; } + +void DerivationExpression_ExpressionPrecisionTimestamp::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, + const ::PROTOBUF_NAMESPACE_ID::Message& from) { + static_cast(to)->MergeFrom( + static_cast(from)); +} + + +void DerivationExpression_ExpressionPrecisionTimestamp::MergeFrom(const DerivationExpression_ExpressionPrecisionTimestamp& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + GOOGLE_DCHECK_NE(&from, this); + uint32_t cached_has_bits = 0; + (void) cached_has_bits; + + if (from._internal_has_precision()) { + _internal_mutable_precision()->::substrait::DerivationExpression::MergeFrom(from._internal_precision()); + } + if (from._internal_variation_pointer() != 0) { + _internal_set_variation_pointer(from._internal_variation_pointer()); + } + if (from._internal_nullability() != 0) { + _internal_set_nullability(from._internal_nullability()); + } + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); +} + +void DerivationExpression_ExpressionPrecisionTimestamp::CopyFrom(const DerivationExpression_ExpressionPrecisionTimestamp& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + if (&from == this) return; + Clear(); + MergeFrom(from); +} + +bool DerivationExpression_ExpressionPrecisionTimestamp::IsInitialized() const { + return true; +} + +void DerivationExpression_ExpressionPrecisionTimestamp::InternalSwap(DerivationExpression_ExpressionPrecisionTimestamp* other) { + using std::swap; + _internal_metadata_.InternalSwap(&other->_internal_metadata_); + ::PROTOBUF_NAMESPACE_ID::internal::memswap< + PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionPrecisionTimestamp, nullability_) + + sizeof(DerivationExpression_ExpressionPrecisionTimestamp::nullability_) + - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionPrecisionTimestamp, precision_)>( + reinterpret_cast(&precision_), + reinterpret_cast(&other->precision_)); +} + +::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionPrecisionTimestamp::GetMetadata() const { + return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( + &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[4]); +} + +// =================================================================== + +class DerivationExpression_ExpressionIntervalDay::_Internal { + public: + static const ::substrait::DerivationExpression& precision(const DerivationExpression_ExpressionIntervalDay* msg); +}; + +const ::substrait::DerivationExpression& +DerivationExpression_ExpressionIntervalDay::_Internal::precision(const DerivationExpression_ExpressionIntervalDay* msg) { + return *msg->precision_; +} +DerivationExpression_ExpressionIntervalDay::DerivationExpression_ExpressionIntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned) + : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { + SharedCtor(); + if (!is_message_owned) { + RegisterArenaDtor(arena); + } + // @@protoc_insertion_point(arena_constructor:substrait.DerivationExpression.ExpressionIntervalDay) +} +DerivationExpression_ExpressionIntervalDay::DerivationExpression_ExpressionIntervalDay(const DerivationExpression_ExpressionIntervalDay& from) + : ::PROTOBUF_NAMESPACE_ID::Message() { + _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); + if (from._internal_has_precision()) { + precision_ = new ::substrait::DerivationExpression(*from.precision_); + } else { + precision_ = nullptr; + } + ::memcpy(&variation_pointer_, &from.variation_pointer_, + static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + // @@protoc_insertion_point(copy_constructor:substrait.DerivationExpression.ExpressionIntervalDay) +} + +inline void DerivationExpression_ExpressionIntervalDay::SharedCtor() { +::memset(reinterpret_cast(this) + static_cast( + reinterpret_cast(&precision_) - reinterpret_cast(this)), + 0, static_cast(reinterpret_cast(&nullability_) - + reinterpret_cast(&precision_)) + sizeof(nullability_)); +} + +DerivationExpression_ExpressionIntervalDay::~DerivationExpression_ExpressionIntervalDay() { + // @@protoc_insertion_point(destructor:substrait.DerivationExpression.ExpressionIntervalDay) + if (GetArenaForAllocation() != nullptr) return; + SharedDtor(); + _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +inline void DerivationExpression_ExpressionIntervalDay::SharedDtor() { + GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); + if (this != internal_default_instance()) delete precision_; +} + +void DerivationExpression_ExpressionIntervalDay::ArenaDtor(void* object) { + DerivationExpression_ExpressionIntervalDay* _this = reinterpret_cast< DerivationExpression_ExpressionIntervalDay* >(object); + (void)_this; +} +void DerivationExpression_ExpressionIntervalDay::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +} +void DerivationExpression_ExpressionIntervalDay::SetCachedSize(int size) const { + _cached_size_.Set(size); +} + +void DerivationExpression_ExpressionIntervalDay::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.DerivationExpression.ExpressionIntervalDay) + uint32_t cached_has_bits = 0; + // Prevent compiler warnings about cached_has_bits being unused + (void) cached_has_bits; + + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; + ::memset(&variation_pointer_, 0, static_cast( + reinterpret_cast(&nullability_) - + reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); + _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); +} + +const char* DerivationExpression_ExpressionIntervalDay::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; ptr = ::PROTOBUF_NAMESPACE_ID::internal::ReadTag(ptr, &tag); switch (tag >> 3) { - // .substrait.DerivationExpression scale = 1; + // .substrait.DerivationExpression precision = 1; case 1: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 10)) { - ptr = ctx->ParseMessage(_internal_mutable_scale(), ptr); + ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); CHK_(ptr); } else goto handle_unusual; continue; - // .substrait.DerivationExpression precision = 2; + // uint32 variation_pointer = 2; case 2: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 18)) { - ptr = ctx->ParseMessage(_internal_mutable_precision(), ptr); + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 16)) { + variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); CHK_(ptr); } else goto handle_unusual; continue; - // uint32 variation_pointer = 3; + // .substrait.Type.Nullability nullability = 3; case 3: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 24)) { - variation_pointer_ = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint32(&ptr); - CHK_(ptr); - } else - goto handle_unusual; - continue; - // .substrait.Type.Nullability nullability = 4; - case 4: - if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 32)) { uint64_t val = ::PROTOBUF_NAMESPACE_ID::internal::ReadVarint64(&ptr); CHK_(ptr); _internal_set_nullability(static_cast<::substrait::Type_Nullability>(val)); @@ -1623,77 +2217,62 @@ const char* DerivationExpression_ExpressionDecimal::_InternalParse(const char* p #undef CHK_ } -uint8_t* DerivationExpression_ExpressionDecimal::_InternalSerialize( +uint8_t* DerivationExpression_ExpressionIntervalDay::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionDecimal) + // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionIntervalDay) uint32_t cached_has_bits = 0; (void) cached_has_bits; - // .substrait.DerivationExpression scale = 1; - if (this->_internal_has_scale()) { - target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: - InternalWriteMessage( - 1, _Internal::scale(this), target, stream); - } - - // .substrait.DerivationExpression precision = 2; + // .substrait.DerivationExpression precision = 1; if (this->_internal_has_precision()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: InternalWriteMessage( - 2, _Internal::precision(this), target, stream); + 1, _Internal::precision(this), target, stream); } - // uint32 variation_pointer = 3; + // uint32 variation_pointer = 2; if (this->_internal_variation_pointer() != 0) { target = stream->EnsureSpace(target); - target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(3, this->_internal_variation_pointer(), target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteUInt32ToArray(2, this->_internal_variation_pointer(), target); } - // .substrait.Type.Nullability nullability = 4; + // .substrait.Type.Nullability nullability = 3; if (this->_internal_nullability() != 0) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::WriteEnumToArray( - 4, this->_internal_nullability(), target); + 3, this->_internal_nullability(), target); } if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionDecimal) + // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionIntervalDay) return target; } -size_t DerivationExpression_ExpressionDecimal::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionDecimal) +size_t DerivationExpression_ExpressionIntervalDay::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionIntervalDay) size_t total_size = 0; uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; - // .substrait.DerivationExpression scale = 1; - if (this->_internal_has_scale()) { - total_size += 1 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *scale_); - } - - // .substrait.DerivationExpression precision = 2; + // .substrait.DerivationExpression precision = 1; if (this->_internal_has_precision()) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( *precision_); } - // uint32 variation_pointer = 3; + // uint32 variation_pointer = 2; if (this->_internal_variation_pointer() != 0) { total_size += ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::UInt32SizePlusOne(this->_internal_variation_pointer()); } - // .substrait.Type.Nullability nullability = 4; + // .substrait.Type.Nullability nullability = 3; if (this->_internal_nullability() != 0) { total_size += 1 + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::EnumSize(this->_internal_nullability()); @@ -1702,28 +2281,25 @@ size_t DerivationExpression_ExpressionDecimal::ByteSizeLong() const { return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionDecimal::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionIntervalDay::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - DerivationExpression_ExpressionDecimal::MergeImpl + DerivationExpression_ExpressionIntervalDay::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionDecimal::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionIntervalDay::GetClassData() const { return &_class_data_; } -void DerivationExpression_ExpressionDecimal::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void DerivationExpression_ExpressionIntervalDay::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void DerivationExpression_ExpressionDecimal::MergeFrom(const DerivationExpression_ExpressionDecimal& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionDecimal) +void DerivationExpression_ExpressionIntervalDay::MergeFrom(const DerivationExpression_ExpressionIntervalDay& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionIntervalDay) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; - if (from._internal_has_scale()) { - _internal_mutable_scale()->::substrait::DerivationExpression::MergeFrom(from._internal_scale()); - } if (from._internal_has_precision()) { _internal_mutable_precision()->::substrait::DerivationExpression::MergeFrom(from._internal_precision()); } @@ -1736,55 +2312,55 @@ void DerivationExpression_ExpressionDecimal::MergeFrom(const DerivationExpressio _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void DerivationExpression_ExpressionDecimal::CopyFrom(const DerivationExpression_ExpressionDecimal& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionDecimal) +void DerivationExpression_ExpressionIntervalDay::CopyFrom(const DerivationExpression_ExpressionIntervalDay& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionIntervalDay) if (&from == this) return; Clear(); MergeFrom(from); } -bool DerivationExpression_ExpressionDecimal::IsInitialized() const { +bool DerivationExpression_ExpressionIntervalDay::IsInitialized() const { return true; } -void DerivationExpression_ExpressionDecimal::InternalSwap(DerivationExpression_ExpressionDecimal* other) { +void DerivationExpression_ExpressionIntervalDay::InternalSwap(DerivationExpression_ExpressionIntervalDay* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionDecimal, nullability_) - + sizeof(DerivationExpression_ExpressionDecimal::nullability_) - - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionDecimal, scale_)>( - reinterpret_cast(&scale_), - reinterpret_cast(&other->scale_)); + PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionIntervalDay, nullability_) + + sizeof(DerivationExpression_ExpressionIntervalDay::nullability_) + - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionIntervalDay, precision_)>( + reinterpret_cast(&precision_), + reinterpret_cast(&other->precision_)); } -::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionDecimal::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionIntervalDay::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[3]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[5]); } // =================================================================== -class DerivationExpression_ExpressionPrecisionTimestamp::_Internal { +class DerivationExpression_ExpressionIntervalCompound::_Internal { public: - static const ::substrait::DerivationExpression& precision(const DerivationExpression_ExpressionPrecisionTimestamp* msg); + static const ::substrait::DerivationExpression& precision(const DerivationExpression_ExpressionIntervalCompound* msg); }; const ::substrait::DerivationExpression& -DerivationExpression_ExpressionPrecisionTimestamp::_Internal::precision(const DerivationExpression_ExpressionPrecisionTimestamp* msg) { +DerivationExpression_ExpressionIntervalCompound::_Internal::precision(const DerivationExpression_ExpressionIntervalCompound* msg) { return *msg->precision_; } -DerivationExpression_ExpressionPrecisionTimestamp::DerivationExpression_ExpressionPrecisionTimestamp(::PROTOBUF_NAMESPACE_ID::Arena* arena, +DerivationExpression_ExpressionIntervalCompound::DerivationExpression_ExpressionIntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned) : ::PROTOBUF_NAMESPACE_ID::Message(arena, is_message_owned) { SharedCtor(); if (!is_message_owned) { RegisterArenaDtor(arena); } - // @@protoc_insertion_point(arena_constructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + // @@protoc_insertion_point(arena_constructor:substrait.DerivationExpression.ExpressionIntervalCompound) } -DerivationExpression_ExpressionPrecisionTimestamp::DerivationExpression_ExpressionPrecisionTimestamp(const DerivationExpression_ExpressionPrecisionTimestamp& from) +DerivationExpression_ExpressionIntervalCompound::DerivationExpression_ExpressionIntervalCompound(const DerivationExpression_ExpressionIntervalCompound& from) : ::PROTOBUF_NAMESPACE_ID::Message() { _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); if (from._internal_has_precision()) { @@ -1795,40 +2371,40 @@ DerivationExpression_ExpressionPrecisionTimestamp::DerivationExpression_Expressi ::memcpy(&variation_pointer_, &from.variation_pointer_, static_cast(reinterpret_cast(&nullability_) - reinterpret_cast(&variation_pointer_)) + sizeof(nullability_)); - // @@protoc_insertion_point(copy_constructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + // @@protoc_insertion_point(copy_constructor:substrait.DerivationExpression.ExpressionIntervalCompound) } -inline void DerivationExpression_ExpressionPrecisionTimestamp::SharedCtor() { +inline void DerivationExpression_ExpressionIntervalCompound::SharedCtor() { ::memset(reinterpret_cast(this) + static_cast( reinterpret_cast(&precision_) - reinterpret_cast(this)), 0, static_cast(reinterpret_cast(&nullability_) - reinterpret_cast(&precision_)) + sizeof(nullability_)); } -DerivationExpression_ExpressionPrecisionTimestamp::~DerivationExpression_ExpressionPrecisionTimestamp() { - // @@protoc_insertion_point(destructor:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +DerivationExpression_ExpressionIntervalCompound::~DerivationExpression_ExpressionIntervalCompound() { + // @@protoc_insertion_point(destructor:substrait.DerivationExpression.ExpressionIntervalCompound) if (GetArenaForAllocation() != nullptr) return; SharedDtor(); _internal_metadata_.Delete<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -inline void DerivationExpression_ExpressionPrecisionTimestamp::SharedDtor() { +inline void DerivationExpression_ExpressionIntervalCompound::SharedDtor() { GOOGLE_DCHECK(GetArenaForAllocation() == nullptr); if (this != internal_default_instance()) delete precision_; } -void DerivationExpression_ExpressionPrecisionTimestamp::ArenaDtor(void* object) { - DerivationExpression_ExpressionPrecisionTimestamp* _this = reinterpret_cast< DerivationExpression_ExpressionPrecisionTimestamp* >(object); +void DerivationExpression_ExpressionIntervalCompound::ArenaDtor(void* object) { + DerivationExpression_ExpressionIntervalCompound* _this = reinterpret_cast< DerivationExpression_ExpressionIntervalCompound* >(object); (void)_this; } -void DerivationExpression_ExpressionPrecisionTimestamp::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { +void DerivationExpression_ExpressionIntervalCompound::RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena*) { } -void DerivationExpression_ExpressionPrecisionTimestamp::SetCachedSize(int size) const { +void DerivationExpression_ExpressionIntervalCompound::SetCachedSize(int size) const { _cached_size_.Set(size); } -void DerivationExpression_ExpressionPrecisionTimestamp::Clear() { -// @@protoc_insertion_point(message_clear_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +void DerivationExpression_ExpressionIntervalCompound::Clear() { +// @@protoc_insertion_point(message_clear_start:substrait.DerivationExpression.ExpressionIntervalCompound) uint32_t cached_has_bits = 0; // Prevent compiler warnings about cached_has_bits being unused (void) cached_has_bits; @@ -1843,7 +2419,7 @@ void DerivationExpression_ExpressionPrecisionTimestamp::Clear() { _internal_metadata_.Clear<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(); } -const char* DerivationExpression_ExpressionPrecisionTimestamp::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { +const char* DerivationExpression_ExpressionIntervalCompound::_InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) { #define CHK_(x) if (PROTOBUF_PREDICT_FALSE(!(x))) goto failure while (!ctx->Done(&ptr)) { uint32_t tag; @@ -1897,9 +2473,9 @@ const char* DerivationExpression_ExpressionPrecisionTimestamp::_InternalParse(co #undef CHK_ } -uint8_t* DerivationExpression_ExpressionPrecisionTimestamp::_InternalSerialize( +uint8_t* DerivationExpression_ExpressionIntervalCompound::_InternalSerialize( uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const { - // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + // @@protoc_insertion_point(serialize_to_array_start:substrait.DerivationExpression.ExpressionIntervalCompound) uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -1928,12 +2504,12 @@ uint8_t* DerivationExpression_ExpressionPrecisionTimestamp::_InternalSerialize( target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); } - // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionPrecisionTimestamp) + // @@protoc_insertion_point(serialize_to_array_end:substrait.DerivationExpression.ExpressionIntervalCompound) return target; } -size_t DerivationExpression_ExpressionPrecisionTimestamp::ByteSizeLong() const { -// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +size_t DerivationExpression_ExpressionIntervalCompound::ByteSizeLong() const { +// @@protoc_insertion_point(message_byte_size_start:substrait.DerivationExpression.ExpressionIntervalCompound) size_t total_size = 0; uint32_t cached_has_bits = 0; @@ -1961,21 +2537,21 @@ size_t DerivationExpression_ExpressionPrecisionTimestamp::ByteSizeLong() const { return MaybeComputeUnknownFieldsSize(total_size, &_cached_size_); } -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionPrecisionTimestamp::_class_data_ = { +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData DerivationExpression_ExpressionIntervalCompound::_class_data_ = { ::PROTOBUF_NAMESPACE_ID::Message::CopyWithSizeCheck, - DerivationExpression_ExpressionPrecisionTimestamp::MergeImpl + DerivationExpression_ExpressionIntervalCompound::MergeImpl }; -const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionPrecisionTimestamp::GetClassData() const { return &_class_data_; } +const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*DerivationExpression_ExpressionIntervalCompound::GetClassData() const { return &_class_data_; } -void DerivationExpression_ExpressionPrecisionTimestamp::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, +void DerivationExpression_ExpressionIntervalCompound::MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from) { - static_cast(to)->MergeFrom( - static_cast(from)); + static_cast(to)->MergeFrom( + static_cast(from)); } -void DerivationExpression_ExpressionPrecisionTimestamp::MergeFrom(const DerivationExpression_ExpressionPrecisionTimestamp& from) { -// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +void DerivationExpression_ExpressionIntervalCompound::MergeFrom(const DerivationExpression_ExpressionIntervalCompound& from) { +// @@protoc_insertion_point(class_specific_merge_from_start:substrait.DerivationExpression.ExpressionIntervalCompound) GOOGLE_DCHECK_NE(&from, this); uint32_t cached_has_bits = 0; (void) cached_has_bits; @@ -1992,32 +2568,32 @@ void DerivationExpression_ExpressionPrecisionTimestamp::MergeFrom(const Derivati _internal_metadata_.MergeFrom<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(from._internal_metadata_); } -void DerivationExpression_ExpressionPrecisionTimestamp::CopyFrom(const DerivationExpression_ExpressionPrecisionTimestamp& from) { -// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionPrecisionTimestamp) +void DerivationExpression_ExpressionIntervalCompound::CopyFrom(const DerivationExpression_ExpressionIntervalCompound& from) { +// @@protoc_insertion_point(class_specific_copy_from_start:substrait.DerivationExpression.ExpressionIntervalCompound) if (&from == this) return; Clear(); MergeFrom(from); } -bool DerivationExpression_ExpressionPrecisionTimestamp::IsInitialized() const { +bool DerivationExpression_ExpressionIntervalCompound::IsInitialized() const { return true; } -void DerivationExpression_ExpressionPrecisionTimestamp::InternalSwap(DerivationExpression_ExpressionPrecisionTimestamp* other) { +void DerivationExpression_ExpressionIntervalCompound::InternalSwap(DerivationExpression_ExpressionIntervalCompound* other) { using std::swap; _internal_metadata_.InternalSwap(&other->_internal_metadata_); ::PROTOBUF_NAMESPACE_ID::internal::memswap< - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionPrecisionTimestamp, nullability_) - + sizeof(DerivationExpression_ExpressionPrecisionTimestamp::nullability_) - - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionPrecisionTimestamp, precision_)>( + PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionIntervalCompound, nullability_) + + sizeof(DerivationExpression_ExpressionIntervalCompound::nullability_) + - PROTOBUF_FIELD_OFFSET(DerivationExpression_ExpressionIntervalCompound, precision_)>( reinterpret_cast(&precision_), reinterpret_cast(&other->precision_)); } -::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionPrecisionTimestamp::GetMetadata() const { +::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionIntervalCompound::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[4]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[6]); } // =================================================================== @@ -2273,7 +2849,7 @@ void DerivationExpression_ExpressionPrecisionTimestampTZ::InternalSwap(Derivatio ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionPrecisionTimestampTZ::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[5]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[7]); } // =================================================================== @@ -2521,7 +3097,7 @@ void DerivationExpression_ExpressionStruct::InternalSwap(DerivationExpression_Ex ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionStruct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[6]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[8]); } // =================================================================== @@ -2754,7 +3330,7 @@ void DerivationExpression_ExpressionNamedStruct::InternalSwap(DerivationExpressi ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionNamedStruct::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[7]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[9]); } // =================================================================== @@ -3010,7 +3586,7 @@ void DerivationExpression_ExpressionList::InternalSwap(DerivationExpression_Expr ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionList::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[8]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[10]); } // =================================================================== @@ -3307,7 +3883,7 @@ void DerivationExpression_ExpressionMap::InternalSwap(DerivationExpression_Expre ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionMap::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[9]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[11]); } // =================================================================== @@ -3544,7 +4120,7 @@ void DerivationExpression_ExpressionUserDefined::InternalSwap(DerivationExpressi ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ExpressionUserDefined::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[10]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[12]); } // =================================================================== @@ -3829,7 +4405,7 @@ void DerivationExpression_IfElse::InternalSwap(DerivationExpression_IfElse* othe ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_IfElse::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[11]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[13]); } // =================================================================== @@ -4059,7 +4635,7 @@ void DerivationExpression_UnaryOp::InternalSwap(DerivationExpression_UnaryOp* ot ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_UnaryOp::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[12]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[14]); } // =================================================================== @@ -4330,7 +4906,7 @@ void DerivationExpression_BinaryOp::InternalSwap(DerivationExpression_BinaryOp* ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_BinaryOp::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[13]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[15]); } // =================================================================== @@ -4576,7 +5152,7 @@ void DerivationExpression_ReturnProgram_Assignment::InternalSwap(DerivationExpre ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ReturnProgram_Assignment::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[14]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[16]); } // =================================================================== @@ -4804,7 +5380,7 @@ void DerivationExpression_ReturnProgram::InternalSwap(DerivationExpression_Retur ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression_ReturnProgram::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[15]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[17]); } // =================================================================== @@ -4824,9 +5400,10 @@ class DerivationExpression::_Internal { static const ::substrait::Type_Date& date(const DerivationExpression* msg); static const ::substrait::Type_Time& time(const DerivationExpression* msg); static const ::substrait::Type_IntervalYear& interval_year(const DerivationExpression* msg); - static const ::substrait::Type_IntervalDay& interval_day(const DerivationExpression* msg); static const ::substrait::Type_TimestampTZ& timestamp_tz(const DerivationExpression* msg); static const ::substrait::Type_UUID& uuid(const DerivationExpression* msg); + static const ::substrait::DerivationExpression_ExpressionIntervalDay& interval_day(const DerivationExpression* msg); + static const ::substrait::DerivationExpression_ExpressionIntervalCompound& interval_compound(const DerivationExpression* msg); static const ::substrait::DerivationExpression_ExpressionFixedChar& fixed_char(const DerivationExpression* msg); static const ::substrait::DerivationExpression_ExpressionVarChar& varchar(const DerivationExpression* msg); static const ::substrait::DerivationExpression_ExpressionFixedBinary& fixed_binary(const DerivationExpression* msg); @@ -4895,10 +5472,6 @@ const ::substrait::Type_IntervalYear& DerivationExpression::_Internal::interval_year(const DerivationExpression* msg) { return *msg->kind_.interval_year_; } -const ::substrait::Type_IntervalDay& -DerivationExpression::_Internal::interval_day(const DerivationExpression* msg) { - return *msg->kind_.interval_day_; -} const ::substrait::Type_TimestampTZ& DerivationExpression::_Internal::timestamp_tz(const DerivationExpression* msg) { return *msg->kind_.timestamp_tz_; @@ -4907,6 +5480,14 @@ const ::substrait::Type_UUID& DerivationExpression::_Internal::uuid(const DerivationExpression* msg) { return *msg->kind_.uuid_; } +const ::substrait::DerivationExpression_ExpressionIntervalDay& +DerivationExpression::_Internal::interval_day(const DerivationExpression* msg) { + return *msg->kind_.interval_day_; +} +const ::substrait::DerivationExpression_ExpressionIntervalCompound& +DerivationExpression::_Internal::interval_compound(const DerivationExpression* msg) { + return *msg->kind_.interval_compound_; +} const ::substrait::DerivationExpression_ExpressionFixedChar& DerivationExpression::_Internal::fixed_char(const DerivationExpression* msg) { return *msg->kind_.fixed_char_; @@ -5288,31 +5869,6 @@ void DerivationExpression::clear_interval_year() { clear_has_kind(); } } -void DerivationExpression::set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - clear_kind(); - if (interval_day) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper< - ::PROTOBUF_NAMESPACE_ID::MessageLite>::GetOwningArena( - reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(interval_day)); - if (message_arena != submessage_arena) { - interval_day = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, interval_day, submessage_arena); - } - set_has_interval_day(); - kind_.interval_day_ = interval_day; - } - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.interval_day) -} -void DerivationExpression::clear_interval_day() { - if (_internal_has_interval_day()) { - if (GetArenaForAllocation() == nullptr) { - delete kind_.interval_day_; - } - clear_has_kind(); - } -} void DerivationExpression::set_allocated_timestamp_tz(::substrait::Type_TimestampTZ* timestamp_tz) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_kind(); @@ -5363,6 +5919,36 @@ void DerivationExpression::clear_uuid() { clear_has_kind(); } } +void DerivationExpression::set_allocated_interval_day(::substrait::DerivationExpression_ExpressionIntervalDay* interval_day) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_kind(); + if (interval_day) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression_ExpressionIntervalDay>::GetOwningArena(interval_day); + if (message_arena != submessage_arena) { + interval_day = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_day, submessage_arena); + } + set_has_interval_day(); + kind_.interval_day_ = interval_day; + } + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.interval_day) +} +void DerivationExpression::set_allocated_interval_compound(::substrait::DerivationExpression_ExpressionIntervalCompound* interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + clear_kind(); + if (interval_compound) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression_ExpressionIntervalCompound>::GetOwningArena(interval_compound); + if (message_arena != submessage_arena) { + interval_compound = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, interval_compound, submessage_arena); + } + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.interval_compound) +} void DerivationExpression::set_allocated_fixed_char(::substrait::DerivationExpression_ExpressionFixedChar* fixed_char) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); clear_kind(); @@ -5639,10 +6225,6 @@ DerivationExpression::DerivationExpression(const DerivationExpression& from) _internal_mutable_interval_year()->::substrait::Type_IntervalYear::MergeFrom(from._internal_interval_year()); break; } - case kIntervalDay: { - _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); - break; - } case kTimestampTz: { _internal_mutable_timestamp_tz()->::substrait::Type_TimestampTZ::MergeFrom(from._internal_timestamp_tz()); break; @@ -5651,6 +6233,14 @@ DerivationExpression::DerivationExpression(const DerivationExpression& from) _internal_mutable_uuid()->::substrait::Type_UUID::MergeFrom(from._internal_uuid()); break; } + case kIntervalDay: { + _internal_mutable_interval_day()->::substrait::DerivationExpression_ExpressionIntervalDay::MergeFrom(from._internal_interval_day()); + break; + } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::DerivationExpression_ExpressionIntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kFixedChar: { _internal_mutable_fixed_char()->::substrait::DerivationExpression_ExpressionFixedChar::MergeFrom(from._internal_fixed_char()); break; @@ -5839,12 +6429,6 @@ void DerivationExpression::clear_kind() { } break; } - case kIntervalDay: { - if (GetArenaForAllocation() == nullptr) { - delete kind_.interval_day_; - } - break; - } case kTimestampTz: { if (GetArenaForAllocation() == nullptr) { delete kind_.timestamp_tz_; @@ -5857,6 +6441,18 @@ void DerivationExpression::clear_kind() { } break; } + case kIntervalDay: { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_day_; + } + break; + } + case kIntervalCompound: { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + break; + } case kFixedChar: { if (GetArenaForAllocation() == nullptr) { delete kind_.fixed_char_; @@ -6085,7 +6681,7 @@ const char* DerivationExpression::_InternalParse(const char* ptr, ::PROTOBUF_NAM } else goto handle_unusual; continue; - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.DerivationExpression.ExpressionIntervalDay interval_day = 20; case 20: if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 162)) { ptr = ctx->ParseMessage(_internal_mutable_interval_day(), ptr); @@ -6257,6 +6853,14 @@ const char* DerivationExpression::_InternalParse(const char* ptr, ::PROTOBUF_NAM } else goto handle_unusual; continue; + // .substrait.DerivationExpression.ExpressionIntervalCompound interval_compound = 42; + case 42: + if (PROTOBUF_PREDICT_TRUE(static_cast(tag) == 82)) { + ptr = ctx->ParseMessage(_internal_mutable_interval_compound(), ptr); + CHK_(ptr); + } else + goto handle_unusual; + continue; default: goto handle_unusual; } // switch @@ -6390,7 +6994,7 @@ uint8_t* DerivationExpression::_InternalSerialize( 19, _Internal::interval_year(this), target, stream); } - // .substrait.Type.IntervalDay interval_day = 20; + // .substrait.DerivationExpression.ExpressionIntervalDay interval_day = 20; if (_internal_has_interval_day()) { target = stream->EnsureSpace(target); target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: @@ -6558,6 +7162,14 @@ uint8_t* DerivationExpression::_InternalSerialize( 41, _Internal::precision_timestamp_tz(this), target, stream); } + // .substrait.DerivationExpression.ExpressionIntervalCompound interval_compound = 42; + if (_internal_has_interval_compound()) { + target = stream->EnsureSpace(target); + target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite:: + InternalWriteMessage( + 42, _Internal::interval_compound(this), target, stream); + } + if (PROTOBUF_PREDICT_FALSE(_internal_metadata_.have_unknown_fields())) { target = ::PROTOBUF_NAMESPACE_ID::internal::WireFormat::InternalSerializeUnknownFieldsToArray( _internal_metadata_.unknown_fields<::PROTOBUF_NAMESPACE_ID::UnknownFieldSet>(::PROTOBUF_NAMESPACE_ID::UnknownFieldSet::default_instance), target, stream); @@ -6666,13 +7278,6 @@ size_t DerivationExpression::ByteSizeLong() const { *kind_.interval_year_); break; } - // .substrait.Type.IntervalDay interval_day = 20; - case kIntervalDay: { - total_size += 2 + - ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( - *kind_.interval_day_); - break; - } // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; case kTimestampTz: { total_size += 2 + @@ -6687,6 +7292,20 @@ size_t DerivationExpression::ByteSizeLong() const { *kind_.uuid_); break; } + // .substrait.DerivationExpression.ExpressionIntervalDay interval_day = 20; + case kIntervalDay: { + total_size += 2 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *kind_.interval_day_); + break; + } + // .substrait.DerivationExpression.ExpressionIntervalCompound interval_compound = 42; + case kIntervalCompound: { + total_size += 2 + + ::PROTOBUF_NAMESPACE_ID::internal::WireFormatLite::MessageSize( + *kind_.interval_compound_); + break; + } // .substrait.DerivationExpression.ExpressionFixedChar fixed_char = 21; case kFixedChar: { total_size += 2 + @@ -6892,10 +7511,6 @@ void DerivationExpression::MergeFrom(const DerivationExpression& from) { _internal_mutable_interval_year()->::substrait::Type_IntervalYear::MergeFrom(from._internal_interval_year()); break; } - case kIntervalDay: { - _internal_mutable_interval_day()->::substrait::Type_IntervalDay::MergeFrom(from._internal_interval_day()); - break; - } case kTimestampTz: { _internal_mutable_timestamp_tz()->::substrait::Type_TimestampTZ::MergeFrom(from._internal_timestamp_tz()); break; @@ -6904,6 +7519,14 @@ void DerivationExpression::MergeFrom(const DerivationExpression& from) { _internal_mutable_uuid()->::substrait::Type_UUID::MergeFrom(from._internal_uuid()); break; } + case kIntervalDay: { + _internal_mutable_interval_day()->::substrait::DerivationExpression_ExpressionIntervalDay::MergeFrom(from._internal_interval_day()); + break; + } + case kIntervalCompound: { + _internal_mutable_interval_compound()->::substrait::DerivationExpression_ExpressionIntervalCompound::MergeFrom(from._internal_interval_compound()); + break; + } case kFixedChar: { _internal_mutable_fixed_char()->::substrait::DerivationExpression_ExpressionFixedChar::MergeFrom(from._internal_fixed_char()); break; @@ -7004,7 +7627,7 @@ void DerivationExpression::InternalSwap(DerivationExpression* other) { ::PROTOBUF_NAMESPACE_ID::Metadata DerivationExpression::GetMetadata() const { return ::PROTOBUF_NAMESPACE_ID::internal::AssignDescriptors( &descriptor_table_substrait_2ftype_5fexpressions_2eproto_getter, &descriptor_table_substrait_2ftype_5fexpressions_2eproto_once, - file_level_metadata_substrait_2ftype_5fexpressions_2eproto[16]); + file_level_metadata_substrait_2ftype_5fexpressions_2eproto[18]); } // @@protoc_insertion_point(namespace_scope) @@ -7025,6 +7648,12 @@ template<> PROTOBUF_NOINLINE ::substrait::DerivationExpression_ExpressionDecimal template<> PROTOBUF_NOINLINE ::substrait::DerivationExpression_ExpressionPrecisionTimestamp* Arena::CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionPrecisionTimestamp >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::DerivationExpression_ExpressionPrecisionTimestamp >(arena); } +template<> PROTOBUF_NOINLINE ::substrait::DerivationExpression_ExpressionIntervalDay* Arena::CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionIntervalDay >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::DerivationExpression_ExpressionIntervalDay >(arena); +} +template<> PROTOBUF_NOINLINE ::substrait::DerivationExpression_ExpressionIntervalCompound* Arena::CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionIntervalCompound >(Arena* arena) { + return Arena::CreateMessageInternal< ::substrait::DerivationExpression_ExpressionIntervalCompound >(arena); +} template<> PROTOBUF_NOINLINE ::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ* Arena::CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ >(Arena* arena) { return Arena::CreateMessageInternal< ::substrait::DerivationExpression_ExpressionPrecisionTimestampTZ >(arena); } diff --git a/third_party/substrait/substrait/type_expressions.pb.h b/third_party/substrait/substrait/type_expressions.pb.h index c2f6f27..f1c92bc 100644 --- a/third_party/substrait/substrait/type_expressions.pb.h +++ b/third_party/substrait/substrait/type_expressions.pb.h @@ -48,7 +48,7 @@ struct TableStruct_substrait_2ftype_5fexpressions_2eproto { PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::AuxiliaryParseTableField aux[] PROTOBUF_SECTION_VARIABLE(protodesc_cold); - static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[17] + static const ::PROTOBUF_NAMESPACE_ID::internal::ParseTable schema[19] PROTOBUF_SECTION_VARIABLE(protodesc_cold); static const ::PROTOBUF_NAMESPACE_ID::internal::FieldMetadata field_metadata[]; static const ::PROTOBUF_NAMESPACE_ID::internal::SerializationTable serialization_table[]; @@ -71,6 +71,12 @@ extern DerivationExpression_ExpressionFixedBinaryDefaultTypeInternal _Derivation class DerivationExpression_ExpressionFixedChar; struct DerivationExpression_ExpressionFixedCharDefaultTypeInternal; extern DerivationExpression_ExpressionFixedCharDefaultTypeInternal _DerivationExpression_ExpressionFixedChar_default_instance_; +class DerivationExpression_ExpressionIntervalCompound; +struct DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal; +extern DerivationExpression_ExpressionIntervalCompoundDefaultTypeInternal _DerivationExpression_ExpressionIntervalCompound_default_instance_; +class DerivationExpression_ExpressionIntervalDay; +struct DerivationExpression_ExpressionIntervalDayDefaultTypeInternal; +extern DerivationExpression_ExpressionIntervalDayDefaultTypeInternal _DerivationExpression_ExpressionIntervalDay_default_instance_; class DerivationExpression_ExpressionList; struct DerivationExpression_ExpressionListDefaultTypeInternal; extern DerivationExpression_ExpressionListDefaultTypeInternal _DerivationExpression_ExpressionList_default_instance_; @@ -114,6 +120,8 @@ template<> ::substrait::DerivationExpression_BinaryOp* Arena::CreateMaybeMessage template<> ::substrait::DerivationExpression_ExpressionDecimal* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionDecimal>(Arena*); template<> ::substrait::DerivationExpression_ExpressionFixedBinary* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionFixedBinary>(Arena*); template<> ::substrait::DerivationExpression_ExpressionFixedChar* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionFixedChar>(Arena*); +template<> ::substrait::DerivationExpression_ExpressionIntervalCompound* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionIntervalCompound>(Arena*); +template<> ::substrait::DerivationExpression_ExpressionIntervalDay* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionIntervalDay>(Arena*); template<> ::substrait::DerivationExpression_ExpressionList* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionList>(Arena*); template<> ::substrait::DerivationExpression_ExpressionMap* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionMap>(Arena*); template<> ::substrait::DerivationExpression_ExpressionNamedStruct* Arena::CreateMaybeMessage<::substrait::DerivationExpression_ExpressionNamedStruct>(Arena*); @@ -1097,24 +1105,24 @@ class DerivationExpression_ExpressionPrecisionTimestamp final : }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionPrecisionTimestampTZ final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionPrecisionTimestampTZ) */ { +class DerivationExpression_ExpressionIntervalDay final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionIntervalDay) */ { public: - inline DerivationExpression_ExpressionPrecisionTimestampTZ() : DerivationExpression_ExpressionPrecisionTimestampTZ(nullptr) {} - ~DerivationExpression_ExpressionPrecisionTimestampTZ() override; - explicit constexpr DerivationExpression_ExpressionPrecisionTimestampTZ(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionIntervalDay() : DerivationExpression_ExpressionIntervalDay(nullptr) {} + ~DerivationExpression_ExpressionIntervalDay() override; + explicit constexpr DerivationExpression_ExpressionIntervalDay(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionPrecisionTimestampTZ(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); - DerivationExpression_ExpressionPrecisionTimestampTZ(DerivationExpression_ExpressionPrecisionTimestampTZ&& from) noexcept - : DerivationExpression_ExpressionPrecisionTimestampTZ() { + DerivationExpression_ExpressionIntervalDay(const DerivationExpression_ExpressionIntervalDay& from); + DerivationExpression_ExpressionIntervalDay(DerivationExpression_ExpressionIntervalDay&& from) noexcept + : DerivationExpression_ExpressionIntervalDay() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionPrecisionTimestampTZ& operator=(const DerivationExpression_ExpressionPrecisionTimestampTZ& from) { + inline DerivationExpression_ExpressionIntervalDay& operator=(const DerivationExpression_ExpressionIntervalDay& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionPrecisionTimestampTZ& operator=(DerivationExpression_ExpressionPrecisionTimestampTZ&& from) noexcept { + inline DerivationExpression_ExpressionIntervalDay& operator=(DerivationExpression_ExpressionIntervalDay&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -1137,20 +1145,20 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionPrecisionTimestampTZ& default_instance() { + static const DerivationExpression_ExpressionIntervalDay& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionPrecisionTimestampTZ* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionPrecisionTimestampTZ_default_instance_); + static inline const DerivationExpression_ExpressionIntervalDay* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionIntervalDay_default_instance_); } static constexpr int kIndexInFileMessages = 5; - friend void swap(DerivationExpression_ExpressionPrecisionTimestampTZ& a, DerivationExpression_ExpressionPrecisionTimestampTZ& b) { + friend void swap(DerivationExpression_ExpressionIntervalDay& a, DerivationExpression_ExpressionIntervalDay& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionPrecisionTimestampTZ* other) { + inline void Swap(DerivationExpression_ExpressionIntervalDay* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -1163,7 +1171,7 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionPrecisionTimestampTZ* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionIntervalDay* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -1171,13 +1179,13 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionPrecisionTimestampTZ* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionIntervalDay* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); + void CopyFrom(const DerivationExpression_ExpressionIntervalDay& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); + void MergeFrom(const DerivationExpression_ExpressionIntervalDay& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -1194,15 +1202,15 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionPrecisionTimestampTZ* other); + void InternalSwap(DerivationExpression_ExpressionIntervalDay* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionPrecisionTimestampTZ"; + return "substrait.DerivationExpression.ExpressionIntervalDay"; } protected: - explicit DerivationExpression_ExpressionPrecisionTimestampTZ(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionIntervalDay(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -1259,7 +1267,7 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : void _internal_set_nullability(::substrait::Type_Nullability value); public: - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionPrecisionTimestampTZ) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionIntervalDay) private: class _Internal; @@ -1274,24 +1282,24 @@ class DerivationExpression_ExpressionPrecisionTimestampTZ final : }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionStruct final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionStruct) */ { +class DerivationExpression_ExpressionIntervalCompound final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionIntervalCompound) */ { public: - inline DerivationExpression_ExpressionStruct() : DerivationExpression_ExpressionStruct(nullptr) {} - ~DerivationExpression_ExpressionStruct() override; - explicit constexpr DerivationExpression_ExpressionStruct(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionIntervalCompound() : DerivationExpression_ExpressionIntervalCompound(nullptr) {} + ~DerivationExpression_ExpressionIntervalCompound() override; + explicit constexpr DerivationExpression_ExpressionIntervalCompound(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionStruct(const DerivationExpression_ExpressionStruct& from); - DerivationExpression_ExpressionStruct(DerivationExpression_ExpressionStruct&& from) noexcept - : DerivationExpression_ExpressionStruct() { + DerivationExpression_ExpressionIntervalCompound(const DerivationExpression_ExpressionIntervalCompound& from); + DerivationExpression_ExpressionIntervalCompound(DerivationExpression_ExpressionIntervalCompound&& from) noexcept + : DerivationExpression_ExpressionIntervalCompound() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionStruct& operator=(const DerivationExpression_ExpressionStruct& from) { + inline DerivationExpression_ExpressionIntervalCompound& operator=(const DerivationExpression_ExpressionIntervalCompound& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionStruct& operator=(DerivationExpression_ExpressionStruct&& from) noexcept { + inline DerivationExpression_ExpressionIntervalCompound& operator=(DerivationExpression_ExpressionIntervalCompound&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -1314,20 +1322,20 @@ class DerivationExpression_ExpressionStruct final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionStruct& default_instance() { + static const DerivationExpression_ExpressionIntervalCompound& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionStruct* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionStruct_default_instance_); + static inline const DerivationExpression_ExpressionIntervalCompound* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionIntervalCompound_default_instance_); } static constexpr int kIndexInFileMessages = 6; - friend void swap(DerivationExpression_ExpressionStruct& a, DerivationExpression_ExpressionStruct& b) { + friend void swap(DerivationExpression_ExpressionIntervalCompound& a, DerivationExpression_ExpressionIntervalCompound& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionStruct* other) { + inline void Swap(DerivationExpression_ExpressionIntervalCompound* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -1340,7 +1348,7 @@ class DerivationExpression_ExpressionStruct final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionStruct* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionIntervalCompound* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -1348,13 +1356,13 @@ class DerivationExpression_ExpressionStruct final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionStruct* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionIntervalCompound* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionStruct& from); + void CopyFrom(const DerivationExpression_ExpressionIntervalCompound& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionStruct& from); + void MergeFrom(const DerivationExpression_ExpressionIntervalCompound& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -1371,15 +1379,15 @@ class DerivationExpression_ExpressionStruct final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionStruct* other); + void InternalSwap(DerivationExpression_ExpressionIntervalCompound* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionStruct"; + return "substrait.DerivationExpression.ExpressionIntervalCompound"; } protected: - explicit DerivationExpression_ExpressionStruct(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionIntervalCompound(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -1396,27 +1404,27 @@ class DerivationExpression_ExpressionStruct final : // accessors ------------------------------------------------------- enum : int { - kTypesFieldNumber = 1, + kPrecisionFieldNumber = 1, kVariationPointerFieldNumber = 2, kNullabilityFieldNumber = 3, }; - // repeated .substrait.DerivationExpression types = 1; - int types_size() const; + // .substrait.DerivationExpression precision = 1; + bool has_precision() const; private: - int _internal_types_size() const; + bool _internal_has_precision() const; public: - void clear_types(); - ::substrait::DerivationExpression* mutable_types(int index); - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression >* - mutable_types(); + void clear_precision(); + const ::substrait::DerivationExpression& precision() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_precision(); + ::substrait::DerivationExpression* mutable_precision(); + void set_allocated_precision(::substrait::DerivationExpression* precision); private: - const ::substrait::DerivationExpression& _internal_types(int index) const; - ::substrait::DerivationExpression* _internal_add_types(); + const ::substrait::DerivationExpression& _internal_precision() const; + ::substrait::DerivationExpression* _internal_mutable_precision(); public: - const ::substrait::DerivationExpression& types(int index) const; - ::substrait::DerivationExpression* add_types(); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression >& - types() const; + void unsafe_arena_set_allocated_precision( + ::substrait::DerivationExpression* precision); + ::substrait::DerivationExpression* unsafe_arena_release_precision(); // uint32 variation_pointer = 2; void clear_variation_pointer(); @@ -1436,14 +1444,14 @@ class DerivationExpression_ExpressionStruct final : void _internal_set_nullability(::substrait::Type_Nullability value); public: - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionStruct) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionIntervalCompound) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression > types_; + ::substrait::DerivationExpression* precision_; uint32_t variation_pointer_; int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; @@ -1451,24 +1459,24 @@ class DerivationExpression_ExpressionStruct final : }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionNamedStruct final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionNamedStruct) */ { +class DerivationExpression_ExpressionPrecisionTimestampTZ final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionPrecisionTimestampTZ) */ { public: - inline DerivationExpression_ExpressionNamedStruct() : DerivationExpression_ExpressionNamedStruct(nullptr) {} - ~DerivationExpression_ExpressionNamedStruct() override; - explicit constexpr DerivationExpression_ExpressionNamedStruct(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionPrecisionTimestampTZ() : DerivationExpression_ExpressionPrecisionTimestampTZ(nullptr) {} + ~DerivationExpression_ExpressionPrecisionTimestampTZ() override; + explicit constexpr DerivationExpression_ExpressionPrecisionTimestampTZ(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionNamedStruct(const DerivationExpression_ExpressionNamedStruct& from); - DerivationExpression_ExpressionNamedStruct(DerivationExpression_ExpressionNamedStruct&& from) noexcept - : DerivationExpression_ExpressionNamedStruct() { + DerivationExpression_ExpressionPrecisionTimestampTZ(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); + DerivationExpression_ExpressionPrecisionTimestampTZ(DerivationExpression_ExpressionPrecisionTimestampTZ&& from) noexcept + : DerivationExpression_ExpressionPrecisionTimestampTZ() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionNamedStruct& operator=(const DerivationExpression_ExpressionNamedStruct& from) { + inline DerivationExpression_ExpressionPrecisionTimestampTZ& operator=(const DerivationExpression_ExpressionPrecisionTimestampTZ& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionNamedStruct& operator=(DerivationExpression_ExpressionNamedStruct&& from) noexcept { + inline DerivationExpression_ExpressionPrecisionTimestampTZ& operator=(DerivationExpression_ExpressionPrecisionTimestampTZ&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -1491,20 +1499,20 @@ class DerivationExpression_ExpressionNamedStruct final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionNamedStruct& default_instance() { + static const DerivationExpression_ExpressionPrecisionTimestampTZ& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionNamedStruct* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionNamedStruct_default_instance_); + static inline const DerivationExpression_ExpressionPrecisionTimestampTZ* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionPrecisionTimestampTZ_default_instance_); } static constexpr int kIndexInFileMessages = 7; - friend void swap(DerivationExpression_ExpressionNamedStruct& a, DerivationExpression_ExpressionNamedStruct& b) { + friend void swap(DerivationExpression_ExpressionPrecisionTimestampTZ& a, DerivationExpression_ExpressionPrecisionTimestampTZ& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionNamedStruct* other) { + inline void Swap(DerivationExpression_ExpressionPrecisionTimestampTZ* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -1517,7 +1525,7 @@ class DerivationExpression_ExpressionNamedStruct final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionNamedStruct* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionPrecisionTimestampTZ* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -1525,13 +1533,13 @@ class DerivationExpression_ExpressionNamedStruct final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionNamedStruct* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionPrecisionTimestampTZ* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionNamedStruct& from); + void CopyFrom(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionNamedStruct& from); + void MergeFrom(const DerivationExpression_ExpressionPrecisionTimestampTZ& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -1548,15 +1556,15 @@ class DerivationExpression_ExpressionNamedStruct final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionNamedStruct* other); + void InternalSwap(DerivationExpression_ExpressionPrecisionTimestampTZ* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionNamedStruct"; + return "substrait.DerivationExpression.ExpressionPrecisionTimestampTZ"; } protected: - explicit DerivationExpression_ExpressionNamedStruct(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionPrecisionTimestampTZ(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -1573,83 +1581,79 @@ class DerivationExpression_ExpressionNamedStruct final : // accessors ------------------------------------------------------- enum : int { - kNamesFieldNumber = 1, - kStructFieldNumber = 2, + kPrecisionFieldNumber = 1, + kVariationPointerFieldNumber = 2, + kNullabilityFieldNumber = 3, }; - // repeated string names = 1; - int names_size() const; + // .substrait.DerivationExpression precision = 1; + bool has_precision() const; private: - int _internal_names_size() const; + bool _internal_has_precision() const; public: - void clear_names(); - const std::string& names(int index) const; - std::string* mutable_names(int index); - void set_names(int index, const std::string& value); - void set_names(int index, std::string&& value); - void set_names(int index, const char* value); - void set_names(int index, const char* value, size_t size); - std::string* add_names(); - void add_names(const std::string& value); - void add_names(std::string&& value); - void add_names(const char* value); - void add_names(const char* value, size_t size); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& names() const; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* mutable_names(); + void clear_precision(); + const ::substrait::DerivationExpression& precision() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_precision(); + ::substrait::DerivationExpression* mutable_precision(); + void set_allocated_precision(::substrait::DerivationExpression* precision); private: - const std::string& _internal_names(int index) const; - std::string* _internal_add_names(); + const ::substrait::DerivationExpression& _internal_precision() const; + ::substrait::DerivationExpression* _internal_mutable_precision(); public: + void unsafe_arena_set_allocated_precision( + ::substrait::DerivationExpression* precision); + ::substrait::DerivationExpression* unsafe_arena_release_precision(); - // .substrait.DerivationExpression.ExpressionStruct struct = 2; - bool has_struct_() const; + // uint32 variation_pointer = 2; + void clear_variation_pointer(); + uint32_t variation_pointer() const; + void set_variation_pointer(uint32_t value); private: - bool _internal_has_struct_() const; + uint32_t _internal_variation_pointer() const; + void _internal_set_variation_pointer(uint32_t value); public: - void clear_struct_(); - const ::substrait::DerivationExpression_ExpressionStruct& struct_() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression_ExpressionStruct* release_struct_(); - ::substrait::DerivationExpression_ExpressionStruct* mutable_struct_(); - void set_allocated_struct_(::substrait::DerivationExpression_ExpressionStruct* struct_); + + // .substrait.Type.Nullability nullability = 3; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); private: - const ::substrait::DerivationExpression_ExpressionStruct& _internal_struct_() const; - ::substrait::DerivationExpression_ExpressionStruct* _internal_mutable_struct_(); + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); public: - void unsafe_arena_set_allocated_struct_( - ::substrait::DerivationExpression_ExpressionStruct* struct_); - ::substrait::DerivationExpression_ExpressionStruct* unsafe_arena_release_struct_(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionNamedStruct) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionPrecisionTimestampTZ) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField names_; - ::substrait::DerivationExpression_ExpressionStruct* struct__; + ::substrait::DerivationExpression* precision_; + uint32_t variation_pointer_; + int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionList final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionList) */ { +class DerivationExpression_ExpressionStruct final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionStruct) */ { public: - inline DerivationExpression_ExpressionList() : DerivationExpression_ExpressionList(nullptr) {} - ~DerivationExpression_ExpressionList() override; - explicit constexpr DerivationExpression_ExpressionList(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionStruct() : DerivationExpression_ExpressionStruct(nullptr) {} + ~DerivationExpression_ExpressionStruct() override; + explicit constexpr DerivationExpression_ExpressionStruct(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionList(const DerivationExpression_ExpressionList& from); - DerivationExpression_ExpressionList(DerivationExpression_ExpressionList&& from) noexcept - : DerivationExpression_ExpressionList() { + DerivationExpression_ExpressionStruct(const DerivationExpression_ExpressionStruct& from); + DerivationExpression_ExpressionStruct(DerivationExpression_ExpressionStruct&& from) noexcept + : DerivationExpression_ExpressionStruct() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionList& operator=(const DerivationExpression_ExpressionList& from) { + inline DerivationExpression_ExpressionStruct& operator=(const DerivationExpression_ExpressionStruct& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionList& operator=(DerivationExpression_ExpressionList&& from) noexcept { + inline DerivationExpression_ExpressionStruct& operator=(DerivationExpression_ExpressionStruct&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -1672,20 +1676,20 @@ class DerivationExpression_ExpressionList final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionList& default_instance() { + static const DerivationExpression_ExpressionStruct& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionList* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionList_default_instance_); + static inline const DerivationExpression_ExpressionStruct* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionStruct_default_instance_); } static constexpr int kIndexInFileMessages = 8; - friend void swap(DerivationExpression_ExpressionList& a, DerivationExpression_ExpressionList& b) { + friend void swap(DerivationExpression_ExpressionStruct& a, DerivationExpression_ExpressionStruct& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionList* other) { + inline void Swap(DerivationExpression_ExpressionStruct* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -1698,7 +1702,7 @@ class DerivationExpression_ExpressionList final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionList* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionStruct* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -1706,13 +1710,13 @@ class DerivationExpression_ExpressionList final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionList* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionStruct* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionList& from); + void CopyFrom(const DerivationExpression_ExpressionStruct& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionList& from); + void MergeFrom(const DerivationExpression_ExpressionStruct& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -1729,15 +1733,15 @@ class DerivationExpression_ExpressionList final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionList* other); + void InternalSwap(DerivationExpression_ExpressionStruct* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionList"; + return "substrait.DerivationExpression.ExpressionStruct"; } protected: - explicit DerivationExpression_ExpressionList(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionStruct(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -1754,27 +1758,27 @@ class DerivationExpression_ExpressionList final : // accessors ------------------------------------------------------- enum : int { - kTypeFieldNumber = 1, + kTypesFieldNumber = 1, kVariationPointerFieldNumber = 2, kNullabilityFieldNumber = 3, }; - // .substrait.DerivationExpression type = 1; - bool has_type() const; + // repeated .substrait.DerivationExpression types = 1; + int types_size() const; private: - bool _internal_has_type() const; + int _internal_types_size() const; public: - void clear_type(); - const ::substrait::DerivationExpression& type() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_type(); - ::substrait::DerivationExpression* mutable_type(); - void set_allocated_type(::substrait::DerivationExpression* type); + void clear_types(); + ::substrait::DerivationExpression* mutable_types(int index); + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression >* + mutable_types(); private: - const ::substrait::DerivationExpression& _internal_type() const; - ::substrait::DerivationExpression* _internal_mutable_type(); + const ::substrait::DerivationExpression& _internal_types(int index) const; + ::substrait::DerivationExpression* _internal_add_types(); public: - void unsafe_arena_set_allocated_type( - ::substrait::DerivationExpression* type); - ::substrait::DerivationExpression* unsafe_arena_release_type(); + const ::substrait::DerivationExpression& types(int index) const; + ::substrait::DerivationExpression* add_types(); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression >& + types() const; // uint32 variation_pointer = 2; void clear_variation_pointer(); @@ -1794,14 +1798,14 @@ class DerivationExpression_ExpressionList final : void _internal_set_nullability(::substrait::Type_Nullability value); public: - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionList) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionStruct) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::DerivationExpression* type_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression > types_; uint32_t variation_pointer_; int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; @@ -1809,24 +1813,24 @@ class DerivationExpression_ExpressionList final : }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionMap final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionMap) */ { +class DerivationExpression_ExpressionNamedStruct final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionNamedStruct) */ { public: - inline DerivationExpression_ExpressionMap() : DerivationExpression_ExpressionMap(nullptr) {} - ~DerivationExpression_ExpressionMap() override; - explicit constexpr DerivationExpression_ExpressionMap(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionNamedStruct() : DerivationExpression_ExpressionNamedStruct(nullptr) {} + ~DerivationExpression_ExpressionNamedStruct() override; + explicit constexpr DerivationExpression_ExpressionNamedStruct(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionMap(const DerivationExpression_ExpressionMap& from); - DerivationExpression_ExpressionMap(DerivationExpression_ExpressionMap&& from) noexcept - : DerivationExpression_ExpressionMap() { + DerivationExpression_ExpressionNamedStruct(const DerivationExpression_ExpressionNamedStruct& from); + DerivationExpression_ExpressionNamedStruct(DerivationExpression_ExpressionNamedStruct&& from) noexcept + : DerivationExpression_ExpressionNamedStruct() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionMap& operator=(const DerivationExpression_ExpressionMap& from) { + inline DerivationExpression_ExpressionNamedStruct& operator=(const DerivationExpression_ExpressionNamedStruct& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionMap& operator=(DerivationExpression_ExpressionMap&& from) noexcept { + inline DerivationExpression_ExpressionNamedStruct& operator=(DerivationExpression_ExpressionNamedStruct&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -1849,20 +1853,20 @@ class DerivationExpression_ExpressionMap final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionMap& default_instance() { + static const DerivationExpression_ExpressionNamedStruct& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionMap* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionMap_default_instance_); + static inline const DerivationExpression_ExpressionNamedStruct* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionNamedStruct_default_instance_); } static constexpr int kIndexInFileMessages = 9; - friend void swap(DerivationExpression_ExpressionMap& a, DerivationExpression_ExpressionMap& b) { + friend void swap(DerivationExpression_ExpressionNamedStruct& a, DerivationExpression_ExpressionNamedStruct& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionMap* other) { + inline void Swap(DerivationExpression_ExpressionNamedStruct* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -1875,7 +1879,7 @@ class DerivationExpression_ExpressionMap final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionMap* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionNamedStruct* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -1883,13 +1887,13 @@ class DerivationExpression_ExpressionMap final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionMap* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionNamedStruct* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionMap& from); + void CopyFrom(const DerivationExpression_ExpressionNamedStruct& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionMap& from); + void MergeFrom(const DerivationExpression_ExpressionNamedStruct& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -1906,15 +1910,15 @@ class DerivationExpression_ExpressionMap final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionMap* other); + void InternalSwap(DerivationExpression_ExpressionNamedStruct* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionMap"; + return "substrait.DerivationExpression.ExpressionNamedStruct"; } protected: - explicit DerivationExpression_ExpressionMap(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionNamedStruct(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -1931,99 +1935,83 @@ class DerivationExpression_ExpressionMap final : // accessors ------------------------------------------------------- enum : int { - kKeyFieldNumber = 1, - kValueFieldNumber = 2, - kVariationPointerFieldNumber = 3, - kNullabilityFieldNumber = 4, + kNamesFieldNumber = 1, + kStructFieldNumber = 2, }; - // .substrait.DerivationExpression key = 1; - bool has_key() const; - private: - bool _internal_has_key() const; - public: - void clear_key(); - const ::substrait::DerivationExpression& key() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_key(); - ::substrait::DerivationExpression* mutable_key(); - void set_allocated_key(::substrait::DerivationExpression* key); - private: - const ::substrait::DerivationExpression& _internal_key() const; - ::substrait::DerivationExpression* _internal_mutable_key(); - public: - void unsafe_arena_set_allocated_key( - ::substrait::DerivationExpression* key); - ::substrait::DerivationExpression* unsafe_arena_release_key(); - - // .substrait.DerivationExpression value = 2; - bool has_value() const; + // repeated string names = 1; + int names_size() const; private: - bool _internal_has_value() const; + int _internal_names_size() const; public: - void clear_value(); - const ::substrait::DerivationExpression& value() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_value(); - ::substrait::DerivationExpression* mutable_value(); - void set_allocated_value(::substrait::DerivationExpression* value); + void clear_names(); + const std::string& names(int index) const; + std::string* mutable_names(int index); + void set_names(int index, const std::string& value); + void set_names(int index, std::string&& value); + void set_names(int index, const char* value); + void set_names(int index, const char* value, size_t size); + std::string* add_names(); + void add_names(const std::string& value); + void add_names(std::string&& value); + void add_names(const char* value); + void add_names(const char* value, size_t size); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField& names() const; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField* mutable_names(); private: - const ::substrait::DerivationExpression& _internal_value() const; - ::substrait::DerivationExpression* _internal_mutable_value(); + const std::string& _internal_names(int index) const; + std::string* _internal_add_names(); public: - void unsafe_arena_set_allocated_value( - ::substrait::DerivationExpression* value); - ::substrait::DerivationExpression* unsafe_arena_release_value(); - // uint32 variation_pointer = 3; - void clear_variation_pointer(); - uint32_t variation_pointer() const; - void set_variation_pointer(uint32_t value); + // .substrait.DerivationExpression.ExpressionStruct struct = 2; + bool has_struct_() const; private: - uint32_t _internal_variation_pointer() const; - void _internal_set_variation_pointer(uint32_t value); + bool _internal_has_struct_() const; public: - - // .substrait.Type.Nullability nullability = 4; - void clear_nullability(); - ::substrait::Type_Nullability nullability() const; - void set_nullability(::substrait::Type_Nullability value); + void clear_struct_(); + const ::substrait::DerivationExpression_ExpressionStruct& struct_() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression_ExpressionStruct* release_struct_(); + ::substrait::DerivationExpression_ExpressionStruct* mutable_struct_(); + void set_allocated_struct_(::substrait::DerivationExpression_ExpressionStruct* struct_); private: - ::substrait::Type_Nullability _internal_nullability() const; - void _internal_set_nullability(::substrait::Type_Nullability value); + const ::substrait::DerivationExpression_ExpressionStruct& _internal_struct_() const; + ::substrait::DerivationExpression_ExpressionStruct* _internal_mutable_struct_(); public: + void unsafe_arena_set_allocated_struct_( + ::substrait::DerivationExpression_ExpressionStruct* struct_); + ::substrait::DerivationExpression_ExpressionStruct* unsafe_arena_release_struct_(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionMap) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionNamedStruct) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::DerivationExpression* key_; - ::substrait::DerivationExpression* value_; - uint32_t variation_pointer_; - int nullability_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField names_; + ::substrait::DerivationExpression_ExpressionStruct* struct__; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_ExpressionUserDefined final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionUserDefined) */ { +class DerivationExpression_ExpressionList final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionList) */ { public: - inline DerivationExpression_ExpressionUserDefined() : DerivationExpression_ExpressionUserDefined(nullptr) {} - ~DerivationExpression_ExpressionUserDefined() override; - explicit constexpr DerivationExpression_ExpressionUserDefined(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionList() : DerivationExpression_ExpressionList(nullptr) {} + ~DerivationExpression_ExpressionList() override; + explicit constexpr DerivationExpression_ExpressionList(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ExpressionUserDefined(const DerivationExpression_ExpressionUserDefined& from); - DerivationExpression_ExpressionUserDefined(DerivationExpression_ExpressionUserDefined&& from) noexcept - : DerivationExpression_ExpressionUserDefined() { + DerivationExpression_ExpressionList(const DerivationExpression_ExpressionList& from); + DerivationExpression_ExpressionList(DerivationExpression_ExpressionList&& from) noexcept + : DerivationExpression_ExpressionList() { *this = ::std::move(from); } - inline DerivationExpression_ExpressionUserDefined& operator=(const DerivationExpression_ExpressionUserDefined& from) { + inline DerivationExpression_ExpressionList& operator=(const DerivationExpression_ExpressionList& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ExpressionUserDefined& operator=(DerivationExpression_ExpressionUserDefined&& from) noexcept { + inline DerivationExpression_ExpressionList& operator=(DerivationExpression_ExpressionList&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -2046,20 +2034,20 @@ class DerivationExpression_ExpressionUserDefined final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ExpressionUserDefined& default_instance() { + static const DerivationExpression_ExpressionList& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ExpressionUserDefined* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ExpressionUserDefined_default_instance_); + static inline const DerivationExpression_ExpressionList* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionList_default_instance_); } static constexpr int kIndexInFileMessages = 10; - friend void swap(DerivationExpression_ExpressionUserDefined& a, DerivationExpression_ExpressionUserDefined& b) { + friend void swap(DerivationExpression_ExpressionList& a, DerivationExpression_ExpressionList& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ExpressionUserDefined* other) { + inline void Swap(DerivationExpression_ExpressionList* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -2072,7 +2060,7 @@ class DerivationExpression_ExpressionUserDefined final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ExpressionUserDefined* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionList* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -2080,13 +2068,13 @@ class DerivationExpression_ExpressionUserDefined final : // implements Message ---------------------------------------------- - DerivationExpression_ExpressionUserDefined* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionList* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ExpressionUserDefined& from); + void CopyFrom(const DerivationExpression_ExpressionList& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ExpressionUserDefined& from); + void MergeFrom(const DerivationExpression_ExpressionList& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -2103,15 +2091,15 @@ class DerivationExpression_ExpressionUserDefined final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ExpressionUserDefined* other); + void InternalSwap(DerivationExpression_ExpressionList* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ExpressionUserDefined"; + return "substrait.DerivationExpression.ExpressionList"; } protected: - explicit DerivationExpression_ExpressionUserDefined(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionList(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -2128,18 +2116,27 @@ class DerivationExpression_ExpressionUserDefined final : // accessors ------------------------------------------------------- enum : int { - kTypePointerFieldNumber = 1, + kTypeFieldNumber = 1, kVariationPointerFieldNumber = 2, kNullabilityFieldNumber = 3, }; - // uint32 type_pointer = 1; - void clear_type_pointer(); - uint32_t type_pointer() const; - void set_type_pointer(uint32_t value); + // .substrait.DerivationExpression type = 1; + bool has_type() const; private: - uint32_t _internal_type_pointer() const; - void _internal_set_type_pointer(uint32_t value); + bool _internal_has_type() const; + public: + void clear_type(); + const ::substrait::DerivationExpression& type() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_type(); + ::substrait::DerivationExpression* mutable_type(); + void set_allocated_type(::substrait::DerivationExpression* type); + private: + const ::substrait::DerivationExpression& _internal_type() const; + ::substrait::DerivationExpression* _internal_mutable_type(); public: + void unsafe_arena_set_allocated_type( + ::substrait::DerivationExpression* type); + ::substrait::DerivationExpression* unsafe_arena_release_type(); // uint32 variation_pointer = 2; void clear_variation_pointer(); @@ -2159,14 +2156,14 @@ class DerivationExpression_ExpressionUserDefined final : void _internal_set_nullability(::substrait::Type_Nullability value); public: - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionUserDefined) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionList) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - uint32_t type_pointer_; + ::substrait::DerivationExpression* type_; uint32_t variation_pointer_; int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; @@ -2174,24 +2171,24 @@ class DerivationExpression_ExpressionUserDefined final : }; // ------------------------------------------------------------------- -class DerivationExpression_IfElse final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.IfElse) */ { +class DerivationExpression_ExpressionMap final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionMap) */ { public: - inline DerivationExpression_IfElse() : DerivationExpression_IfElse(nullptr) {} - ~DerivationExpression_IfElse() override; - explicit constexpr DerivationExpression_IfElse(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionMap() : DerivationExpression_ExpressionMap(nullptr) {} + ~DerivationExpression_ExpressionMap() override; + explicit constexpr DerivationExpression_ExpressionMap(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_IfElse(const DerivationExpression_IfElse& from); - DerivationExpression_IfElse(DerivationExpression_IfElse&& from) noexcept - : DerivationExpression_IfElse() { + DerivationExpression_ExpressionMap(const DerivationExpression_ExpressionMap& from); + DerivationExpression_ExpressionMap(DerivationExpression_ExpressionMap&& from) noexcept + : DerivationExpression_ExpressionMap() { *this = ::std::move(from); } - inline DerivationExpression_IfElse& operator=(const DerivationExpression_IfElse& from) { + inline DerivationExpression_ExpressionMap& operator=(const DerivationExpression_ExpressionMap& from) { CopyFrom(from); return *this; } - inline DerivationExpression_IfElse& operator=(DerivationExpression_IfElse&& from) noexcept { + inline DerivationExpression_ExpressionMap& operator=(DerivationExpression_ExpressionMap&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -2214,20 +2211,20 @@ class DerivationExpression_IfElse final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_IfElse& default_instance() { + static const DerivationExpression_ExpressionMap& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_IfElse* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_IfElse_default_instance_); + static inline const DerivationExpression_ExpressionMap* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionMap_default_instance_); } static constexpr int kIndexInFileMessages = 11; - friend void swap(DerivationExpression_IfElse& a, DerivationExpression_IfElse& b) { + friend void swap(DerivationExpression_ExpressionMap& a, DerivationExpression_ExpressionMap& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_IfElse* other) { + inline void Swap(DerivationExpression_ExpressionMap* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -2240,7 +2237,7 @@ class DerivationExpression_IfElse final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_IfElse* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionMap* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -2248,13 +2245,13 @@ class DerivationExpression_IfElse final : // implements Message ---------------------------------------------- - DerivationExpression_IfElse* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionMap* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_IfElse& from); + void CopyFrom(const DerivationExpression_ExpressionMap& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_IfElse& from); + void MergeFrom(const DerivationExpression_ExpressionMap& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -2271,15 +2268,15 @@ class DerivationExpression_IfElse final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_IfElse* other); + void InternalSwap(DerivationExpression_ExpressionMap* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.IfElse"; + return "substrait.DerivationExpression.ExpressionMap"; } protected: - explicit DerivationExpression_IfElse(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionMap(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -2296,97 +2293,99 @@ class DerivationExpression_IfElse final : // accessors ------------------------------------------------------- enum : int { - kIfConditionFieldNumber = 1, - kIfReturnFieldNumber = 2, - kElseReturnFieldNumber = 3, + kKeyFieldNumber = 1, + kValueFieldNumber = 2, + kVariationPointerFieldNumber = 3, + kNullabilityFieldNumber = 4, }; - // .substrait.DerivationExpression if_condition = 1; - bool has_if_condition() const; + // .substrait.DerivationExpression key = 1; + bool has_key() const; private: - bool _internal_has_if_condition() const; + bool _internal_has_key() const; public: - void clear_if_condition(); - const ::substrait::DerivationExpression& if_condition() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_if_condition(); - ::substrait::DerivationExpression* mutable_if_condition(); - void set_allocated_if_condition(::substrait::DerivationExpression* if_condition); + void clear_key(); + const ::substrait::DerivationExpression& key() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_key(); + ::substrait::DerivationExpression* mutable_key(); + void set_allocated_key(::substrait::DerivationExpression* key); private: - const ::substrait::DerivationExpression& _internal_if_condition() const; - ::substrait::DerivationExpression* _internal_mutable_if_condition(); + const ::substrait::DerivationExpression& _internal_key() const; + ::substrait::DerivationExpression* _internal_mutable_key(); public: - void unsafe_arena_set_allocated_if_condition( - ::substrait::DerivationExpression* if_condition); - ::substrait::DerivationExpression* unsafe_arena_release_if_condition(); + void unsafe_arena_set_allocated_key( + ::substrait::DerivationExpression* key); + ::substrait::DerivationExpression* unsafe_arena_release_key(); - // .substrait.DerivationExpression if_return = 2; - bool has_if_return() const; + // .substrait.DerivationExpression value = 2; + bool has_value() const; private: - bool _internal_has_if_return() const; + bool _internal_has_value() const; public: - void clear_if_return(); - const ::substrait::DerivationExpression& if_return() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_if_return(); - ::substrait::DerivationExpression* mutable_if_return(); - void set_allocated_if_return(::substrait::DerivationExpression* if_return); + void clear_value(); + const ::substrait::DerivationExpression& value() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_value(); + ::substrait::DerivationExpression* mutable_value(); + void set_allocated_value(::substrait::DerivationExpression* value); private: - const ::substrait::DerivationExpression& _internal_if_return() const; - ::substrait::DerivationExpression* _internal_mutable_if_return(); + const ::substrait::DerivationExpression& _internal_value() const; + ::substrait::DerivationExpression* _internal_mutable_value(); public: - void unsafe_arena_set_allocated_if_return( - ::substrait::DerivationExpression* if_return); - ::substrait::DerivationExpression* unsafe_arena_release_if_return(); + void unsafe_arena_set_allocated_value( + ::substrait::DerivationExpression* value); + ::substrait::DerivationExpression* unsafe_arena_release_value(); - // .substrait.DerivationExpression else_return = 3; - bool has_else_return() const; + // uint32 variation_pointer = 3; + void clear_variation_pointer(); + uint32_t variation_pointer() const; + void set_variation_pointer(uint32_t value); private: - bool _internal_has_else_return() const; + uint32_t _internal_variation_pointer() const; + void _internal_set_variation_pointer(uint32_t value); public: - void clear_else_return(); - const ::substrait::DerivationExpression& else_return() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_else_return(); - ::substrait::DerivationExpression* mutable_else_return(); - void set_allocated_else_return(::substrait::DerivationExpression* else_return); + + // .substrait.Type.Nullability nullability = 4; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); private: - const ::substrait::DerivationExpression& _internal_else_return() const; - ::substrait::DerivationExpression* _internal_mutable_else_return(); + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); public: - void unsafe_arena_set_allocated_else_return( - ::substrait::DerivationExpression* else_return); - ::substrait::DerivationExpression* unsafe_arena_release_else_return(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.IfElse) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionMap) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::DerivationExpression* if_condition_; - ::substrait::DerivationExpression* if_return_; - ::substrait::DerivationExpression* else_return_; + ::substrait::DerivationExpression* key_; + ::substrait::DerivationExpression* value_; + uint32_t variation_pointer_; + int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_UnaryOp final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.UnaryOp) */ { +class DerivationExpression_ExpressionUserDefined final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ExpressionUserDefined) */ { public: - inline DerivationExpression_UnaryOp() : DerivationExpression_UnaryOp(nullptr) {} - ~DerivationExpression_UnaryOp() override; - explicit constexpr DerivationExpression_UnaryOp(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ExpressionUserDefined() : DerivationExpression_ExpressionUserDefined(nullptr) {} + ~DerivationExpression_ExpressionUserDefined() override; + explicit constexpr DerivationExpression_ExpressionUserDefined(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_UnaryOp(const DerivationExpression_UnaryOp& from); - DerivationExpression_UnaryOp(DerivationExpression_UnaryOp&& from) noexcept - : DerivationExpression_UnaryOp() { + DerivationExpression_ExpressionUserDefined(const DerivationExpression_ExpressionUserDefined& from); + DerivationExpression_ExpressionUserDefined(DerivationExpression_ExpressionUserDefined&& from) noexcept + : DerivationExpression_ExpressionUserDefined() { *this = ::std::move(from); } - inline DerivationExpression_UnaryOp& operator=(const DerivationExpression_UnaryOp& from) { + inline DerivationExpression_ExpressionUserDefined& operator=(const DerivationExpression_ExpressionUserDefined& from) { CopyFrom(from); return *this; } - inline DerivationExpression_UnaryOp& operator=(DerivationExpression_UnaryOp&& from) noexcept { + inline DerivationExpression_ExpressionUserDefined& operator=(DerivationExpression_ExpressionUserDefined&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -2409,20 +2408,20 @@ class DerivationExpression_UnaryOp final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_UnaryOp& default_instance() { + static const DerivationExpression_ExpressionUserDefined& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_UnaryOp* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_UnaryOp_default_instance_); + static inline const DerivationExpression_ExpressionUserDefined* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ExpressionUserDefined_default_instance_); } static constexpr int kIndexInFileMessages = 12; - friend void swap(DerivationExpression_UnaryOp& a, DerivationExpression_UnaryOp& b) { + friend void swap(DerivationExpression_ExpressionUserDefined& a, DerivationExpression_ExpressionUserDefined& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_UnaryOp* other) { + inline void Swap(DerivationExpression_ExpressionUserDefined* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -2435,7 +2434,7 @@ class DerivationExpression_UnaryOp final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_UnaryOp* other) { + void UnsafeArenaSwap(DerivationExpression_ExpressionUserDefined* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -2443,13 +2442,13 @@ class DerivationExpression_UnaryOp final : // implements Message ---------------------------------------------- - DerivationExpression_UnaryOp* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_ExpressionUserDefined* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_UnaryOp& from); + void CopyFrom(const DerivationExpression_ExpressionUserDefined& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_UnaryOp& from); + void MergeFrom(const DerivationExpression_ExpressionUserDefined& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -2466,15 +2465,15 @@ class DerivationExpression_UnaryOp final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_UnaryOp* other); + void InternalSwap(DerivationExpression_ExpressionUserDefined* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.UnaryOp"; + return "substrait.DerivationExpression.ExpressionUserDefined"; } protected: - explicit DerivationExpression_UnaryOp(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_ExpressionUserDefined(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -2488,101 +2487,73 @@ class DerivationExpression_UnaryOp final : // nested types ---------------------------------------------------- - typedef DerivationExpression_UnaryOp_UnaryOpType UnaryOpType; - static constexpr UnaryOpType UNARY_OP_TYPE_UNSPECIFIED = - DerivationExpression_UnaryOp_UnaryOpType_UNARY_OP_TYPE_UNSPECIFIED; - static constexpr UnaryOpType UNARY_OP_TYPE_BOOLEAN_NOT = - DerivationExpression_UnaryOp_UnaryOpType_UNARY_OP_TYPE_BOOLEAN_NOT; - static inline bool UnaryOpType_IsValid(int value) { - return DerivationExpression_UnaryOp_UnaryOpType_IsValid(value); - } - static constexpr UnaryOpType UnaryOpType_MIN = - DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_MIN; - static constexpr UnaryOpType UnaryOpType_MAX = - DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_MAX; - static constexpr int UnaryOpType_ARRAYSIZE = - DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_ARRAYSIZE; - static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* - UnaryOpType_descriptor() { - return DerivationExpression_UnaryOp_UnaryOpType_descriptor(); - } - template - static inline const std::string& UnaryOpType_Name(T enum_t_value) { - static_assert(::std::is_same::value || - ::std::is_integral::value, - "Incorrect type passed to function UnaryOpType_Name."); - return DerivationExpression_UnaryOp_UnaryOpType_Name(enum_t_value); - } - static inline bool UnaryOpType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, - UnaryOpType* value) { - return DerivationExpression_UnaryOp_UnaryOpType_Parse(name, value); - } - // accessors ------------------------------------------------------- enum : int { - kArgFieldNumber = 2, - kOpTypeFieldNumber = 1, + kTypePointerFieldNumber = 1, + kVariationPointerFieldNumber = 2, + kNullabilityFieldNumber = 3, }; - // .substrait.DerivationExpression arg = 2; - bool has_arg() const; + // uint32 type_pointer = 1; + void clear_type_pointer(); + uint32_t type_pointer() const; + void set_type_pointer(uint32_t value); private: - bool _internal_has_arg() const; + uint32_t _internal_type_pointer() const; + void _internal_set_type_pointer(uint32_t value); public: - void clear_arg(); - const ::substrait::DerivationExpression& arg() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg(); - ::substrait::DerivationExpression* mutable_arg(); - void set_allocated_arg(::substrait::DerivationExpression* arg); + + // uint32 variation_pointer = 2; + void clear_variation_pointer(); + uint32_t variation_pointer() const; + void set_variation_pointer(uint32_t value); private: - const ::substrait::DerivationExpression& _internal_arg() const; - ::substrait::DerivationExpression* _internal_mutable_arg(); + uint32_t _internal_variation_pointer() const; + void _internal_set_variation_pointer(uint32_t value); public: - void unsafe_arena_set_allocated_arg( - ::substrait::DerivationExpression* arg); - ::substrait::DerivationExpression* unsafe_arena_release_arg(); - // .substrait.DerivationExpression.UnaryOp.UnaryOpType op_type = 1; - void clear_op_type(); - ::substrait::DerivationExpression_UnaryOp_UnaryOpType op_type() const; - void set_op_type(::substrait::DerivationExpression_UnaryOp_UnaryOpType value); + // .substrait.Type.Nullability nullability = 3; + void clear_nullability(); + ::substrait::Type_Nullability nullability() const; + void set_nullability(::substrait::Type_Nullability value); private: - ::substrait::DerivationExpression_UnaryOp_UnaryOpType _internal_op_type() const; - void _internal_set_op_type(::substrait::DerivationExpression_UnaryOp_UnaryOpType value); + ::substrait::Type_Nullability _internal_nullability() const; + void _internal_set_nullability(::substrait::Type_Nullability value); public: - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.UnaryOp) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ExpressionUserDefined) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::DerivationExpression* arg_; - int op_type_; + uint32_t type_pointer_; + uint32_t variation_pointer_; + int nullability_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_BinaryOp final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.BinaryOp) */ { +class DerivationExpression_IfElse final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.IfElse) */ { public: - inline DerivationExpression_BinaryOp() : DerivationExpression_BinaryOp(nullptr) {} - ~DerivationExpression_BinaryOp() override; - explicit constexpr DerivationExpression_BinaryOp(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_IfElse() : DerivationExpression_IfElse(nullptr) {} + ~DerivationExpression_IfElse() override; + explicit constexpr DerivationExpression_IfElse(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_BinaryOp(const DerivationExpression_BinaryOp& from); - DerivationExpression_BinaryOp(DerivationExpression_BinaryOp&& from) noexcept - : DerivationExpression_BinaryOp() { + DerivationExpression_IfElse(const DerivationExpression_IfElse& from); + DerivationExpression_IfElse(DerivationExpression_IfElse&& from) noexcept + : DerivationExpression_IfElse() { *this = ::std::move(from); } - inline DerivationExpression_BinaryOp& operator=(const DerivationExpression_BinaryOp& from) { + inline DerivationExpression_IfElse& operator=(const DerivationExpression_IfElse& from) { CopyFrom(from); return *this; } - inline DerivationExpression_BinaryOp& operator=(DerivationExpression_BinaryOp&& from) noexcept { + inline DerivationExpression_IfElse& operator=(DerivationExpression_IfElse&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -2605,20 +2576,20 @@ class DerivationExpression_BinaryOp final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_BinaryOp& default_instance() { + static const DerivationExpression_IfElse& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_BinaryOp* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_BinaryOp_default_instance_); + static inline const DerivationExpression_IfElse* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_IfElse_default_instance_); } static constexpr int kIndexInFileMessages = 13; - friend void swap(DerivationExpression_BinaryOp& a, DerivationExpression_BinaryOp& b) { + friend void swap(DerivationExpression_IfElse& a, DerivationExpression_IfElse& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_BinaryOp* other) { + inline void Swap(DerivationExpression_IfElse* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -2631,7 +2602,7 @@ class DerivationExpression_BinaryOp final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_BinaryOp* other) { + void UnsafeArenaSwap(DerivationExpression_IfElse* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -2639,13 +2610,13 @@ class DerivationExpression_BinaryOp final : // implements Message ---------------------------------------------- - DerivationExpression_BinaryOp* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_IfElse* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_BinaryOp& from); + void CopyFrom(const DerivationExpression_IfElse& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_BinaryOp& from); + void MergeFrom(const DerivationExpression_IfElse& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -2662,15 +2633,15 @@ class DerivationExpression_BinaryOp final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_BinaryOp* other); + void InternalSwap(DerivationExpression_IfElse* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.BinaryOp"; + return "substrait.DerivationExpression.IfElse"; } protected: - explicit DerivationExpression_BinaryOp(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_IfElse(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -2684,143 +2655,100 @@ class DerivationExpression_BinaryOp final : // nested types ---------------------------------------------------- - typedef DerivationExpression_BinaryOp_BinaryOpType BinaryOpType; - static constexpr BinaryOpType BINARY_OP_TYPE_UNSPECIFIED = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_UNSPECIFIED; - static constexpr BinaryOpType BINARY_OP_TYPE_PLUS = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_PLUS; - static constexpr BinaryOpType BINARY_OP_TYPE_MINUS = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MINUS; - static constexpr BinaryOpType BINARY_OP_TYPE_MULTIPLY = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MULTIPLY; - static constexpr BinaryOpType BINARY_OP_TYPE_DIVIDE = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_DIVIDE; - static constexpr BinaryOpType BINARY_OP_TYPE_MIN = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MIN; - static constexpr BinaryOpType BINARY_OP_TYPE_MAX = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MAX; - static constexpr BinaryOpType BINARY_OP_TYPE_GREATER_THAN = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_GREATER_THAN; - static constexpr BinaryOpType BINARY_OP_TYPE_LESS_THAN = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_LESS_THAN; - static constexpr BinaryOpType BINARY_OP_TYPE_AND = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_AND; - static constexpr BinaryOpType BINARY_OP_TYPE_OR = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_OR; - static constexpr BinaryOpType BINARY_OP_TYPE_EQUALS = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_EQUALS; - static constexpr BinaryOpType BINARY_OP_TYPE_COVERS = - DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_COVERS; - static inline bool BinaryOpType_IsValid(int value) { - return DerivationExpression_BinaryOp_BinaryOpType_IsValid(value); - } - static constexpr BinaryOpType BinaryOpType_MIN = - DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_MIN; - static constexpr BinaryOpType BinaryOpType_MAX = - DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_MAX; - static constexpr int BinaryOpType_ARRAYSIZE = - DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_ARRAYSIZE; - static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* - BinaryOpType_descriptor() { - return DerivationExpression_BinaryOp_BinaryOpType_descriptor(); - } - template - static inline const std::string& BinaryOpType_Name(T enum_t_value) { - static_assert(::std::is_same::value || - ::std::is_integral::value, - "Incorrect type passed to function BinaryOpType_Name."); - return DerivationExpression_BinaryOp_BinaryOpType_Name(enum_t_value); - } - static inline bool BinaryOpType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, - BinaryOpType* value) { - return DerivationExpression_BinaryOp_BinaryOpType_Parse(name, value); - } - // accessors ------------------------------------------------------- enum : int { - kArg1FieldNumber = 2, - kArg2FieldNumber = 3, - kOpTypeFieldNumber = 1, + kIfConditionFieldNumber = 1, + kIfReturnFieldNumber = 2, + kElseReturnFieldNumber = 3, }; - // .substrait.DerivationExpression arg1 = 2; - bool has_arg1() const; + // .substrait.DerivationExpression if_condition = 1; + bool has_if_condition() const; private: - bool _internal_has_arg1() const; + bool _internal_has_if_condition() const; public: - void clear_arg1(); - const ::substrait::DerivationExpression& arg1() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg1(); - ::substrait::DerivationExpression* mutable_arg1(); - void set_allocated_arg1(::substrait::DerivationExpression* arg1); + void clear_if_condition(); + const ::substrait::DerivationExpression& if_condition() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_if_condition(); + ::substrait::DerivationExpression* mutable_if_condition(); + void set_allocated_if_condition(::substrait::DerivationExpression* if_condition); private: - const ::substrait::DerivationExpression& _internal_arg1() const; - ::substrait::DerivationExpression* _internal_mutable_arg1(); + const ::substrait::DerivationExpression& _internal_if_condition() const; + ::substrait::DerivationExpression* _internal_mutable_if_condition(); public: - void unsafe_arena_set_allocated_arg1( - ::substrait::DerivationExpression* arg1); - ::substrait::DerivationExpression* unsafe_arena_release_arg1(); + void unsafe_arena_set_allocated_if_condition( + ::substrait::DerivationExpression* if_condition); + ::substrait::DerivationExpression* unsafe_arena_release_if_condition(); - // .substrait.DerivationExpression arg2 = 3; - bool has_arg2() const; + // .substrait.DerivationExpression if_return = 2; + bool has_if_return() const; private: - bool _internal_has_arg2() const; + bool _internal_has_if_return() const; public: - void clear_arg2(); - const ::substrait::DerivationExpression& arg2() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg2(); - ::substrait::DerivationExpression* mutable_arg2(); - void set_allocated_arg2(::substrait::DerivationExpression* arg2); + void clear_if_return(); + const ::substrait::DerivationExpression& if_return() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_if_return(); + ::substrait::DerivationExpression* mutable_if_return(); + void set_allocated_if_return(::substrait::DerivationExpression* if_return); private: - const ::substrait::DerivationExpression& _internal_arg2() const; - ::substrait::DerivationExpression* _internal_mutable_arg2(); + const ::substrait::DerivationExpression& _internal_if_return() const; + ::substrait::DerivationExpression* _internal_mutable_if_return(); public: - void unsafe_arena_set_allocated_arg2( - ::substrait::DerivationExpression* arg2); - ::substrait::DerivationExpression* unsafe_arena_release_arg2(); + void unsafe_arena_set_allocated_if_return( + ::substrait::DerivationExpression* if_return); + ::substrait::DerivationExpression* unsafe_arena_release_if_return(); - // .substrait.DerivationExpression.BinaryOp.BinaryOpType op_type = 1; - void clear_op_type(); - ::substrait::DerivationExpression_BinaryOp_BinaryOpType op_type() const; - void set_op_type(::substrait::DerivationExpression_BinaryOp_BinaryOpType value); + // .substrait.DerivationExpression else_return = 3; + bool has_else_return() const; private: - ::substrait::DerivationExpression_BinaryOp_BinaryOpType _internal_op_type() const; - void _internal_set_op_type(::substrait::DerivationExpression_BinaryOp_BinaryOpType value); + bool _internal_has_else_return() const; public: + void clear_else_return(); + const ::substrait::DerivationExpression& else_return() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_else_return(); + ::substrait::DerivationExpression* mutable_else_return(); + void set_allocated_else_return(::substrait::DerivationExpression* else_return); + private: + const ::substrait::DerivationExpression& _internal_else_return() const; + ::substrait::DerivationExpression* _internal_mutable_else_return(); + public: + void unsafe_arena_set_allocated_else_return( + ::substrait::DerivationExpression* else_return); + ::substrait::DerivationExpression* unsafe_arena_release_else_return(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.BinaryOp) + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.IfElse) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::substrait::DerivationExpression* arg1_; - ::substrait::DerivationExpression* arg2_; - int op_type_; + ::substrait::DerivationExpression* if_condition_; + ::substrait::DerivationExpression* if_return_; + ::substrait::DerivationExpression* else_return_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_ReturnProgram_Assignment final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ReturnProgram.Assignment) */ { +class DerivationExpression_UnaryOp final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.UnaryOp) */ { public: - inline DerivationExpression_ReturnProgram_Assignment() : DerivationExpression_ReturnProgram_Assignment(nullptr) {} - ~DerivationExpression_ReturnProgram_Assignment() override; - explicit constexpr DerivationExpression_ReturnProgram_Assignment(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_UnaryOp() : DerivationExpression_UnaryOp(nullptr) {} + ~DerivationExpression_UnaryOp() override; + explicit constexpr DerivationExpression_UnaryOp(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ReturnProgram_Assignment(const DerivationExpression_ReturnProgram_Assignment& from); - DerivationExpression_ReturnProgram_Assignment(DerivationExpression_ReturnProgram_Assignment&& from) noexcept - : DerivationExpression_ReturnProgram_Assignment() { + DerivationExpression_UnaryOp(const DerivationExpression_UnaryOp& from); + DerivationExpression_UnaryOp(DerivationExpression_UnaryOp&& from) noexcept + : DerivationExpression_UnaryOp() { *this = ::std::move(from); } - inline DerivationExpression_ReturnProgram_Assignment& operator=(const DerivationExpression_ReturnProgram_Assignment& from) { + inline DerivationExpression_UnaryOp& operator=(const DerivationExpression_UnaryOp& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ReturnProgram_Assignment& operator=(DerivationExpression_ReturnProgram_Assignment&& from) noexcept { + inline DerivationExpression_UnaryOp& operator=(DerivationExpression_UnaryOp&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -2843,20 +2771,20 @@ class DerivationExpression_ReturnProgram_Assignment final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ReturnProgram_Assignment& default_instance() { + static const DerivationExpression_UnaryOp& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ReturnProgram_Assignment* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ReturnProgram_Assignment_default_instance_); + static inline const DerivationExpression_UnaryOp* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_UnaryOp_default_instance_); } static constexpr int kIndexInFileMessages = 14; - friend void swap(DerivationExpression_ReturnProgram_Assignment& a, DerivationExpression_ReturnProgram_Assignment& b) { + friend void swap(DerivationExpression_UnaryOp& a, DerivationExpression_UnaryOp& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ReturnProgram_Assignment* other) { + inline void Swap(DerivationExpression_UnaryOp* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -2869,7 +2797,7 @@ class DerivationExpression_ReturnProgram_Assignment final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ReturnProgram_Assignment* other) { + void UnsafeArenaSwap(DerivationExpression_UnaryOp* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -2877,13 +2805,13 @@ class DerivationExpression_ReturnProgram_Assignment final : // implements Message ---------------------------------------------- - DerivationExpression_ReturnProgram_Assignment* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_UnaryOp* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ReturnProgram_Assignment& from); + void CopyFrom(const DerivationExpression_UnaryOp& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ReturnProgram_Assignment& from); + void MergeFrom(const DerivationExpression_UnaryOp& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -2900,15 +2828,15 @@ class DerivationExpression_ReturnProgram_Assignment final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ReturnProgram_Assignment* other); + void InternalSwap(DerivationExpression_UnaryOp* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ReturnProgram.Assignment"; + return "substrait.DerivationExpression.UnaryOp"; } protected: - explicit DerivationExpression_ReturnProgram_Assignment(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_UnaryOp(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -2922,76 +2850,101 @@ class DerivationExpression_ReturnProgram_Assignment final : // nested types ---------------------------------------------------- - // accessors ------------------------------------------------------- - - enum : int { - kNameFieldNumber = 1, - kExpressionFieldNumber = 2, - }; - // string name = 1; - void clear_name(); - const std::string& name() const; - template - void set_name(ArgT0&& arg0, ArgT... args); - std::string* mutable_name(); - PROTOBUF_NODISCARD std::string* release_name(); - void set_allocated_name(std::string* name); - private: - const std::string& _internal_name() const; - inline PROTOBUF_ALWAYS_INLINE void _internal_set_name(const std::string& value); - std::string* _internal_mutable_name(); - public: - - // .substrait.DerivationExpression expression = 2; - bool has_expression() const; - private: - bool _internal_has_expression() const; - public: - void clear_expression(); - const ::substrait::DerivationExpression& expression() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_expression(); - ::substrait::DerivationExpression* mutable_expression(); - void set_allocated_expression(::substrait::DerivationExpression* expression); + typedef DerivationExpression_UnaryOp_UnaryOpType UnaryOpType; + static constexpr UnaryOpType UNARY_OP_TYPE_UNSPECIFIED = + DerivationExpression_UnaryOp_UnaryOpType_UNARY_OP_TYPE_UNSPECIFIED; + static constexpr UnaryOpType UNARY_OP_TYPE_BOOLEAN_NOT = + DerivationExpression_UnaryOp_UnaryOpType_UNARY_OP_TYPE_BOOLEAN_NOT; + static inline bool UnaryOpType_IsValid(int value) { + return DerivationExpression_UnaryOp_UnaryOpType_IsValid(value); + } + static constexpr UnaryOpType UnaryOpType_MIN = + DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_MIN; + static constexpr UnaryOpType UnaryOpType_MAX = + DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_MAX; + static constexpr int UnaryOpType_ARRAYSIZE = + DerivationExpression_UnaryOp_UnaryOpType_UnaryOpType_ARRAYSIZE; + static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* + UnaryOpType_descriptor() { + return DerivationExpression_UnaryOp_UnaryOpType_descriptor(); + } + template + static inline const std::string& UnaryOpType_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function UnaryOpType_Name."); + return DerivationExpression_UnaryOp_UnaryOpType_Name(enum_t_value); + } + static inline bool UnaryOpType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, + UnaryOpType* value) { + return DerivationExpression_UnaryOp_UnaryOpType_Parse(name, value); + } + + // accessors ------------------------------------------------------- + + enum : int { + kArgFieldNumber = 2, + kOpTypeFieldNumber = 1, + }; + // .substrait.DerivationExpression arg = 2; + bool has_arg() const; private: - const ::substrait::DerivationExpression& _internal_expression() const; - ::substrait::DerivationExpression* _internal_mutable_expression(); + bool _internal_has_arg() const; public: - void unsafe_arena_set_allocated_expression( - ::substrait::DerivationExpression* expression); - ::substrait::DerivationExpression* unsafe_arena_release_expression(); + void clear_arg(); + const ::substrait::DerivationExpression& arg() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg(); + ::substrait::DerivationExpression* mutable_arg(); + void set_allocated_arg(::substrait::DerivationExpression* arg); + private: + const ::substrait::DerivationExpression& _internal_arg() const; + ::substrait::DerivationExpression* _internal_mutable_arg(); + public: + void unsafe_arena_set_allocated_arg( + ::substrait::DerivationExpression* arg); + ::substrait::DerivationExpression* unsafe_arena_release_arg(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ReturnProgram.Assignment) + // .substrait.DerivationExpression.UnaryOp.UnaryOpType op_type = 1; + void clear_op_type(); + ::substrait::DerivationExpression_UnaryOp_UnaryOpType op_type() const; + void set_op_type(::substrait::DerivationExpression_UnaryOp_UnaryOpType value); + private: + ::substrait::DerivationExpression_UnaryOp_UnaryOpType _internal_op_type() const; + void _internal_set_op_type(::substrait::DerivationExpression_UnaryOp_UnaryOpType value); + public: + + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.UnaryOp) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr name_; - ::substrait::DerivationExpression* expression_; + ::substrait::DerivationExpression* arg_; + int op_type_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression_ReturnProgram final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ReturnProgram) */ { +class DerivationExpression_BinaryOp final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.BinaryOp) */ { public: - inline DerivationExpression_ReturnProgram() : DerivationExpression_ReturnProgram(nullptr) {} - ~DerivationExpression_ReturnProgram() override; - explicit constexpr DerivationExpression_ReturnProgram(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_BinaryOp() : DerivationExpression_BinaryOp(nullptr) {} + ~DerivationExpression_BinaryOp() override; + explicit constexpr DerivationExpression_BinaryOp(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); - DerivationExpression_ReturnProgram(const DerivationExpression_ReturnProgram& from); - DerivationExpression_ReturnProgram(DerivationExpression_ReturnProgram&& from) noexcept - : DerivationExpression_ReturnProgram() { + DerivationExpression_BinaryOp(const DerivationExpression_BinaryOp& from); + DerivationExpression_BinaryOp(DerivationExpression_BinaryOp&& from) noexcept + : DerivationExpression_BinaryOp() { *this = ::std::move(from); } - inline DerivationExpression_ReturnProgram& operator=(const DerivationExpression_ReturnProgram& from) { + inline DerivationExpression_BinaryOp& operator=(const DerivationExpression_BinaryOp& from) { CopyFrom(from); return *this; } - inline DerivationExpression_ReturnProgram& operator=(DerivationExpression_ReturnProgram&& from) noexcept { + inline DerivationExpression_BinaryOp& operator=(DerivationExpression_BinaryOp&& from) noexcept { if (this == &from) return *this; if (GetOwningArena() == from.GetOwningArena() #ifdef PROTOBUF_FORCE_COPY_IN_MOVE @@ -3014,20 +2967,20 @@ class DerivationExpression_ReturnProgram final : static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { return default_instance().GetMetadata().reflection; } - static const DerivationExpression_ReturnProgram& default_instance() { + static const DerivationExpression_BinaryOp& default_instance() { return *internal_default_instance(); } - static inline const DerivationExpression_ReturnProgram* internal_default_instance() { - return reinterpret_cast( - &_DerivationExpression_ReturnProgram_default_instance_); + static inline const DerivationExpression_BinaryOp* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_BinaryOp_default_instance_); } static constexpr int kIndexInFileMessages = 15; - friend void swap(DerivationExpression_ReturnProgram& a, DerivationExpression_ReturnProgram& b) { + friend void swap(DerivationExpression_BinaryOp& a, DerivationExpression_BinaryOp& b) { a.Swap(&b); } - inline void Swap(DerivationExpression_ReturnProgram* other) { + inline void Swap(DerivationExpression_BinaryOp* other) { if (other == this) return; #ifdef PROTOBUF_FORCE_COPY_IN_SWAP if (GetOwningArena() != nullptr && @@ -3040,7 +2993,7 @@ class DerivationExpression_ReturnProgram final : ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); } } - void UnsafeArenaSwap(DerivationExpression_ReturnProgram* other) { + void UnsafeArenaSwap(DerivationExpression_BinaryOp* other) { if (other == this) return; GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); InternalSwap(other); @@ -3048,13 +3001,13 @@ class DerivationExpression_ReturnProgram final : // implements Message ---------------------------------------------- - DerivationExpression_ReturnProgram* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { - return CreateMaybeMessage(arena); + DerivationExpression_BinaryOp* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); } using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; - void CopyFrom(const DerivationExpression_ReturnProgram& from); + void CopyFrom(const DerivationExpression_BinaryOp& from); using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; - void MergeFrom(const DerivationExpression_ReturnProgram& from); + void MergeFrom(const DerivationExpression_BinaryOp& from); private: static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); public: @@ -3071,15 +3024,15 @@ class DerivationExpression_ReturnProgram final : void SharedCtor(); void SharedDtor(); void SetCachedSize(int size) const final; - void InternalSwap(DerivationExpression_ReturnProgram* other); + void InternalSwap(DerivationExpression_BinaryOp* other); private: friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { - return "substrait.DerivationExpression.ReturnProgram"; + return "substrait.DerivationExpression.BinaryOp"; } protected: - explicit DerivationExpression_ReturnProgram(::PROTOBUF_NAMESPACE_ID::Arena* arena, + explicit DerivationExpression_BinaryOp(::PROTOBUF_NAMESPACE_ID::Arena* arena, bool is_message_owned = false); private: static void ArenaDtor(void* object); @@ -3093,70 +3046,479 @@ class DerivationExpression_ReturnProgram final : // nested types ---------------------------------------------------- - typedef DerivationExpression_ReturnProgram_Assignment Assignment; + typedef DerivationExpression_BinaryOp_BinaryOpType BinaryOpType; + static constexpr BinaryOpType BINARY_OP_TYPE_UNSPECIFIED = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_UNSPECIFIED; + static constexpr BinaryOpType BINARY_OP_TYPE_PLUS = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_PLUS; + static constexpr BinaryOpType BINARY_OP_TYPE_MINUS = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MINUS; + static constexpr BinaryOpType BINARY_OP_TYPE_MULTIPLY = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MULTIPLY; + static constexpr BinaryOpType BINARY_OP_TYPE_DIVIDE = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_DIVIDE; + static constexpr BinaryOpType BINARY_OP_TYPE_MIN = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MIN; + static constexpr BinaryOpType BINARY_OP_TYPE_MAX = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_MAX; + static constexpr BinaryOpType BINARY_OP_TYPE_GREATER_THAN = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_GREATER_THAN; + static constexpr BinaryOpType BINARY_OP_TYPE_LESS_THAN = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_LESS_THAN; + static constexpr BinaryOpType BINARY_OP_TYPE_AND = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_AND; + static constexpr BinaryOpType BINARY_OP_TYPE_OR = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_OR; + static constexpr BinaryOpType BINARY_OP_TYPE_EQUALS = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_EQUALS; + static constexpr BinaryOpType BINARY_OP_TYPE_COVERS = + DerivationExpression_BinaryOp_BinaryOpType_BINARY_OP_TYPE_COVERS; + static inline bool BinaryOpType_IsValid(int value) { + return DerivationExpression_BinaryOp_BinaryOpType_IsValid(value); + } + static constexpr BinaryOpType BinaryOpType_MIN = + DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_MIN; + static constexpr BinaryOpType BinaryOpType_MAX = + DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_MAX; + static constexpr int BinaryOpType_ARRAYSIZE = + DerivationExpression_BinaryOp_BinaryOpType_BinaryOpType_ARRAYSIZE; + static inline const ::PROTOBUF_NAMESPACE_ID::EnumDescriptor* + BinaryOpType_descriptor() { + return DerivationExpression_BinaryOp_BinaryOpType_descriptor(); + } + template + static inline const std::string& BinaryOpType_Name(T enum_t_value) { + static_assert(::std::is_same::value || + ::std::is_integral::value, + "Incorrect type passed to function BinaryOpType_Name."); + return DerivationExpression_BinaryOp_BinaryOpType_Name(enum_t_value); + } + static inline bool BinaryOpType_Parse(::PROTOBUF_NAMESPACE_ID::ConstStringParam name, + BinaryOpType* value) { + return DerivationExpression_BinaryOp_BinaryOpType_Parse(name, value); + } // accessors ------------------------------------------------------- enum : int { - kAssignmentsFieldNumber = 1, - kFinalExpressionFieldNumber = 2, + kArg1FieldNumber = 2, + kArg2FieldNumber = 3, + kOpTypeFieldNumber = 1, }; - // repeated .substrait.DerivationExpression.ReturnProgram.Assignment assignments = 1; - int assignments_size() const; + // .substrait.DerivationExpression arg1 = 2; + bool has_arg1() const; private: - int _internal_assignments_size() const; + bool _internal_has_arg1() const; public: - void clear_assignments(); - ::substrait::DerivationExpression_ReturnProgram_Assignment* mutable_assignments(int index); - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment >* - mutable_assignments(); + void clear_arg1(); + const ::substrait::DerivationExpression& arg1() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg1(); + ::substrait::DerivationExpression* mutable_arg1(); + void set_allocated_arg1(::substrait::DerivationExpression* arg1); private: - const ::substrait::DerivationExpression_ReturnProgram_Assignment& _internal_assignments(int index) const; - ::substrait::DerivationExpression_ReturnProgram_Assignment* _internal_add_assignments(); + const ::substrait::DerivationExpression& _internal_arg1() const; + ::substrait::DerivationExpression* _internal_mutable_arg1(); public: - const ::substrait::DerivationExpression_ReturnProgram_Assignment& assignments(int index) const; - ::substrait::DerivationExpression_ReturnProgram_Assignment* add_assignments(); - const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment >& - assignments() const; + void unsafe_arena_set_allocated_arg1( + ::substrait::DerivationExpression* arg1); + ::substrait::DerivationExpression* unsafe_arena_release_arg1(); - // .substrait.DerivationExpression final_expression = 2; - bool has_final_expression() const; + // .substrait.DerivationExpression arg2 = 3; + bool has_arg2() const; private: - bool _internal_has_final_expression() const; + bool _internal_has_arg2() const; public: - void clear_final_expression(); - const ::substrait::DerivationExpression& final_expression() const; - PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_final_expression(); - ::substrait::DerivationExpression* mutable_final_expression(); - void set_allocated_final_expression(::substrait::DerivationExpression* final_expression); + void clear_arg2(); + const ::substrait::DerivationExpression& arg2() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_arg2(); + ::substrait::DerivationExpression* mutable_arg2(); + void set_allocated_arg2(::substrait::DerivationExpression* arg2); private: - const ::substrait::DerivationExpression& _internal_final_expression() const; - ::substrait::DerivationExpression* _internal_mutable_final_expression(); + const ::substrait::DerivationExpression& _internal_arg2() const; + ::substrait::DerivationExpression* _internal_mutable_arg2(); public: - void unsafe_arena_set_allocated_final_expression( - ::substrait::DerivationExpression* final_expression); - ::substrait::DerivationExpression* unsafe_arena_release_final_expression(); + void unsafe_arena_set_allocated_arg2( + ::substrait::DerivationExpression* arg2); + ::substrait::DerivationExpression* unsafe_arena_release_arg2(); - // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ReturnProgram) + // .substrait.DerivationExpression.BinaryOp.BinaryOpType op_type = 1; + void clear_op_type(); + ::substrait::DerivationExpression_BinaryOp_BinaryOpType op_type() const; + void set_op_type(::substrait::DerivationExpression_BinaryOp_BinaryOpType value); + private: + ::substrait::DerivationExpression_BinaryOp_BinaryOpType _internal_op_type() const; + void _internal_set_op_type(::substrait::DerivationExpression_BinaryOp_BinaryOpType value); + public: + + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.BinaryOp) private: class _Internal; template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; typedef void InternalArenaConstructable_; typedef void DestructorSkippable_; - ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment > assignments_; - ::substrait::DerivationExpression* final_expression_; + ::substrait::DerivationExpression* arg1_; + ::substrait::DerivationExpression* arg2_; + int op_type_; mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; }; // ------------------------------------------------------------------- -class DerivationExpression final : - public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression) */ { +class DerivationExpression_ReturnProgram_Assignment final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ReturnProgram.Assignment) */ { public: - inline DerivationExpression() : DerivationExpression(nullptr) {} - ~DerivationExpression() override; - explicit constexpr DerivationExpression(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + inline DerivationExpression_ReturnProgram_Assignment() : DerivationExpression_ReturnProgram_Assignment(nullptr) {} + ~DerivationExpression_ReturnProgram_Assignment() override; + explicit constexpr DerivationExpression_ReturnProgram_Assignment(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + DerivationExpression_ReturnProgram_Assignment(const DerivationExpression_ReturnProgram_Assignment& from); + DerivationExpression_ReturnProgram_Assignment(DerivationExpression_ReturnProgram_Assignment&& from) noexcept + : DerivationExpression_ReturnProgram_Assignment() { + *this = ::std::move(from); + } + + inline DerivationExpression_ReturnProgram_Assignment& operator=(const DerivationExpression_ReturnProgram_Assignment& from) { + CopyFrom(from); + return *this; + } + inline DerivationExpression_ReturnProgram_Assignment& operator=(DerivationExpression_ReturnProgram_Assignment&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const DerivationExpression_ReturnProgram_Assignment& default_instance() { + return *internal_default_instance(); + } + static inline const DerivationExpression_ReturnProgram_Assignment* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ReturnProgram_Assignment_default_instance_); + } + static constexpr int kIndexInFileMessages = + 16; + + friend void swap(DerivationExpression_ReturnProgram_Assignment& a, DerivationExpression_ReturnProgram_Assignment& b) { + a.Swap(&b); + } + inline void Swap(DerivationExpression_ReturnProgram_Assignment* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(DerivationExpression_ReturnProgram_Assignment* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + DerivationExpression_ReturnProgram_Assignment* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const DerivationExpression_ReturnProgram_Assignment& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const DerivationExpression_ReturnProgram_Assignment& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(DerivationExpression_ReturnProgram_Assignment* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.DerivationExpression.ReturnProgram.Assignment"; + } + protected: + explicit DerivationExpression_ReturnProgram_Assignment(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + // accessors ------------------------------------------------------- + + enum : int { + kNameFieldNumber = 1, + kExpressionFieldNumber = 2, + }; + // string name = 1; + void clear_name(); + const std::string& name() const; + template + void set_name(ArgT0&& arg0, ArgT... args); + std::string* mutable_name(); + PROTOBUF_NODISCARD std::string* release_name(); + void set_allocated_name(std::string* name); + private: + const std::string& _internal_name() const; + inline PROTOBUF_ALWAYS_INLINE void _internal_set_name(const std::string& value); + std::string* _internal_mutable_name(); + public: + + // .substrait.DerivationExpression expression = 2; + bool has_expression() const; + private: + bool _internal_has_expression() const; + public: + void clear_expression(); + const ::substrait::DerivationExpression& expression() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_expression(); + ::substrait::DerivationExpression* mutable_expression(); + void set_allocated_expression(::substrait::DerivationExpression* expression); + private: + const ::substrait::DerivationExpression& _internal_expression() const; + ::substrait::DerivationExpression* _internal_mutable_expression(); + public: + void unsafe_arena_set_allocated_expression( + ::substrait::DerivationExpression* expression); + ::substrait::DerivationExpression* unsafe_arena_release_expression(); + + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ReturnProgram.Assignment) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::PROTOBUF_NAMESPACE_ID::internal::ArenaStringPtr name_; + ::substrait::DerivationExpression* expression_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; +}; +// ------------------------------------------------------------------- + +class DerivationExpression_ReturnProgram final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression.ReturnProgram) */ { + public: + inline DerivationExpression_ReturnProgram() : DerivationExpression_ReturnProgram(nullptr) {} + ~DerivationExpression_ReturnProgram() override; + explicit constexpr DerivationExpression_ReturnProgram(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); + + DerivationExpression_ReturnProgram(const DerivationExpression_ReturnProgram& from); + DerivationExpression_ReturnProgram(DerivationExpression_ReturnProgram&& from) noexcept + : DerivationExpression_ReturnProgram() { + *this = ::std::move(from); + } + + inline DerivationExpression_ReturnProgram& operator=(const DerivationExpression_ReturnProgram& from) { + CopyFrom(from); + return *this; + } + inline DerivationExpression_ReturnProgram& operator=(DerivationExpression_ReturnProgram&& from) noexcept { + if (this == &from) return *this; + if (GetOwningArena() == from.GetOwningArena() + #ifdef PROTOBUF_FORCE_COPY_IN_MOVE + && GetOwningArena() != nullptr + #endif // !PROTOBUF_FORCE_COPY_IN_MOVE + ) { + InternalSwap(&from); + } else { + CopyFrom(from); + } + return *this; + } + + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* descriptor() { + return GetDescriptor(); + } + static const ::PROTOBUF_NAMESPACE_ID::Descriptor* GetDescriptor() { + return default_instance().GetMetadata().descriptor; + } + static const ::PROTOBUF_NAMESPACE_ID::Reflection* GetReflection() { + return default_instance().GetMetadata().reflection; + } + static const DerivationExpression_ReturnProgram& default_instance() { + return *internal_default_instance(); + } + static inline const DerivationExpression_ReturnProgram* internal_default_instance() { + return reinterpret_cast( + &_DerivationExpression_ReturnProgram_default_instance_); + } + static constexpr int kIndexInFileMessages = + 17; + + friend void swap(DerivationExpression_ReturnProgram& a, DerivationExpression_ReturnProgram& b) { + a.Swap(&b); + } + inline void Swap(DerivationExpression_ReturnProgram* other) { + if (other == this) return; + #ifdef PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() != nullptr && + GetOwningArena() == other->GetOwningArena()) { + #else // PROTOBUF_FORCE_COPY_IN_SWAP + if (GetOwningArena() == other->GetOwningArena()) { + #endif // !PROTOBUF_FORCE_COPY_IN_SWAP + InternalSwap(other); + } else { + ::PROTOBUF_NAMESPACE_ID::internal::GenericSwap(this, other); + } + } + void UnsafeArenaSwap(DerivationExpression_ReturnProgram* other) { + if (other == this) return; + GOOGLE_DCHECK(GetOwningArena() == other->GetOwningArena()); + InternalSwap(other); + } + + // implements Message ---------------------------------------------- + + DerivationExpression_ReturnProgram* New(::PROTOBUF_NAMESPACE_ID::Arena* arena = nullptr) const final { + return CreateMaybeMessage(arena); + } + using ::PROTOBUF_NAMESPACE_ID::Message::CopyFrom; + void CopyFrom(const DerivationExpression_ReturnProgram& from); + using ::PROTOBUF_NAMESPACE_ID::Message::MergeFrom; + void MergeFrom(const DerivationExpression_ReturnProgram& from); + private: + static void MergeImpl(::PROTOBUF_NAMESPACE_ID::Message* to, const ::PROTOBUF_NAMESPACE_ID::Message& from); + public: + PROTOBUF_ATTRIBUTE_REINITIALIZES void Clear() final; + bool IsInitialized() const final; + + size_t ByteSizeLong() const final; + const char* _InternalParse(const char* ptr, ::PROTOBUF_NAMESPACE_ID::internal::ParseContext* ctx) final; + uint8_t* _InternalSerialize( + uint8_t* target, ::PROTOBUF_NAMESPACE_ID::io::EpsCopyOutputStream* stream) const final; + int GetCachedSize() const final { return _cached_size_.Get(); } + + private: + void SharedCtor(); + void SharedDtor(); + void SetCachedSize(int size) const final; + void InternalSwap(DerivationExpression_ReturnProgram* other); + + private: + friend class ::PROTOBUF_NAMESPACE_ID::internal::AnyMetadata; + static ::PROTOBUF_NAMESPACE_ID::StringPiece FullMessageName() { + return "substrait.DerivationExpression.ReturnProgram"; + } + protected: + explicit DerivationExpression_ReturnProgram(::PROTOBUF_NAMESPACE_ID::Arena* arena, + bool is_message_owned = false); + private: + static void ArenaDtor(void* object); + inline void RegisterArenaDtor(::PROTOBUF_NAMESPACE_ID::Arena* arena); + public: + + static const ClassData _class_data_; + const ::PROTOBUF_NAMESPACE_ID::Message::ClassData*GetClassData() const final; + + ::PROTOBUF_NAMESPACE_ID::Metadata GetMetadata() const final; + + // nested types ---------------------------------------------------- + + typedef DerivationExpression_ReturnProgram_Assignment Assignment; + + // accessors ------------------------------------------------------- + + enum : int { + kAssignmentsFieldNumber = 1, + kFinalExpressionFieldNumber = 2, + }; + // repeated .substrait.DerivationExpression.ReturnProgram.Assignment assignments = 1; + int assignments_size() const; + private: + int _internal_assignments_size() const; + public: + void clear_assignments(); + ::substrait::DerivationExpression_ReturnProgram_Assignment* mutable_assignments(int index); + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment >* + mutable_assignments(); + private: + const ::substrait::DerivationExpression_ReturnProgram_Assignment& _internal_assignments(int index) const; + ::substrait::DerivationExpression_ReturnProgram_Assignment* _internal_add_assignments(); + public: + const ::substrait::DerivationExpression_ReturnProgram_Assignment& assignments(int index) const; + ::substrait::DerivationExpression_ReturnProgram_Assignment* add_assignments(); + const ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment >& + assignments() const; + + // .substrait.DerivationExpression final_expression = 2; + bool has_final_expression() const; + private: + bool _internal_has_final_expression() const; + public: + void clear_final_expression(); + const ::substrait::DerivationExpression& final_expression() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression* release_final_expression(); + ::substrait::DerivationExpression* mutable_final_expression(); + void set_allocated_final_expression(::substrait::DerivationExpression* final_expression); + private: + const ::substrait::DerivationExpression& _internal_final_expression() const; + ::substrait::DerivationExpression* _internal_mutable_final_expression(); + public: + void unsafe_arena_set_allocated_final_expression( + ::substrait::DerivationExpression* final_expression); + ::substrait::DerivationExpression* unsafe_arena_release_final_expression(); + + // @@protoc_insertion_point(class_scope:substrait.DerivationExpression.ReturnProgram) + private: + class _Internal; + + template friend class ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper; + typedef void InternalArenaConstructable_; + typedef void DestructorSkippable_; + ::PROTOBUF_NAMESPACE_ID::RepeatedPtrField< ::substrait::DerivationExpression_ReturnProgram_Assignment > assignments_; + ::substrait::DerivationExpression* final_expression_; + mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; + friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; +}; +// ------------------------------------------------------------------- + +class DerivationExpression final : + public ::PROTOBUF_NAMESPACE_ID::Message /* @@protoc_insertion_point(class_definition:substrait.DerivationExpression) */ { + public: + inline DerivationExpression() : DerivationExpression(nullptr) {} + ~DerivationExpression() override; + explicit constexpr DerivationExpression(::PROTOBUF_NAMESPACE_ID::internal::ConstantInitialized); DerivationExpression(const DerivationExpression& from); DerivationExpression(DerivationExpression&& from) noexcept @@ -3208,9 +3570,10 @@ class DerivationExpression final : kDate = 16, kTime = 17, kIntervalYear = 19, - kIntervalDay = 20, kTimestampTz = 29, kUuid = 32, + kIntervalDay = 20, + kIntervalCompound = 42, kFixedChar = 21, kVarchar = 22, kFixedBinary = 23, @@ -3237,7 +3600,7 @@ class DerivationExpression final : &_DerivationExpression_default_instance_); } static constexpr int kIndexInFileMessages = - 16; + 18; friend void swap(DerivationExpression& a, DerivationExpression& b) { a.Swap(&b); @@ -3313,6 +3676,8 @@ class DerivationExpression final : typedef DerivationExpression_ExpressionFixedBinary ExpressionFixedBinary; typedef DerivationExpression_ExpressionDecimal ExpressionDecimal; typedef DerivationExpression_ExpressionPrecisionTimestamp ExpressionPrecisionTimestamp; + typedef DerivationExpression_ExpressionIntervalDay ExpressionIntervalDay; + typedef DerivationExpression_ExpressionIntervalCompound ExpressionIntervalCompound; typedef DerivationExpression_ExpressionPrecisionTimestampTZ ExpressionPrecisionTimestampTZ; typedef DerivationExpression_ExpressionStruct ExpressionStruct; typedef DerivationExpression_ExpressionNamedStruct ExpressionNamedStruct; @@ -3340,9 +3705,10 @@ class DerivationExpression final : kDateFieldNumber = 16, kTimeFieldNumber = 17, kIntervalYearFieldNumber = 19, - kIntervalDayFieldNumber = 20, kTimestampTzFieldNumber = 29, kUuidFieldNumber = 32, + kIntervalDayFieldNumber = 20, + kIntervalCompoundFieldNumber = 42, kFixedCharFieldNumber = 21, kVarcharFieldNumber = 22, kFixedBinaryFieldNumber = 23, @@ -3596,24 +3962,6 @@ class DerivationExpression final : ::substrait::Type_IntervalYear* interval_year); ::substrait::Type_IntervalYear* unsafe_arena_release_interval_year(); - // .substrait.Type.IntervalDay interval_day = 20; - bool has_interval_day() const; - private: - bool _internal_has_interval_day() const; - public: - void clear_interval_day(); - const ::substrait::Type_IntervalDay& interval_day() const; - PROTOBUF_NODISCARD ::substrait::Type_IntervalDay* release_interval_day(); - ::substrait::Type_IntervalDay* mutable_interval_day(); - void set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day); - private: - const ::substrait::Type_IntervalDay& _internal_interval_day() const; - ::substrait::Type_IntervalDay* _internal_mutable_interval_day(); - public: - void unsafe_arena_set_allocated_interval_day( - ::substrait::Type_IntervalDay* interval_day); - ::substrait::Type_IntervalDay* unsafe_arena_release_interval_day(); - // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; PROTOBUF_DEPRECATED bool has_timestamp_tz() const; private: @@ -3650,6 +3998,42 @@ class DerivationExpression final : ::substrait::Type_UUID* uuid); ::substrait::Type_UUID* unsafe_arena_release_uuid(); + // .substrait.DerivationExpression.ExpressionIntervalDay interval_day = 20; + bool has_interval_day() const; + private: + bool _internal_has_interval_day() const; + public: + void clear_interval_day(); + const ::substrait::DerivationExpression_ExpressionIntervalDay& interval_day() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression_ExpressionIntervalDay* release_interval_day(); + ::substrait::DerivationExpression_ExpressionIntervalDay* mutable_interval_day(); + void set_allocated_interval_day(::substrait::DerivationExpression_ExpressionIntervalDay* interval_day); + private: + const ::substrait::DerivationExpression_ExpressionIntervalDay& _internal_interval_day() const; + ::substrait::DerivationExpression_ExpressionIntervalDay* _internal_mutable_interval_day(); + public: + void unsafe_arena_set_allocated_interval_day( + ::substrait::DerivationExpression_ExpressionIntervalDay* interval_day); + ::substrait::DerivationExpression_ExpressionIntervalDay* unsafe_arena_release_interval_day(); + + // .substrait.DerivationExpression.ExpressionIntervalCompound interval_compound = 42; + bool has_interval_compound() const; + private: + bool _internal_has_interval_compound() const; + public: + void clear_interval_compound(); + const ::substrait::DerivationExpression_ExpressionIntervalCompound& interval_compound() const; + PROTOBUF_NODISCARD ::substrait::DerivationExpression_ExpressionIntervalCompound* release_interval_compound(); + ::substrait::DerivationExpression_ExpressionIntervalCompound* mutable_interval_compound(); + void set_allocated_interval_compound(::substrait::DerivationExpression_ExpressionIntervalCompound* interval_compound); + private: + const ::substrait::DerivationExpression_ExpressionIntervalCompound& _internal_interval_compound() const; + ::substrait::DerivationExpression_ExpressionIntervalCompound* _internal_mutable_interval_compound(); + public: + void unsafe_arena_set_allocated_interval_compound( + ::substrait::DerivationExpression_ExpressionIntervalCompound* interval_compound); + ::substrait::DerivationExpression_ExpressionIntervalCompound* unsafe_arena_release_interval_compound(); + // .substrait.DerivationExpression.ExpressionFixedChar fixed_char = 21; bool has_fixed_char() const; private: @@ -3982,9 +4366,10 @@ class DerivationExpression final : void set_has_date(); void set_has_time(); void set_has_interval_year(); - void set_has_interval_day(); void set_has_timestamp_tz(); void set_has_uuid(); + void set_has_interval_day(); + void set_has_interval_compound(); void set_has_fixed_char(); void set_has_varchar(); void set_has_fixed_binary(); @@ -4026,9 +4411,10 @@ class DerivationExpression final : ::substrait::Type_Date* date_; ::substrait::Type_Time* time_; ::substrait::Type_IntervalYear* interval_year_; - ::substrait::Type_IntervalDay* interval_day_; ::substrait::Type_TimestampTZ* timestamp_tz_; ::substrait::Type_UUID* uuid_; + ::substrait::DerivationExpression_ExpressionIntervalDay* interval_day_; + ::substrait::DerivationExpression_ExpressionIntervalCompound* interval_compound_; ::substrait::DerivationExpression_ExpressionFixedChar* fixed_char_; ::substrait::DerivationExpression_ExpressionVarChar* varchar_; ::substrait::DerivationExpression_ExpressionFixedBinary* fixed_binary_; @@ -4051,42 +4437,310 @@ class DerivationExpression final : mutable ::PROTOBUF_NAMESPACE_ID::internal::CachedSize _cached_size_; uint32_t _oneof_case_[1]; - friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; -}; -// =================================================================== + friend struct ::TableStruct_substrait_2ftype_5fexpressions_2eproto; +}; +// =================================================================== + + +// =================================================================== + +#ifdef __GNUC__ + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wstrict-aliasing" +#endif // __GNUC__ +// DerivationExpression_ExpressionFixedChar + +// .substrait.DerivationExpression length = 1; +inline bool DerivationExpression_ExpressionFixedChar::_internal_has_length() const { + return this != internal_default_instance() && length_ != nullptr; +} +inline bool DerivationExpression_ExpressionFixedChar::has_length() const { + return _internal_has_length(); +} +inline void DerivationExpression_ExpressionFixedChar::clear_length() { + if (GetArenaForAllocation() == nullptr && length_ != nullptr) { + delete length_; + } + length_ = nullptr; +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedChar::_internal_length() const { + const ::substrait::DerivationExpression* p = length_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_DerivationExpression_default_instance_); +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedChar::length() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.length) + return _internal_length(); +} +inline void DerivationExpression_ExpressionFixedChar::unsafe_arena_set_allocated_length( + ::substrait::DerivationExpression* length) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(length_); + } + length_ = length; + if (length) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionFixedChar.length) +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::release_length() { + + ::substrait::DerivationExpression* temp = length_; + length_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::unsafe_arena_release_length() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionFixedChar.length) + + ::substrait::DerivationExpression* temp = length_; + length_ = nullptr; + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::_internal_mutable_length() { + + if (length_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); + length_ = p; + } + return length_; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::mutable_length() { + ::substrait::DerivationExpression* _msg = _internal_mutable_length(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionFixedChar.length) + return _msg; +} +inline void DerivationExpression_ExpressionFixedChar::set_allocated_length(::substrait::DerivationExpression* length) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete length_; + } + if (length) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(length); + if (message_arena != submessage_arena) { + length = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, length, submessage_arena); + } + + } else { + + } + length_ = length; + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionFixedChar.length) +} + +// uint32 variation_pointer = 2; +inline void DerivationExpression_ExpressionFixedChar::clear_variation_pointer() { + variation_pointer_ = 0u; +} +inline uint32_t DerivationExpression_ExpressionFixedChar::_internal_variation_pointer() const { + return variation_pointer_; +} +inline uint32_t DerivationExpression_ExpressionFixedChar::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.variation_pointer) + return _internal_variation_pointer(); +} +inline void DerivationExpression_ExpressionFixedChar::_internal_set_variation_pointer(uint32_t value) { + + variation_pointer_ = value; +} +inline void DerivationExpression_ExpressionFixedChar::set_variation_pointer(uint32_t value) { + _internal_set_variation_pointer(value); + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedChar.variation_pointer) +} + +// .substrait.Type.Nullability nullability = 3; +inline void DerivationExpression_ExpressionFixedChar::clear_nullability() { + nullability_ = 0; +} +inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedChar::_internal_nullability() const { + return static_cast< ::substrait::Type_Nullability >(nullability_); +} +inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedChar::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.nullability) + return _internal_nullability(); +} +inline void DerivationExpression_ExpressionFixedChar::_internal_set_nullability(::substrait::Type_Nullability value) { + + nullability_ = value; +} +inline void DerivationExpression_ExpressionFixedChar::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedChar.nullability) +} + +// ------------------------------------------------------------------- + +// DerivationExpression_ExpressionVarChar + +// .substrait.DerivationExpression length = 1; +inline bool DerivationExpression_ExpressionVarChar::_internal_has_length() const { + return this != internal_default_instance() && length_ != nullptr; +} +inline bool DerivationExpression_ExpressionVarChar::has_length() const { + return _internal_has_length(); +} +inline void DerivationExpression_ExpressionVarChar::clear_length() { + if (GetArenaForAllocation() == nullptr && length_ != nullptr) { + delete length_; + } + length_ = nullptr; +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionVarChar::_internal_length() const { + const ::substrait::DerivationExpression* p = length_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_DerivationExpression_default_instance_); +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionVarChar::length() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.length) + return _internal_length(); +} +inline void DerivationExpression_ExpressionVarChar::unsafe_arena_set_allocated_length( + ::substrait::DerivationExpression* length) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(length_); + } + length_ = length; + if (length) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionVarChar.length) +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::release_length() { + + ::substrait::DerivationExpression* temp = length_; + length_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::unsafe_arena_release_length() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionVarChar.length) + + ::substrait::DerivationExpression* temp = length_; + length_ = nullptr; + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::_internal_mutable_length() { + + if (length_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); + length_ = p; + } + return length_; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::mutable_length() { + ::substrait::DerivationExpression* _msg = _internal_mutable_length(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionVarChar.length) + return _msg; +} +inline void DerivationExpression_ExpressionVarChar::set_allocated_length(::substrait::DerivationExpression* length) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete length_; + } + if (length) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(length); + if (message_arena != submessage_arena) { + length = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, length, submessage_arena); + } + + } else { + + } + length_ = length; + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionVarChar.length) +} + +// uint32 variation_pointer = 2; +inline void DerivationExpression_ExpressionVarChar::clear_variation_pointer() { + variation_pointer_ = 0u; +} +inline uint32_t DerivationExpression_ExpressionVarChar::_internal_variation_pointer() const { + return variation_pointer_; +} +inline uint32_t DerivationExpression_ExpressionVarChar::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.variation_pointer) + return _internal_variation_pointer(); +} +inline void DerivationExpression_ExpressionVarChar::_internal_set_variation_pointer(uint32_t value) { + + variation_pointer_ = value; +} +inline void DerivationExpression_ExpressionVarChar::set_variation_pointer(uint32_t value) { + _internal_set_variation_pointer(value); + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionVarChar.variation_pointer) +} +// .substrait.Type.Nullability nullability = 3; +inline void DerivationExpression_ExpressionVarChar::clear_nullability() { + nullability_ = 0; +} +inline ::substrait::Type_Nullability DerivationExpression_ExpressionVarChar::_internal_nullability() const { + return static_cast< ::substrait::Type_Nullability >(nullability_); +} +inline ::substrait::Type_Nullability DerivationExpression_ExpressionVarChar::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.nullability) + return _internal_nullability(); +} +inline void DerivationExpression_ExpressionVarChar::_internal_set_nullability(::substrait::Type_Nullability value) { + + nullability_ = value; +} +inline void DerivationExpression_ExpressionVarChar::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionVarChar.nullability) +} -// =================================================================== +// ------------------------------------------------------------------- -#ifdef __GNUC__ - #pragma GCC diagnostic push - #pragma GCC diagnostic ignored "-Wstrict-aliasing" -#endif // __GNUC__ -// DerivationExpression_ExpressionFixedChar +// DerivationExpression_ExpressionFixedBinary // .substrait.DerivationExpression length = 1; -inline bool DerivationExpression_ExpressionFixedChar::_internal_has_length() const { +inline bool DerivationExpression_ExpressionFixedBinary::_internal_has_length() const { return this != internal_default_instance() && length_ != nullptr; } -inline bool DerivationExpression_ExpressionFixedChar::has_length() const { +inline bool DerivationExpression_ExpressionFixedBinary::has_length() const { return _internal_has_length(); } -inline void DerivationExpression_ExpressionFixedChar::clear_length() { +inline void DerivationExpression_ExpressionFixedBinary::clear_length() { if (GetArenaForAllocation() == nullptr && length_ != nullptr) { delete length_; } length_ = nullptr; } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedChar::_internal_length() const { +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedBinary::_internal_length() const { const ::substrait::DerivationExpression* p = length_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_DerivationExpression_default_instance_); } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedChar::length() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.length) +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedBinary::length() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.length) return _internal_length(); } -inline void DerivationExpression_ExpressionFixedChar::unsafe_arena_set_allocated_length( +inline void DerivationExpression_ExpressionFixedBinary::unsafe_arena_set_allocated_length( ::substrait::DerivationExpression* length) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(length_); @@ -4097,9 +4751,9 @@ inline void DerivationExpression_ExpressionFixedChar::unsafe_arena_set_allocated } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionFixedChar.length) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionFixedBinary.length) } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::release_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::release_length() { ::substrait::DerivationExpression* temp = length_; length_ = nullptr; @@ -4114,14 +4768,14 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedCh #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::unsafe_arena_release_length() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionFixedChar.length) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::unsafe_arena_release_length() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionFixedBinary.length) ::substrait::DerivationExpression* temp = length_; length_ = nullptr; return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::_internal_mutable_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::_internal_mutable_length() { if (length_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); @@ -4129,12 +4783,12 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedCh } return length_; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedChar::mutable_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::mutable_length() { ::substrait::DerivationExpression* _msg = _internal_mutable_length(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionFixedChar.length) + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionFixedBinary.length) return _msg; } -inline void DerivationExpression_ExpressionFixedChar::set_allocated_length(::substrait::DerivationExpression* length) { +inline void DerivationExpression_ExpressionFixedBinary::set_allocated_length(::substrait::DerivationExpression* length) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete length_; @@ -4151,92 +4805,182 @@ inline void DerivationExpression_ExpressionFixedChar::set_allocated_length(::sub } length_ = length; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionFixedChar.length) + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionFixedBinary.length) } // uint32 variation_pointer = 2; -inline void DerivationExpression_ExpressionFixedChar::clear_variation_pointer() { +inline void DerivationExpression_ExpressionFixedBinary::clear_variation_pointer() { variation_pointer_ = 0u; } -inline uint32_t DerivationExpression_ExpressionFixedChar::_internal_variation_pointer() const { +inline uint32_t DerivationExpression_ExpressionFixedBinary::_internal_variation_pointer() const { return variation_pointer_; } -inline uint32_t DerivationExpression_ExpressionFixedChar::variation_pointer() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.variation_pointer) +inline uint32_t DerivationExpression_ExpressionFixedBinary::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.variation_pointer) return _internal_variation_pointer(); } -inline void DerivationExpression_ExpressionFixedChar::_internal_set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionFixedBinary::_internal_set_variation_pointer(uint32_t value) { variation_pointer_ = value; } -inline void DerivationExpression_ExpressionFixedChar::set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionFixedBinary::set_variation_pointer(uint32_t value) { _internal_set_variation_pointer(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedChar.variation_pointer) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedBinary.variation_pointer) } // .substrait.Type.Nullability nullability = 3; -inline void DerivationExpression_ExpressionFixedChar::clear_nullability() { +inline void DerivationExpression_ExpressionFixedBinary::clear_nullability() { nullability_ = 0; } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedChar::_internal_nullability() const { +inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedBinary::_internal_nullability() const { return static_cast< ::substrait::Type_Nullability >(nullability_); } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedChar::nullability() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedChar.nullability) +inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedBinary::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.nullability) return _internal_nullability(); } -inline void DerivationExpression_ExpressionFixedChar::_internal_set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionFixedBinary::_internal_set_nullability(::substrait::Type_Nullability value) { nullability_ = value; } -inline void DerivationExpression_ExpressionFixedChar::set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionFixedBinary::set_nullability(::substrait::Type_Nullability value) { _internal_set_nullability(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedChar.nullability) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedBinary.nullability) } // ------------------------------------------------------------------- -// DerivationExpression_ExpressionVarChar +// DerivationExpression_ExpressionDecimal -// .substrait.DerivationExpression length = 1; -inline bool DerivationExpression_ExpressionVarChar::_internal_has_length() const { - return this != internal_default_instance() && length_ != nullptr; +// .substrait.DerivationExpression scale = 1; +inline bool DerivationExpression_ExpressionDecimal::_internal_has_scale() const { + return this != internal_default_instance() && scale_ != nullptr; } -inline bool DerivationExpression_ExpressionVarChar::has_length() const { - return _internal_has_length(); +inline bool DerivationExpression_ExpressionDecimal::has_scale() const { + return _internal_has_scale(); } -inline void DerivationExpression_ExpressionVarChar::clear_length() { - if (GetArenaForAllocation() == nullptr && length_ != nullptr) { - delete length_; +inline void DerivationExpression_ExpressionDecimal::clear_scale() { + if (GetArenaForAllocation() == nullptr && scale_ != nullptr) { + delete scale_; } - length_ = nullptr; + scale_ = nullptr; } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionVarChar::_internal_length() const { - const ::substrait::DerivationExpression* p = length_; +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::_internal_scale() const { + const ::substrait::DerivationExpression* p = scale_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_DerivationExpression_default_instance_); } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionVarChar::length() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.length) - return _internal_length(); +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::scale() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.scale) + return _internal_scale(); } -inline void DerivationExpression_ExpressionVarChar::unsafe_arena_set_allocated_length( - ::substrait::DerivationExpression* length) { +inline void DerivationExpression_ExpressionDecimal::unsafe_arena_set_allocated_scale( + ::substrait::DerivationExpression* scale) { if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(length_); + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(scale_); } - length_ = length; - if (length) { + scale_ = scale; + if (scale) { + + } else { + + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionDecimal.scale) +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::release_scale() { + + ::substrait::DerivationExpression* temp = scale_; + scale_ = nullptr; +#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE + auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + if (GetArenaForAllocation() == nullptr) { delete old; } +#else // PROTOBUF_FORCE_COPY_IN_RELEASE + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } +#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::unsafe_arena_release_scale() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionDecimal.scale) + + ::substrait::DerivationExpression* temp = scale_; + scale_ = nullptr; + return temp; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::_internal_mutable_scale() { + + if (scale_ == nullptr) { + auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); + scale_ = p; + } + return scale_; +} +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::mutable_scale() { + ::substrait::DerivationExpression* _msg = _internal_mutable_scale(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionDecimal.scale) + return _msg; +} +inline void DerivationExpression_ExpressionDecimal::set_allocated_scale(::substrait::DerivationExpression* scale) { + ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); + if (message_arena == nullptr) { + delete scale_; + } + if (scale) { + ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(scale); + if (message_arena != submessage_arena) { + scale = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, scale, submessage_arena); + } + + } else { + + } + scale_ = scale; + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionDecimal.scale) +} + +// .substrait.DerivationExpression precision = 2; +inline bool DerivationExpression_ExpressionDecimal::_internal_has_precision() const { + return this != internal_default_instance() && precision_ != nullptr; +} +inline bool DerivationExpression_ExpressionDecimal::has_precision() const { + return _internal_has_precision(); +} +inline void DerivationExpression_ExpressionDecimal::clear_precision() { + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; + } + precision_ = nullptr; +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::_internal_precision() const { + const ::substrait::DerivationExpression* p = precision_; + return p != nullptr ? *p : reinterpret_cast( + ::substrait::_DerivationExpression_default_instance_); +} +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::precision() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.precision) + return _internal_precision(); +} +inline void DerivationExpression_ExpressionDecimal::unsafe_arena_set_allocated_precision( + ::substrait::DerivationExpression* precision) { + if (GetArenaForAllocation() == nullptr) { + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); + } + precision_ = precision; + if (precision) { } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionVarChar.length) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionDecimal.precision) } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::release_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::release_precision() { - ::substrait::DerivationExpression* temp = length_; - length_ = nullptr; + ::substrait::DerivationExpression* temp = precision_; + precision_ = nullptr; #ifdef PROTOBUF_FORCE_COPY_IN_RELEASE auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); @@ -4248,129 +4992,129 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::unsafe_arena_release_length() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionVarChar.length) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionDecimal.precision) - ::substrait::DerivationExpression* temp = length_; - length_ = nullptr; + ::substrait::DerivationExpression* temp = precision_; + precision_ = nullptr; return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::_internal_mutable_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::_internal_mutable_precision() { - if (length_ == nullptr) { + if (precision_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); - length_ = p; + precision_ = p; } - return length_; + return precision_; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionVarChar::mutable_length() { - ::substrait::DerivationExpression* _msg = _internal_mutable_length(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionVarChar.length) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::mutable_precision() { + ::substrait::DerivationExpression* _msg = _internal_mutable_precision(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionDecimal.precision) return _msg; } -inline void DerivationExpression_ExpressionVarChar::set_allocated_length(::substrait::DerivationExpression* length) { +inline void DerivationExpression_ExpressionDecimal::set_allocated_precision(::substrait::DerivationExpression* precision) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { - delete length_; + delete precision_; } - if (length) { + if (precision) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(length); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(precision); if (message_arena != submessage_arena) { - length = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, length, submessage_arena); + precision = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision, submessage_arena); } } else { } - length_ = length; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionVarChar.length) + precision_ = precision; + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionDecimal.precision) } -// uint32 variation_pointer = 2; -inline void DerivationExpression_ExpressionVarChar::clear_variation_pointer() { +// uint32 variation_pointer = 3; +inline void DerivationExpression_ExpressionDecimal::clear_variation_pointer() { variation_pointer_ = 0u; } -inline uint32_t DerivationExpression_ExpressionVarChar::_internal_variation_pointer() const { +inline uint32_t DerivationExpression_ExpressionDecimal::_internal_variation_pointer() const { return variation_pointer_; } -inline uint32_t DerivationExpression_ExpressionVarChar::variation_pointer() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.variation_pointer) +inline uint32_t DerivationExpression_ExpressionDecimal::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.variation_pointer) return _internal_variation_pointer(); } -inline void DerivationExpression_ExpressionVarChar::_internal_set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionDecimal::_internal_set_variation_pointer(uint32_t value) { variation_pointer_ = value; } -inline void DerivationExpression_ExpressionVarChar::set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionDecimal::set_variation_pointer(uint32_t value) { _internal_set_variation_pointer(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionVarChar.variation_pointer) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionDecimal.variation_pointer) } -// .substrait.Type.Nullability nullability = 3; -inline void DerivationExpression_ExpressionVarChar::clear_nullability() { +// .substrait.Type.Nullability nullability = 4; +inline void DerivationExpression_ExpressionDecimal::clear_nullability() { nullability_ = 0; } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionVarChar::_internal_nullability() const { +inline ::substrait::Type_Nullability DerivationExpression_ExpressionDecimal::_internal_nullability() const { return static_cast< ::substrait::Type_Nullability >(nullability_); } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionVarChar::nullability() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionVarChar.nullability) +inline ::substrait::Type_Nullability DerivationExpression_ExpressionDecimal::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.nullability) return _internal_nullability(); } -inline void DerivationExpression_ExpressionVarChar::_internal_set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionDecimal::_internal_set_nullability(::substrait::Type_Nullability value) { nullability_ = value; } -inline void DerivationExpression_ExpressionVarChar::set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionDecimal::set_nullability(::substrait::Type_Nullability value) { _internal_set_nullability(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionVarChar.nullability) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionDecimal.nullability) } // ------------------------------------------------------------------- -// DerivationExpression_ExpressionFixedBinary +// DerivationExpression_ExpressionPrecisionTimestamp -// .substrait.DerivationExpression length = 1; -inline bool DerivationExpression_ExpressionFixedBinary::_internal_has_length() const { - return this != internal_default_instance() && length_ != nullptr; +// .substrait.DerivationExpression precision = 1; +inline bool DerivationExpression_ExpressionPrecisionTimestamp::_internal_has_precision() const { + return this != internal_default_instance() && precision_ != nullptr; } -inline bool DerivationExpression_ExpressionFixedBinary::has_length() const { - return _internal_has_length(); +inline bool DerivationExpression_ExpressionPrecisionTimestamp::has_precision() const { + return _internal_has_precision(); } -inline void DerivationExpression_ExpressionFixedBinary::clear_length() { - if (GetArenaForAllocation() == nullptr && length_ != nullptr) { - delete length_; +inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_precision() { + if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { + delete precision_; } - length_ = nullptr; + precision_ = nullptr; } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedBinary::_internal_length() const { - const ::substrait::DerivationExpression* p = length_; +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionPrecisionTimestamp::_internal_precision() const { + const ::substrait::DerivationExpression* p = precision_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_DerivationExpression_default_instance_); } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionFixedBinary::length() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.length) - return _internal_length(); +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionPrecisionTimestamp::precision() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) + return _internal_precision(); } -inline void DerivationExpression_ExpressionFixedBinary::unsafe_arena_set_allocated_length( - ::substrait::DerivationExpression* length) { +inline void DerivationExpression_ExpressionPrecisionTimestamp::unsafe_arena_set_allocated_precision( + ::substrait::DerivationExpression* precision) { if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(length_); + delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); } - length_ = length; - if (length) { + precision_ = precision; + if (precision) { } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionFixedBinary.length) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::release_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::release_precision() { - ::substrait::DerivationExpression* temp = length_; - length_ = nullptr; + ::substrait::DerivationExpression* temp = precision_; + precision_ = nullptr; #ifdef PROTOBUF_FORCE_COPY_IN_RELEASE auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); @@ -4382,203 +5126,113 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBi #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::unsafe_arena_release_length() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionFixedBinary.length) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) - ::substrait::DerivationExpression* temp = length_; - length_ = nullptr; + ::substrait::DerivationExpression* temp = precision_; + precision_ = nullptr; return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::_internal_mutable_length() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::_internal_mutable_precision() { - if (length_ == nullptr) { + if (precision_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); - length_ = p; + precision_ = p; } - return length_; + return precision_; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionFixedBinary::mutable_length() { - ::substrait::DerivationExpression* _msg = _internal_mutable_length(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionFixedBinary.length) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::mutable_precision() { + ::substrait::DerivationExpression* _msg = _internal_mutable_precision(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) return _msg; } -inline void DerivationExpression_ExpressionFixedBinary::set_allocated_length(::substrait::DerivationExpression* length) { +inline void DerivationExpression_ExpressionPrecisionTimestamp::set_allocated_precision(::substrait::DerivationExpression* precision) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { - delete length_; + delete precision_; } - if (length) { + if (precision) { ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(length); + ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(precision); if (message_arena != submessage_arena) { - length = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, length, submessage_arena); + precision = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( + message_arena, precision, submessage_arena); } } else { } - length_ = length; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionFixedBinary.length) + precision_ = precision; + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) } // uint32 variation_pointer = 2; -inline void DerivationExpression_ExpressionFixedBinary::clear_variation_pointer() { +inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_variation_pointer() { variation_pointer_ = 0u; } -inline uint32_t DerivationExpression_ExpressionFixedBinary::_internal_variation_pointer() const { +inline uint32_t DerivationExpression_ExpressionPrecisionTimestamp::_internal_variation_pointer() const { return variation_pointer_; } -inline uint32_t DerivationExpression_ExpressionFixedBinary::variation_pointer() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.variation_pointer) +inline uint32_t DerivationExpression_ExpressionPrecisionTimestamp::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.variation_pointer) return _internal_variation_pointer(); } -inline void DerivationExpression_ExpressionFixedBinary::_internal_set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionPrecisionTimestamp::_internal_set_variation_pointer(uint32_t value) { variation_pointer_ = value; } -inline void DerivationExpression_ExpressionFixedBinary::set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionPrecisionTimestamp::set_variation_pointer(uint32_t value) { _internal_set_variation_pointer(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedBinary.variation_pointer) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionPrecisionTimestamp.variation_pointer) } // .substrait.Type.Nullability nullability = 3; -inline void DerivationExpression_ExpressionFixedBinary::clear_nullability() { +inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_nullability() { nullability_ = 0; } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedBinary::_internal_nullability() const { +inline ::substrait::Type_Nullability DerivationExpression_ExpressionPrecisionTimestamp::_internal_nullability() const { return static_cast< ::substrait::Type_Nullability >(nullability_); } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionFixedBinary::nullability() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionFixedBinary.nullability) - return _internal_nullability(); -} -inline void DerivationExpression_ExpressionFixedBinary::_internal_set_nullability(::substrait::Type_Nullability value) { - - nullability_ = value; -} -inline void DerivationExpression_ExpressionFixedBinary::set_nullability(::substrait::Type_Nullability value) { - _internal_set_nullability(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionFixedBinary.nullability) -} - -// ------------------------------------------------------------------- - -// DerivationExpression_ExpressionDecimal - -// .substrait.DerivationExpression scale = 1; -inline bool DerivationExpression_ExpressionDecimal::_internal_has_scale() const { - return this != internal_default_instance() && scale_ != nullptr; -} -inline bool DerivationExpression_ExpressionDecimal::has_scale() const { - return _internal_has_scale(); -} -inline void DerivationExpression_ExpressionDecimal::clear_scale() { - if (GetArenaForAllocation() == nullptr && scale_ != nullptr) { - delete scale_; - } - scale_ = nullptr; -} -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::_internal_scale() const { - const ::substrait::DerivationExpression* p = scale_; - return p != nullptr ? *p : reinterpret_cast( - ::substrait::_DerivationExpression_default_instance_); -} -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::scale() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.scale) - return _internal_scale(); -} -inline void DerivationExpression_ExpressionDecimal::unsafe_arena_set_allocated_scale( - ::substrait::DerivationExpression* scale) { - if (GetArenaForAllocation() == nullptr) { - delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(scale_); - } - scale_ = scale; - if (scale) { - - } else { - - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionDecimal.scale) -} -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::release_scale() { - - ::substrait::DerivationExpression* temp = scale_; - scale_ = nullptr; -#ifdef PROTOBUF_FORCE_COPY_IN_RELEASE - auto* old = reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(temp); - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - if (GetArenaForAllocation() == nullptr) { delete old; } -#else // PROTOBUF_FORCE_COPY_IN_RELEASE - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } -#endif // !PROTOBUF_FORCE_COPY_IN_RELEASE - return temp; -} -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::unsafe_arena_release_scale() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionDecimal.scale) - - ::substrait::DerivationExpression* temp = scale_; - scale_ = nullptr; - return temp; +inline ::substrait::Type_Nullability DerivationExpression_ExpressionPrecisionTimestamp::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.nullability) + return _internal_nullability(); } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::_internal_mutable_scale() { +inline void DerivationExpression_ExpressionPrecisionTimestamp::_internal_set_nullability(::substrait::Type_Nullability value) { - if (scale_ == nullptr) { - auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); - scale_ = p; - } - return scale_; -} -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::mutable_scale() { - ::substrait::DerivationExpression* _msg = _internal_mutable_scale(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionDecimal.scale) - return _msg; + nullability_ = value; } -inline void DerivationExpression_ExpressionDecimal::set_allocated_scale(::substrait::DerivationExpression* scale) { - ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); - if (message_arena == nullptr) { - delete scale_; - } - if (scale) { - ::PROTOBUF_NAMESPACE_ID::Arena* submessage_arena = - ::PROTOBUF_NAMESPACE_ID::Arena::InternalHelper<::substrait::DerivationExpression>::GetOwningArena(scale); - if (message_arena != submessage_arena) { - scale = ::PROTOBUF_NAMESPACE_ID::internal::GetOwnedMessage( - message_arena, scale, submessage_arena); - } - - } else { - - } - scale_ = scale; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionDecimal.scale) +inline void DerivationExpression_ExpressionPrecisionTimestamp::set_nullability(::substrait::Type_Nullability value) { + _internal_set_nullability(value); + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionPrecisionTimestamp.nullability) } -// .substrait.DerivationExpression precision = 2; -inline bool DerivationExpression_ExpressionDecimal::_internal_has_precision() const { +// ------------------------------------------------------------------- + +// DerivationExpression_ExpressionIntervalDay + +// .substrait.DerivationExpression precision = 1; +inline bool DerivationExpression_ExpressionIntervalDay::_internal_has_precision() const { return this != internal_default_instance() && precision_ != nullptr; } -inline bool DerivationExpression_ExpressionDecimal::has_precision() const { +inline bool DerivationExpression_ExpressionIntervalDay::has_precision() const { return _internal_has_precision(); } -inline void DerivationExpression_ExpressionDecimal::clear_precision() { +inline void DerivationExpression_ExpressionIntervalDay::clear_precision() { if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { delete precision_; } precision_ = nullptr; } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::_internal_precision() const { +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionIntervalDay::_internal_precision() const { const ::substrait::DerivationExpression* p = precision_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_DerivationExpression_default_instance_); } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionDecimal::precision() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.precision) +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionIntervalDay::precision() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalDay.precision) return _internal_precision(); } -inline void DerivationExpression_ExpressionDecimal::unsafe_arena_set_allocated_precision( +inline void DerivationExpression_ExpressionIntervalDay::unsafe_arena_set_allocated_precision( ::substrait::DerivationExpression* precision) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); @@ -4589,9 +5243,9 @@ inline void DerivationExpression_ExpressionDecimal::unsafe_arena_set_allocated_p } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionDecimal.precision) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionIntervalDay.precision) } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::release_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalDay::release_precision() { ::substrait::DerivationExpression* temp = precision_; precision_ = nullptr; @@ -4606,14 +5260,14 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::unsafe_arena_release_precision() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionDecimal.precision) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalDay::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionIntervalDay.precision) ::substrait::DerivationExpression* temp = precision_; precision_ = nullptr; return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::_internal_mutable_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalDay::_internal_mutable_precision() { if (precision_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); @@ -4621,12 +5275,12 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal } return precision_; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionDecimal::mutable_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalDay::mutable_precision() { ::substrait::DerivationExpression* _msg = _internal_mutable_precision(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionDecimal.precision) + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionIntervalDay.precision) return _msg; } -inline void DerivationExpression_ExpressionDecimal::set_allocated_precision(::substrait::DerivationExpression* precision) { +inline void DerivationExpression_ExpressionIntervalDay::set_allocated_precision(::substrait::DerivationExpression* precision) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete precision_; @@ -4643,76 +5297,76 @@ inline void DerivationExpression_ExpressionDecimal::set_allocated_precision(::su } precision_ = precision; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionDecimal.precision) + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionIntervalDay.precision) } -// uint32 variation_pointer = 3; -inline void DerivationExpression_ExpressionDecimal::clear_variation_pointer() { +// uint32 variation_pointer = 2; +inline void DerivationExpression_ExpressionIntervalDay::clear_variation_pointer() { variation_pointer_ = 0u; } -inline uint32_t DerivationExpression_ExpressionDecimal::_internal_variation_pointer() const { +inline uint32_t DerivationExpression_ExpressionIntervalDay::_internal_variation_pointer() const { return variation_pointer_; } -inline uint32_t DerivationExpression_ExpressionDecimal::variation_pointer() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.variation_pointer) +inline uint32_t DerivationExpression_ExpressionIntervalDay::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalDay.variation_pointer) return _internal_variation_pointer(); } -inline void DerivationExpression_ExpressionDecimal::_internal_set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionIntervalDay::_internal_set_variation_pointer(uint32_t value) { variation_pointer_ = value; } -inline void DerivationExpression_ExpressionDecimal::set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionIntervalDay::set_variation_pointer(uint32_t value) { _internal_set_variation_pointer(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionDecimal.variation_pointer) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionIntervalDay.variation_pointer) } -// .substrait.Type.Nullability nullability = 4; -inline void DerivationExpression_ExpressionDecimal::clear_nullability() { +// .substrait.Type.Nullability nullability = 3; +inline void DerivationExpression_ExpressionIntervalDay::clear_nullability() { nullability_ = 0; } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionDecimal::_internal_nullability() const { +inline ::substrait::Type_Nullability DerivationExpression_ExpressionIntervalDay::_internal_nullability() const { return static_cast< ::substrait::Type_Nullability >(nullability_); } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionDecimal::nullability() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionDecimal.nullability) +inline ::substrait::Type_Nullability DerivationExpression_ExpressionIntervalDay::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalDay.nullability) return _internal_nullability(); } -inline void DerivationExpression_ExpressionDecimal::_internal_set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionIntervalDay::_internal_set_nullability(::substrait::Type_Nullability value) { nullability_ = value; } -inline void DerivationExpression_ExpressionDecimal::set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionIntervalDay::set_nullability(::substrait::Type_Nullability value) { _internal_set_nullability(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionDecimal.nullability) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionIntervalDay.nullability) } // ------------------------------------------------------------------- -// DerivationExpression_ExpressionPrecisionTimestamp +// DerivationExpression_ExpressionIntervalCompound // .substrait.DerivationExpression precision = 1; -inline bool DerivationExpression_ExpressionPrecisionTimestamp::_internal_has_precision() const { +inline bool DerivationExpression_ExpressionIntervalCompound::_internal_has_precision() const { return this != internal_default_instance() && precision_ != nullptr; } -inline bool DerivationExpression_ExpressionPrecisionTimestamp::has_precision() const { +inline bool DerivationExpression_ExpressionIntervalCompound::has_precision() const { return _internal_has_precision(); } -inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_precision() { +inline void DerivationExpression_ExpressionIntervalCompound::clear_precision() { if (GetArenaForAllocation() == nullptr && precision_ != nullptr) { delete precision_; } precision_ = nullptr; } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionPrecisionTimestamp::_internal_precision() const { +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionIntervalCompound::_internal_precision() const { const ::substrait::DerivationExpression* p = precision_; return p != nullptr ? *p : reinterpret_cast( ::substrait::_DerivationExpression_default_instance_); } -inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionPrecisionTimestamp::precision() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) +inline const ::substrait::DerivationExpression& DerivationExpression_ExpressionIntervalCompound::precision() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalCompound.precision) return _internal_precision(); } -inline void DerivationExpression_ExpressionPrecisionTimestamp::unsafe_arena_set_allocated_precision( +inline void DerivationExpression_ExpressionIntervalCompound::unsafe_arena_set_allocated_precision( ::substrait::DerivationExpression* precision) { if (GetArenaForAllocation() == nullptr) { delete reinterpret_cast<::PROTOBUF_NAMESPACE_ID::MessageLite*>(precision_); @@ -4723,9 +5377,9 @@ inline void DerivationExpression_ExpressionPrecisionTimestamp::unsafe_arena_set_ } else { } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.ExpressionIntervalCompound.precision) } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::release_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalCompound::release_precision() { ::substrait::DerivationExpression* temp = precision_; precision_ = nullptr; @@ -4740,14 +5394,14 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisi #endif // !PROTOBUF_FORCE_COPY_IN_RELEASE return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::unsafe_arena_release_precision() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalCompound::unsafe_arena_release_precision() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.ExpressionIntervalCompound.precision) ::substrait::DerivationExpression* temp = precision_; precision_ = nullptr; return temp; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::_internal_mutable_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalCompound::_internal_mutable_precision() { if (precision_ == nullptr) { auto* p = CreateMaybeMessage<::substrait::DerivationExpression>(GetArenaForAllocation()); @@ -4755,12 +5409,12 @@ inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisi } return precision_; } -inline ::substrait::DerivationExpression* DerivationExpression_ExpressionPrecisionTimestamp::mutable_precision() { +inline ::substrait::DerivationExpression* DerivationExpression_ExpressionIntervalCompound::mutable_precision() { ::substrait::DerivationExpression* _msg = _internal_mutable_precision(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.ExpressionIntervalCompound.precision) return _msg; } -inline void DerivationExpression_ExpressionPrecisionTimestamp::set_allocated_precision(::substrait::DerivationExpression* precision) { +inline void DerivationExpression_ExpressionIntervalCompound::set_allocated_precision(::substrait::DerivationExpression* precision) { ::PROTOBUF_NAMESPACE_ID::Arena* message_arena = GetArenaForAllocation(); if (message_arena == nullptr) { delete precision_; @@ -4777,47 +5431,47 @@ inline void DerivationExpression_ExpressionPrecisionTimestamp::set_allocated_pre } precision_ = precision; - // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionPrecisionTimestamp.precision) + // @@protoc_insertion_point(field_set_allocated:substrait.DerivationExpression.ExpressionIntervalCompound.precision) } // uint32 variation_pointer = 2; -inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_variation_pointer() { +inline void DerivationExpression_ExpressionIntervalCompound::clear_variation_pointer() { variation_pointer_ = 0u; } -inline uint32_t DerivationExpression_ExpressionPrecisionTimestamp::_internal_variation_pointer() const { +inline uint32_t DerivationExpression_ExpressionIntervalCompound::_internal_variation_pointer() const { return variation_pointer_; } -inline uint32_t DerivationExpression_ExpressionPrecisionTimestamp::variation_pointer() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.variation_pointer) +inline uint32_t DerivationExpression_ExpressionIntervalCompound::variation_pointer() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalCompound.variation_pointer) return _internal_variation_pointer(); } -inline void DerivationExpression_ExpressionPrecisionTimestamp::_internal_set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionIntervalCompound::_internal_set_variation_pointer(uint32_t value) { variation_pointer_ = value; } -inline void DerivationExpression_ExpressionPrecisionTimestamp::set_variation_pointer(uint32_t value) { +inline void DerivationExpression_ExpressionIntervalCompound::set_variation_pointer(uint32_t value) { _internal_set_variation_pointer(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionPrecisionTimestamp.variation_pointer) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionIntervalCompound.variation_pointer) } // .substrait.Type.Nullability nullability = 3; -inline void DerivationExpression_ExpressionPrecisionTimestamp::clear_nullability() { +inline void DerivationExpression_ExpressionIntervalCompound::clear_nullability() { nullability_ = 0; } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionPrecisionTimestamp::_internal_nullability() const { +inline ::substrait::Type_Nullability DerivationExpression_ExpressionIntervalCompound::_internal_nullability() const { return static_cast< ::substrait::Type_Nullability >(nullability_); } -inline ::substrait::Type_Nullability DerivationExpression_ExpressionPrecisionTimestamp::nullability() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionPrecisionTimestamp.nullability) +inline ::substrait::Type_Nullability DerivationExpression_ExpressionIntervalCompound::nullability() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.ExpressionIntervalCompound.nullability) return _internal_nullability(); } -inline void DerivationExpression_ExpressionPrecisionTimestamp::_internal_set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionIntervalCompound::_internal_set_nullability(::substrait::Type_Nullability value) { nullability_ = value; } -inline void DerivationExpression_ExpressionPrecisionTimestamp::set_nullability(::substrait::Type_Nullability value) { +inline void DerivationExpression_ExpressionIntervalCompound::set_nullability(::substrait::Type_Nullability value) { _internal_set_nullability(value); - // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionPrecisionTimestamp.nullability) + // @@protoc_insertion_point(field_set:substrait.DerivationExpression.ExpressionIntervalCompound.nullability) } // ------------------------------------------------------------------- @@ -7362,72 +8016,6 @@ inline ::substrait::Type_IntervalYear* DerivationExpression::mutable_interval_ye return _msg; } -// .substrait.Type.IntervalDay interval_day = 20; -inline bool DerivationExpression::_internal_has_interval_day() const { - return kind_case() == kIntervalDay; -} -inline bool DerivationExpression::has_interval_day() const { - return _internal_has_interval_day(); -} -inline void DerivationExpression::set_has_interval_day() { - _oneof_case_[0] = kIntervalDay; -} -inline ::substrait::Type_IntervalDay* DerivationExpression::release_interval_day() { - // @@protoc_insertion_point(field_release:substrait.DerivationExpression.interval_day) - if (_internal_has_interval_day()) { - clear_has_kind(); - ::substrait::Type_IntervalDay* temp = kind_.interval_day_; - if (GetArenaForAllocation() != nullptr) { - temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); - } - kind_.interval_day_ = nullptr; - return temp; - } else { - return nullptr; - } -} -inline const ::substrait::Type_IntervalDay& DerivationExpression::_internal_interval_day() const { - return _internal_has_interval_day() - ? *kind_.interval_day_ - : reinterpret_cast< ::substrait::Type_IntervalDay&>(::substrait::_Type_IntervalDay_default_instance_); -} -inline const ::substrait::Type_IntervalDay& DerivationExpression::interval_day() const { - // @@protoc_insertion_point(field_get:substrait.DerivationExpression.interval_day) - return _internal_interval_day(); -} -inline ::substrait::Type_IntervalDay* DerivationExpression::unsafe_arena_release_interval_day() { - // @@protoc_insertion_point(field_unsafe_arena_release:substrait.DerivationExpression.interval_day) - if (_internal_has_interval_day()) { - clear_has_kind(); - ::substrait::Type_IntervalDay* temp = kind_.interval_day_; - kind_.interval_day_ = nullptr; - return temp; - } else { - return nullptr; - } -} -inline void DerivationExpression::unsafe_arena_set_allocated_interval_day(::substrait::Type_IntervalDay* interval_day) { - clear_kind(); - if (interval_day) { - set_has_interval_day(); - kind_.interval_day_ = interval_day; - } - // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.interval_day) -} -inline ::substrait::Type_IntervalDay* DerivationExpression::_internal_mutable_interval_day() { - if (!_internal_has_interval_day()) { - clear_kind(); - set_has_interval_day(); - kind_.interval_day_ = CreateMaybeMessage< ::substrait::Type_IntervalDay >(GetArenaForAllocation()); - } - return kind_.interval_day_; -} -inline ::substrait::Type_IntervalDay* DerivationExpression::mutable_interval_day() { - ::substrait::Type_IntervalDay* _msg = _internal_mutable_interval_day(); - // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.interval_day) - return _msg; -} - // .substrait.Type.TimestampTZ timestamp_tz = 29 [deprecated = true]; inline bool DerivationExpression::_internal_has_timestamp_tz() const { return kind_case() == kTimestampTz; @@ -7560,6 +8148,154 @@ inline ::substrait::Type_UUID* DerivationExpression::mutable_uuid() { return _msg; } +// .substrait.DerivationExpression.ExpressionIntervalDay interval_day = 20; +inline bool DerivationExpression::_internal_has_interval_day() const { + return kind_case() == kIntervalDay; +} +inline bool DerivationExpression::has_interval_day() const { + return _internal_has_interval_day(); +} +inline void DerivationExpression::set_has_interval_day() { + _oneof_case_[0] = kIntervalDay; +} +inline void DerivationExpression::clear_interval_day() { + if (_internal_has_interval_day()) { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_day_; + } + clear_has_kind(); + } +} +inline ::substrait::DerivationExpression_ExpressionIntervalDay* DerivationExpression::release_interval_day() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.interval_day) + if (_internal_has_interval_day()) { + clear_has_kind(); + ::substrait::DerivationExpression_ExpressionIntervalDay* temp = kind_.interval_day_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + kind_.interval_day_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::DerivationExpression_ExpressionIntervalDay& DerivationExpression::_internal_interval_day() const { + return _internal_has_interval_day() + ? *kind_.interval_day_ + : reinterpret_cast< ::substrait::DerivationExpression_ExpressionIntervalDay&>(::substrait::_DerivationExpression_ExpressionIntervalDay_default_instance_); +} +inline const ::substrait::DerivationExpression_ExpressionIntervalDay& DerivationExpression::interval_day() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.interval_day) + return _internal_interval_day(); +} +inline ::substrait::DerivationExpression_ExpressionIntervalDay* DerivationExpression::unsafe_arena_release_interval_day() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.DerivationExpression.interval_day) + if (_internal_has_interval_day()) { + clear_has_kind(); + ::substrait::DerivationExpression_ExpressionIntervalDay* temp = kind_.interval_day_; + kind_.interval_day_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void DerivationExpression::unsafe_arena_set_allocated_interval_day(::substrait::DerivationExpression_ExpressionIntervalDay* interval_day) { + clear_kind(); + if (interval_day) { + set_has_interval_day(); + kind_.interval_day_ = interval_day; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.interval_day) +} +inline ::substrait::DerivationExpression_ExpressionIntervalDay* DerivationExpression::_internal_mutable_interval_day() { + if (!_internal_has_interval_day()) { + clear_kind(); + set_has_interval_day(); + kind_.interval_day_ = CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionIntervalDay >(GetArenaForAllocation()); + } + return kind_.interval_day_; +} +inline ::substrait::DerivationExpression_ExpressionIntervalDay* DerivationExpression::mutable_interval_day() { + ::substrait::DerivationExpression_ExpressionIntervalDay* _msg = _internal_mutable_interval_day(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.interval_day) + return _msg; +} + +// .substrait.DerivationExpression.ExpressionIntervalCompound interval_compound = 42; +inline bool DerivationExpression::_internal_has_interval_compound() const { + return kind_case() == kIntervalCompound; +} +inline bool DerivationExpression::has_interval_compound() const { + return _internal_has_interval_compound(); +} +inline void DerivationExpression::set_has_interval_compound() { + _oneof_case_[0] = kIntervalCompound; +} +inline void DerivationExpression::clear_interval_compound() { + if (_internal_has_interval_compound()) { + if (GetArenaForAllocation() == nullptr) { + delete kind_.interval_compound_; + } + clear_has_kind(); + } +} +inline ::substrait::DerivationExpression_ExpressionIntervalCompound* DerivationExpression::release_interval_compound() { + // @@protoc_insertion_point(field_release:substrait.DerivationExpression.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::DerivationExpression_ExpressionIntervalCompound* temp = kind_.interval_compound_; + if (GetArenaForAllocation() != nullptr) { + temp = ::PROTOBUF_NAMESPACE_ID::internal::DuplicateIfNonNull(temp); + } + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline const ::substrait::DerivationExpression_ExpressionIntervalCompound& DerivationExpression::_internal_interval_compound() const { + return _internal_has_interval_compound() + ? *kind_.interval_compound_ + : reinterpret_cast< ::substrait::DerivationExpression_ExpressionIntervalCompound&>(::substrait::_DerivationExpression_ExpressionIntervalCompound_default_instance_); +} +inline const ::substrait::DerivationExpression_ExpressionIntervalCompound& DerivationExpression::interval_compound() const { + // @@protoc_insertion_point(field_get:substrait.DerivationExpression.interval_compound) + return _internal_interval_compound(); +} +inline ::substrait::DerivationExpression_ExpressionIntervalCompound* DerivationExpression::unsafe_arena_release_interval_compound() { + // @@protoc_insertion_point(field_unsafe_arena_release:substrait.DerivationExpression.interval_compound) + if (_internal_has_interval_compound()) { + clear_has_kind(); + ::substrait::DerivationExpression_ExpressionIntervalCompound* temp = kind_.interval_compound_; + kind_.interval_compound_ = nullptr; + return temp; + } else { + return nullptr; + } +} +inline void DerivationExpression::unsafe_arena_set_allocated_interval_compound(::substrait::DerivationExpression_ExpressionIntervalCompound* interval_compound) { + clear_kind(); + if (interval_compound) { + set_has_interval_compound(); + kind_.interval_compound_ = interval_compound; + } + // @@protoc_insertion_point(field_unsafe_arena_set_allocated:substrait.DerivationExpression.interval_compound) +} +inline ::substrait::DerivationExpression_ExpressionIntervalCompound* DerivationExpression::_internal_mutable_interval_compound() { + if (!_internal_has_interval_compound()) { + clear_kind(); + set_has_interval_compound(); + kind_.interval_compound_ = CreateMaybeMessage< ::substrait::DerivationExpression_ExpressionIntervalCompound >(GetArenaForAllocation()); + } + return kind_.interval_compound_; +} +inline ::substrait::DerivationExpression_ExpressionIntervalCompound* DerivationExpression::mutable_interval_compound() { + ::substrait::DerivationExpression_ExpressionIntervalCompound* _msg = _internal_mutable_interval_compound(); + // @@protoc_insertion_point(field_mutable:substrait.DerivationExpression.interval_compound) + return _msg; +} + // .substrait.DerivationExpression.ExpressionFixedChar fixed_char = 21; inline bool DerivationExpression::_internal_has_fixed_char() const { return kind_case() == kFixedChar; @@ -8880,6 +9616,10 @@ inline DerivationExpression::KindCase DerivationExpression::kind_case() const { // ------------------------------------------------------------------- +// ------------------------------------------------------------------- + +// ------------------------------------------------------------------- + // @@protoc_insertion_point(namespace_scope)