From 41de996e965c9730003ce76ac3d12ce6ed8936c2 Mon Sep 17 00:00:00 2001 From: Deepak Majeti Date: Thu, 10 Nov 2022 21:31:53 -0500 Subject: [PATCH] [native] Add support for TPC-H Connector Protocol Bindings The protocol bindings for TPC-H Connector were manually added in PR #18367. These bindings will now be generated automatically. The following improvements have been made: 1. Improve README of presto_cpp/presto_protocol. 2. Add default make target to generate Presto protocol files. 3. Include TpchConnector classes in the yml file. 4. Extend special Connector files to support tpch. 5. Customize TpchTransactionHandle since the corresponding class in Java is an enum. 6. Customize TpchColumnHandle since we require an implementation of operator<(). --- .../presto_cpp/presto_protocol/Makefile | 1 + .../presto_cpp/presto_protocol/README.md | 10 +- .../presto_protocol/presto_protocol.cpp | 17969 ++++++++-------- .../presto_protocol/presto_protocol.h | 3309 +-- .../presto_protocol/presto_protocol.json | 8300 +++---- .../presto_protocol/presto_protocol.yml | 10 + .../special/ColumnHandle.cpp.inc | 14 +- .../ConnectorPartitioningHandle.cpp.inc | 10 + .../special/ConnectorSplit.cpp.inc | 12 + .../special/ConnectorTableHandle.cpp.inc | 12 + .../ConnectorTableLayoutHandle.cpp.inc | 12 + .../ConnectorTransactionHandle.cpp.inc | 20 +- .../special/TpchColumnHandle.hpp.inc | 31 + .../special/TpchTransactionHandle.cpp.inc | 30 + .../special/TpchTransactionHandle.hpp.inc | 28 + 15 files changed, 15105 insertions(+), 14663 deletions(-) create mode 100644 presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc create mode 100644 presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.cpp.inc create mode 100644 presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.hpp.inc diff --git a/presto-native-execution/presto_cpp/presto_protocol/Makefile b/presto-native-execution/presto_cpp/presto_protocol/Makefile index 2283406344623..56aa96f576eea 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/Makefile +++ b/presto-native-execution/presto_cpp/presto_protocol/Makefile @@ -11,6 +11,7 @@ # limitations under the License. .PHONY: presto_protocol presto_protocol-json presto_protocol-cpp +default: presto_protocol presto_protocol: presto_protocol-cpp diff --git a/presto-native-execution/presto_cpp/presto_protocol/README.md b/presto-native-execution/presto_cpp/presto_protocol/README.md index cbd8ec7918f68..84103af803acd 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/README.md +++ b/presto-native-execution/presto_cpp/presto_protocol/README.md @@ -23,6 +23,12 @@ Dependencies to Install You will need to note the path to the installed executable and add this to your PATH. Something like `/Users/adutta/Library/Python/3.8/bin`. + + Presto repo must be present at $HOME/presto. The PRESTO_HOME environment variable can + be set to override this location. + +Presto Protocol Code Generation: + Run `make` in terminal. Implementation Notes: @@ -40,9 +46,7 @@ source generation. * Abstract classes, their subclasses and the subclass type key are listed in AbstractClasses. - * Java files from Presto are listed in JavaClasses. The script defaults the -presto repo checkout to $HOME/presto. The PRESTO_HOME environment variable can -be set to override this location. + * Java files from the Presto repo are listed in JavaClasses. * ExtraProps lists a few additional java class properties that failed to be extracted via Jackson annotations. diff --git a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp index dd9e42b0d7b7e..931b77530f6b6 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.cpp @@ -54,338 +54,459 @@ std::string json_map_key(const VariableReferenceExpression& p) { } } // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ErrorCause_enum_table[] = - { // NOLINT: cert-err58-cpp - {ErrorCause::UNKNOWN, "UNKNOWN"}, - {ErrorCause::LOW_PARTITION_COUNT, "LOW_PARTITION_COUNT"}, - {ErrorCause::EXCEEDS_BROADCAST_MEMORY_LIMIT, - "EXCEEDS_BROADCAST_MEMORY_LIMIT"}}; -void to_json(json& j, const ErrorCause& e) { - static_assert(std::is_enum::value, "ErrorCause must be an enum!"); - const auto* it = std::find_if( - std::begin(ErrorCause_enum_table), - std::end(ErrorCause_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ErrorCause_enum_table)) - ? it - : std::begin(ErrorCause_enum_table)) - ->second; -} -void from_json(const json& j, ErrorCause& e) { - static_assert(std::is_enum::value, "ErrorCause must be an enum!"); - const auto* it = std::find_if( - std::begin(ErrorCause_enum_table), - std::end(ErrorCause_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ErrorCause_enum_table)) - ? it - : std::begin(ErrorCause_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ErrorType_enum_table[] = - { // NOLINT: cert-err58-cpp - {ErrorType::USER_ERROR, "USER_ERROR"}, - {ErrorType::INTERNAL_ERROR, "INTERNAL_ERROR"}, - {ErrorType::INSUFFICIENT_RESOURCES, "INSUFFICIENT_RESOURCES"}, - {ErrorType::EXTERNAL, "EXTERNAL"}}; -void to_json(json& j, const ErrorType& e) { - static_assert(std::is_enum::value, "ErrorType must be an enum!"); - const auto* it = std::find_if( - std::begin(ErrorType_enum_table), - std::end(ErrorType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ErrorType_enum_table)) - ? it - : std::begin(ErrorType_enum_table)) - ->second; -} -void from_json(const json& j, ErrorType& e) { - static_assert(std::is_enum::value, "ErrorType must be an enum!"); - const auto* it = std::find_if( - std::begin(ErrorType_enum_table), - std::end(ErrorType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ErrorType_enum_table)) - ? it - : std::begin(ErrorType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const ErrorCode& p) { +void to_json(json& j, const SourceLocation& p) { j = json::object(); - to_json_key(j, "code", p.code, "ErrorCode", "int", "code"); - to_json_key(j, "name", p.name, "ErrorCode", "String", "name"); - to_json_key(j, "type", p.type, "ErrorCode", "ErrorType", "type"); - to_json_key(j, "retriable", p.retriable, "ErrorCode", "bool", "retriable"); + to_json_key(j, "line", p.line, "SourceLocation", "int", "line"); + to_json_key(j, "column", p.column, "SourceLocation", "int", "column"); } -void from_json(const json& j, ErrorCode& p) { - from_json_key(j, "code", p.code, "ErrorCode", "int", "code"); - from_json_key(j, "name", p.name, "ErrorCode", "String", "name"); - from_json_key(j, "type", p.type, "ErrorCode", "ErrorType", "type"); - from_json_key(j, "retriable", p.retriable, "ErrorCode", "bool", "retriable"); +void from_json(const json& j, SourceLocation& p) { + from_json_key(j, "line", p.line, "SourceLocation", "int", "line"); + from_json_key(j, "column", p.column, "SourceLocation", "int", "column"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { - -void to_json(json& j, const ErrorLocation& p) { - j = json::object(); - to_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); - to_json_key( - j, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); -} - -void from_json(const json& j, ErrorLocation& p) { - from_json_key( - j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); - from_json_key( - j, - "columnNumber", - p.columnNumber, - "ErrorLocation", - "int", - "columnNumber"); +VariableReferenceExpression::VariableReferenceExpression() noexcept { + _type = "variable"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const ExecutionFailureInfo& p) { +void to_json(json& j, const VariableReferenceExpression& p) { j = json::object(); - to_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); - to_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); - to_json_key( - j, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); - to_json_key( - j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); - to_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); - to_json_key( - j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); - to_json_key( - j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + j["@type"] = "variable"; to_json_key( j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); to_json_key( - j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); + j, "name", p.name, "VariableReferenceExpression", "String", "name"); + to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } -void from_json(const json& j, ExecutionFailureInfo& p) { - from_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); - from_json_key( - j, "message", p.message, "ExecutionFailureInfo", "String", "message"); - from_json_key( - j, - "cause", - p.cause, - "ExecutionFailureInfo", - "ExecutionFailureInfo", - "cause"); - from_json_key( - j, - "suppressed", - p.suppressed, - "ExecutionFailureInfo", - "List", - "suppressed"); - from_json_key( - j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); - from_json_key( - j, - "errorLocation", - p.errorLocation, - "ExecutionFailureInfo", - "ErrorLocation", - "errorLocation"); +void from_json(const json& j, VariableReferenceExpression& p) { + p._type = j["@type"]; from_json_key( j, - "errorCode", - p.errorCode, - "ExecutionFailureInfo", - "ErrorCode", - "errorCode"); + "sourceLocation", + p.sourceLocation, + "VariableReferenceExpression", + "SourceLocation", + "sourceLocation"); from_json_key( - j, - "remoteHost", - p.remoteHost, - "ExecutionFailureInfo", - "HostAddress", - "remoteHost"); + j, "name", p.name, "VariableReferenceExpression", "String", "name"); from_json_key( - j, - "errorCause", - p.errorCause, - "ExecutionFailureInfo", - "ErrorCause", - "errorCause"); + j, "type", p.type, "VariableReferenceExpression", "Type", "type"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(nlohmann::json& j, const DataSize& p) { - j = p.toString(); -} - -void from_json(const nlohmann::json& j, DataSize& p) { - p = DataSize(std::string(j)); -} - -std::ostream& operator<<(std::ostream& os, const DataSize& d) { - return os << d.toString(); -} - -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -namespace facebook::presto::protocol { + if (type == ".AggregationNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".DistinctLimitNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".FilterNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".LimitNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SortNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.OutputNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".ProjectNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".TableScanNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".TopNNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == ".ValuesNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { + j = *std::static_pointer_cast(p); + return; + } -void to_json(json& j, const Duration& p) { - j = p.toString(); + throw TypeError(type + " no abstract type PlanNode "); } -void from_json(const json& j, Duration& p) { - p = Duration(std::string(j)); -} +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " PlanNode PlanNode"); + } -std::ostream& operator<<(std::ostream& os, const Duration& d) { - return os << d.toString(); -} + if (type == ".AggregationNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".DistinctLimitNode") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".FilterNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".LimitNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SortNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.OutputNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".ProjectNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".TableScanNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".TopNNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == ".ValuesNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + throw TypeError(type + " no abstract type PlanNode "); +} } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +GroupIdNode::GroupIdNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; +} -void to_json(json& j, const ResourceEstimates& p) { +void to_json(json& j, const GroupIdNode& p) { j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.GroupIdNode"; + to_json_key(j, "id", p.id, "GroupIdNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "GroupIdNode", "PlanNode", "source"); to_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", - "Duration", - "executionTime"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); to_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + j, + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); to_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", - "DataSize", - "peakMemory"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); to_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); } -void from_json(const json& j, ResourceEstimates& p) { +void from_json(const json& j, GroupIdNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "GroupIdNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "GroupIdNode", "PlanNode", "source"); from_json_key( j, - "executionTime", - p.executionTime, - "ResourceEstimates", - "Duration", - "executionTime"); + "groupingSets", + p.groupingSets, + "GroupIdNode", + "List>", + "groupingSets"); from_json_key( - j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + j, + "groupingColumns", + p.groupingColumns, + "GroupIdNode", + "Map", + "groupingColumns"); from_json_key( j, - "peakMemory", - p.peakMemory, - "ResourceEstimates", - "DataSize", - "peakMemory"); + "aggregationArguments", + p.aggregationArguments, + "GroupIdNode", + "List", + "aggregationArguments"); from_json_key( j, - "peakTaskMemory", - p.peakTaskMemory, - "ResourceEstimates", - "DataSize", - "peakTaskMemory"); + "groupIdVariable", + p.groupIdVariable, + "GroupIdNode", + "VariableReferenceExpression", + "groupIdVariable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const DwrfEncryptionMetadata& p) { + j = json::object(); + to_json_key( + j, + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); + to_json_key( + j, + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); + to_json_key( + j, + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); + to_json_key( + j, + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); +} + +void from_json(const json& j, DwrfEncryptionMetadata& p) { + from_json_key( + j, + "fieldToKeyData", + p.fieldToKeyData, + "DwrfEncryptionMetadata", + "Map", + "fieldToKeyData"); + from_json_key( + j, + "extraMetadata", + p.extraMetadata, + "DwrfEncryptionMetadata", + "Map", + "extraMetadata"); + from_json_key( + j, + "encryptionAlgorithm", + p.encryptionAlgorithm, + "DwrfEncryptionMetadata", + "String", + "encryptionAlgorithm"); + from_json_key( + j, + "encryptionProvider", + p.encryptionProvider, + "DwrfEncryptionMetadata", + "String", + "encryptionProvider"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const EncryptionInformation& p) { + j = json::object(); + to_json_key( + j, + "dwrfEncryptionMetadata", + p.dwrfEncryptionMetadata, + "EncryptionInformation", + "DwrfEncryptionMetadata", + "dwrfEncryptionMetadata"); +} + +void from_json(const json& j, EncryptionInformation& p) { + from_json_key( + j, + "dwrfEncryptionMetadata", + p.dwrfEncryptionMetadata, + "EncryptionInformation", + "DwrfEncryptionMetadata", + "dwrfEncryptionMetadata"); } } // namespace facebook::presto::protocol /* @@ -428,1980 +549,2379 @@ void from_json(const json& j, Lifespan& p) { } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair TaskState_enum_table[] = - { // NOLINT: cert-err58-cpp - {TaskState::PLANNED, "PLANNED"}, - {TaskState::RUNNING, "RUNNING"}, - {TaskState::FINISHED, "FINISHED"}, - {TaskState::CANCELED, "CANCELED"}, - {TaskState::ABORTED, "ABORTED"}, - {TaskState::FAILED, "FAILED"}}; -void to_json(json& j, const TaskState& e) { - static_assert(std::is_enum::value, "TaskState must be an enum!"); - const auto* it = std::find_if( - std::begin(TaskState_enum_table), - std::end(TaskState_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(TaskState_enum_table)) - ? it - : std::begin(TaskState_enum_table)) - ->second; +void to_json(json& j, const SplitContext& p) { + j = json::object(); + to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); } -void from_json(const json& j, TaskState& e) { - static_assert(std::is_enum::value, "TaskState must be an enum!"); - const auto* it = std::find_if( - std::begin(TaskState_enum_table), - std::end(TaskState_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(TaskState_enum_table)) - ? it - : std::begin(TaskState_enum_table)) - ->first; + +void from_json(const json& j, SplitContext& p) { + from_json_key( + j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. + namespace facebook::presto::protocol { -void to_json(json& j, const TaskStatus& p) { - j = json::object(); - to_json_key( - j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); - to_json_key( - j, - "taskInstanceIdMostSignificantBits", - p.taskInstanceIdMostSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdMostSignificantBits"); - to_json_key(j, "version", p.version, "TaskStatus", "int64_t", "version"); - to_json_key(j, "state", p.state, "TaskStatus", "TaskState", "state"); - to_json_key(j, "self", p.self, "TaskStatus", "URI", "self"); - to_json_key( - j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", - "List", - "completedDriverGroups"); +void to_json(json& j, const TpchTransactionHandle& p) { + j = json::array(); + j.push_back(p._type); + j.push_back(p.instance); +} + +void from_json(const json& j, TpchTransactionHandle& p) { + j[0].get_to(p._type); + j[1].get_to(p.instance); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// dependency TpchTransactionHandle + +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorTransactionHandle"); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + // TPC-H transactionHandle is an array ["tpch","INSTANCE"]. + if (j.is_array()) { + type = j[0]; + } else { + type = p->getSubclassKey(j); + } + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorTransactionHandle ConnectorTransactionHandle"); + } + + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorTransactionHandle"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "$empty") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorSplit"); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorSplit"); + } + + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (type == "$empty") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorSplit"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Split& p) { + j = json::object(); to_json_key( - j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); + j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); to_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); to_json_key( j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); - to_json_key( + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); +} + +void from_json(const json& j, Split& p) { + from_json_key( + j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); + from_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", - "bool", - "outputBufferOverutilized"); - to_json_key( + "transactionHandle", + p.transactionHandle, + "Split", + "ConnectorTransactionHandle", + "transactionHandle"); + from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", - "int64_t", - "physicalWrittenDataSizeInBytes"); - to_json_key( + "connectorSplit", + p.connectorSplit, + "Split", + "ConnectorSplit", + "connectorSplit"); + from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + from_json_key( j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", - "int64_t", - "memoryReservationInBytes"); + "splitContext", + p.splitContext, + "Split", + "SplitContext", + "splitContext"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ScheduledSplit& p) { + j = json::object(); + to_json_key( + j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "systemMemoryReservationInBytes"); - to_json_key( - j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "peakNodeTotalMemoryReservationInBytes"); - to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); - to_json_key( + "planNodeId", + p.planNodeId, + "ScheduledSplit", + "PlanNodeId", + "planNodeId"); + to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); +} + +void from_json(const json& j, ScheduledSplit& p) { + from_json_key( + j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", - "int64_t", - "fullGcTimeInMillis"); + "planNodeId", + p.planNodeId, + "ScheduledSplit", + "PlanNodeId", + "planNodeId"); + from_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskSource& p) { + j = json::object(); to_json_key( - j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", - "int64_t", - "totalCpuTimeInNanos"); + j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); to_json_key( - j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", - "int64_t", - "taskAgeInMillis"); + j, "splits", p.splits, "TaskSource", "List", "splits"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "queuedPartitionedSplitsWeight"); + "noMoreSplitsForLifespan", + p.noMoreSplitsForLifespan, + "TaskSource", + "List", + "noMoreSplitsForLifespan"); to_json_key( - j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "runningPartitionedSplitsWeight"); + j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); } -void from_json(const json& j, TaskStatus& p) { +void from_json(const json& j, TaskSource& p) { from_json_key( - j, - "taskInstanceIdLeastSignificantBits", - p.taskInstanceIdLeastSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdLeastSignificantBits"); + j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); from_json_key( - j, - "taskInstanceIdMostSignificantBits", - p.taskInstanceIdMostSignificantBits, - "TaskStatus", - "int64_t", - "taskInstanceIdMostSignificantBits"); - from_json_key(j, "version", p.version, "TaskStatus", "int64_t", "version"); - from_json_key(j, "state", p.state, "TaskStatus", "TaskState", "state"); - from_json_key(j, "self", p.self, "TaskStatus", "URI", "self"); + j, "splits", p.splits, "TaskSource", "List", "splits"); from_json_key( j, - "completedDriverGroups", - p.completedDriverGroups, - "TaskStatus", + "noMoreSplitsForLifespan", + p.noMoreSplitsForLifespan, + "TaskSource", "List", - "completedDriverGroups"); - from_json_key( - j, - "failures", - p.failures, - "TaskStatus", - "List", - "failures"); - from_json_key( - j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "TaskStatus", - "int", - "queuedPartitionedDrivers"); - from_json_key( - j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "TaskStatus", - "int", - "runningPartitionedDrivers"); - from_json_key( - j, - "outputBufferUtilization", - p.outputBufferUtilization, - "TaskStatus", - "double", - "outputBufferUtilization"); + "noMoreSplitsForLifespan"); from_json_key( + j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TypeVariableConstraint& p) { + j = json::object(); + to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); + to_json_key( j, - "outputBufferOverutilized", - p.outputBufferOverutilized, - "TaskStatus", + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", "bool", - "outputBufferOverutilized"); - from_json_key( - j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStatus", - "int64_t", - "physicalWrittenDataSizeInBytes"); - from_json_key( - j, - "memoryReservationInBytes", - p.memoryReservationInBytes, - "TaskStatus", - "int64_t", - "memoryReservationInBytes"); - from_json_key( + "comparableRequired"); + to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); + to_json_key( j, - "peakNodeTotalMemoryReservationInBytes", - p.peakNodeTotalMemoryReservationInBytes, - "TaskStatus", - "int64_t", - "peakNodeTotalMemoryReservationInBytes"); - from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); - from_json_key( + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); + to_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStatus", - "int64_t", - "fullGcTimeInMillis"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); +} + +void from_json(const json& j, TypeVariableConstraint& p) { + from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStatus", - "int64_t", - "totalCpuTimeInNanos"); + "comparableRequired", + p.comparableRequired, + "TypeVariableConstraint", + "bool", + "comparableRequired"); from_json_key( j, - "taskAgeInMillis", - p.taskAgeInMillis, - "TaskStatus", - "int64_t", - "taskAgeInMillis"); + "orderableRequired", + p.orderableRequired, + "TypeVariableConstraint", + "bool", + "orderableRequired"); from_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "queuedPartitionedSplitsWeight"); + "variadicBound", + p.variadicBound, + "TypeVariableConstraint", + "String", + "variadicBound"); from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "TaskStatus", - "int64_t", - "runningPartitionedSplitsWeight"); + "nonDecimalNumericRequired", + p.nonDecimalNumericRequired, + "TypeVariableConstraint", + "bool", + "nonDecimalNumericRequired"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair CacheQuotaScope_enum_table[] = +static const std::pair SelectedRoleType_enum_table[] = { // NOLINT: cert-err58-cpp - {CacheQuotaScope::GLOBAL, "GLOBAL"}, - {CacheQuotaScope::SCHEMA, "SCHEMA"}, - {CacheQuotaScope::TABLE, "TABLE"}, - {CacheQuotaScope::PARTITION, "PARTITION"}}; -void to_json(json& j, const CacheQuotaScope& e) { + {SelectedRoleType::ROLE, "ROLE"}, + {SelectedRoleType::ALL, "ALL"}, + {SelectedRoleType::NONE, "NONE"}}; +void to_json(json& j, const SelectedRoleType& e) { static_assert( - std::is_enum::value, "CacheQuotaScope must be an enum!"); - const auto* it = std::find_if( - std::begin(CacheQuotaScope_enum_table), - std::end(CacheQuotaScope_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::is_enum::value, + "SelectedRoleType must be an enum!"); + const auto* it = std::find_if( + std::begin(SelectedRoleType_enum_table), + std::end(SelectedRoleType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(CacheQuotaScope_enum_table)) + j = ((it != std::end(SelectedRoleType_enum_table)) ? it - : std::begin(CacheQuotaScope_enum_table)) + : std::begin(SelectedRoleType_enum_table)) ->second; } -void from_json(const json& j, CacheQuotaScope& e) { +void from_json(const json& j, SelectedRoleType& e) { static_assert( - std::is_enum::value, "CacheQuotaScope must be an enum!"); + std::is_enum::value, + "SelectedRoleType must be an enum!"); const auto* it = std::find_if( - std::begin(CacheQuotaScope_enum_table), - std::end(CacheQuotaScope_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(SelectedRoleType_enum_table), + std::end(SelectedRoleType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(CacheQuotaScope_enum_table)) + e = ((it != std::end(SelectedRoleType_enum_table)) ? it - : std::begin(CacheQuotaScope_enum_table)) + : std::begin(SelectedRoleType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const CacheQuotaRequirement& p) { +void to_json(json& j, const SelectedRole& p) { + j = json::object(); + to_json_key(j, "type", p.type, "SelectedRole", "SelectedRoleType", "type"); + to_json_key(j, "role", p.role, "SelectedRole", "String", "role"); +} + +void from_json(const json& j, SelectedRole& p) { + from_json_key(j, "type", p.type, "SelectedRole", "SelectedRoleType", "type"); + from_json_key(j, "role", p.role, "SelectedRole", "String", "role"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ExchangeNodeScope_enum_table[] = + { // NOLINT: cert-err58-cpp + {ExchangeNodeScope::LOCAL, "LOCAL"}, + {ExchangeNodeScope::REMOTE_STREAMING, "REMOTE_STREAMING"}, + {ExchangeNodeScope::REMOTE_MATERIALIZED, "REMOTE_MATERIALIZED"}}; +void to_json(json& j, const ExchangeNodeScope& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeScope must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeScope_enum_table), + std::end(ExchangeNodeScope_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ExchangeNodeScope_enum_table)) + ? it + : std::begin(ExchangeNodeScope_enum_table)) + ->second; +} +void from_json(const json& j, ExchangeNodeScope& e) { + static_assert( + std::is_enum::value, + "ExchangeNodeScope must be an enum!"); + const auto* it = std::find_if( + std::begin(ExchangeNodeScope_enum_table), + std::end(ExchangeNodeScope_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ExchangeNodeScope_enum_table)) + ? it + : std::begin(ExchangeNodeScope_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair SortOrder_enum_table[] = + { // NOLINT: cert-err58-cpp + {SortOrder::ASC_NULLS_FIRST, "ASC_NULLS_FIRST"}, + {SortOrder::ASC_NULLS_LAST, "ASC_NULLS_LAST"}, + {SortOrder::DESC_NULLS_FIRST, "DESC_NULLS_FIRST"}, + {SortOrder::DESC_NULLS_LAST, "DESC_NULLS_LAST"}}; +void to_json(json& j, const SortOrder& e) { + static_assert(std::is_enum::value, "SortOrder must be an enum!"); + const auto* it = std::find_if( + std::begin(SortOrder_enum_table), + std::end(SortOrder_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(SortOrder_enum_table)) + ? it + : std::begin(SortOrder_enum_table)) + ->second; +} +void from_json(const json& j, SortOrder& e) { + static_assert(std::is_enum::value, "SortOrder must be an enum!"); + const auto* it = std::find_if( + std::begin(SortOrder_enum_table), + std::end(SortOrder_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(SortOrder_enum_table)) + ? it + : std::begin(SortOrder_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Ordering& p) { j = json::object(); to_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); to_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } -void from_json(const json& j, CacheQuotaRequirement& p) { +void from_json(const json& j, Ordering& p) { from_json_key( j, - "cacheQuotaScope", - p.cacheQuotaScope, - "CacheQuotaRequirement", - "CacheQuotaScope", - "cacheQuotaScope"); + "variable", + p.variable, + "Ordering", + "VariableReferenceExpression", + "variable"); from_json_key( - j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); + j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const OperatorInfo& p) {} -void from_json(const json& j, OperatorInfo& p) {} + +void to_json(json& j, const OrderingScheme& p) { + j = json::object(); + to_json_key( + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); +} + +void from_json(const json& j, OrderingScheme& p) { + from_json_key( + j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); +} } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair BlockedReason_enum_table[] = - { // NOLINT: cert-err58-cpp - {BlockedReason::WAITING_FOR_MEMORY, "WAITING_FOR_MEMORY"}}; -void to_json(json& j, const BlockedReason& e) { +static const std::pair ExchangeNodeType_enum_table[] = { + // NOLINT: cert-err58-cpp + {ExchangeNodeType::GATHER, "GATHER"}, + {ExchangeNodeType::REPARTITION, "REPARTITION"}, + {ExchangeNodeType::REPLICATE, "REPLICATE"}, +}; +void to_json(json& j, const ExchangeNodeType& e) { static_assert( - std::is_enum::value, "BlockedReason must be an enum!"); + std::is_enum::value, + "ExchangeNodeType must be an enum!"); const auto* it = std::find_if( - std::begin(BlockedReason_enum_table), - std::end(BlockedReason_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(ExchangeNodeType_enum_table), + std::end(ExchangeNodeType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(BlockedReason_enum_table)) + j = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(BlockedReason_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->second; } -void from_json(const json& j, BlockedReason& e) { +void from_json(const json& j, ExchangeNodeType& e) { static_assert( - std::is_enum::value, "BlockedReason must be an enum!"); + std::is_enum::value, + "ExchangeNodeType must be an enum!"); const auto* it = std::find_if( - std::begin(BlockedReason_enum_table), - std::end(BlockedReason_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(ExchangeNodeType_enum_table), + std::end(ExchangeNodeType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(BlockedReason_enum_table)) + e = ((it != std::end(ExchangeNodeType_enum_table)) ? it - : std::begin(BlockedReason_enum_table)) + : std::begin(ExchangeNodeType_enum_table)) ->first; } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; -void to_json(json& j, const OperatorStats& p) { - j = json::object(); - to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); - to_json_key( - j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); - to_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); - to_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); - to_json_key( - j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); - to_json_key( - j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + if (type == "$remote") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorPartitioningHandle"); + } + + if (type == "$remote") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PartitioningHandle& p) { + j = json::object(); to_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); - to_json_key( + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); +} + +void from_json(const json& j, PartitioningHandle& p) { + from_json_key( j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); - to_json_key( + "connectorId", + p.connectorId, + "PartitioningHandle", + "ConnectorId", + "connectorId"); + from_json_key( j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); - to_json_key( + "transactionHandle", + p.transactionHandle, + "PartitioningHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); + "connectorHandle", + p.connectorHandle, + "PartitioningHandle", + "ConnectorPartitioningHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "call") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "constant") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "special") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "lambda") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "variable") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type RowExpression "); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " RowExpression RowExpression"); + } + + if (type == "call") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "constant") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "special") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "lambda") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "variable") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type RowExpression "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Partitioning& p) { + j = json::object(); to_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); + j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); to_json_key( j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); - to_json_key( + "arguments", + p.arguments, + "Partitioning", + "List>", + "arguments"); +} + +void from_json(const json& j, Partitioning& p) { + from_json_key( + j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); + from_json_key( j, - "inputPositions", - p.inputPositions, - "OperatorStats", - "int64_t", - "inputPositions"); + "arguments", + p.arguments, + "Partitioning", + "List>", + "arguments"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PartitioningScheme& p) { + j = json::object(); to_json_key( j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); to_json_key( j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", - "int64_t", - "getOutputCalls"); - to_json_key( - j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); - to_json_key( - j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); - to_json_key( - j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); - to_json_key( - j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); - to_json_key( - j, - "outputPositions", - p.outputPositions, - "OperatorStats", - "int64_t", - "outputPositions"); + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); to_json_key( j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); to_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); - to_json_key( + "bucketToPartition", + p.bucketToPartition, + "PartitioningScheme", + "List", + "bucketToPartition"); +} + +void from_json(const json& j, PartitioningScheme& p) { + from_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", - "int64_t", - "finishCalls"); - to_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); - to_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); - to_json_key( + "partitioning", + p.partitioning, + "PartitioningScheme", + "Partitioning", + "partitioning"); + from_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); - to_json_key( + "outputLayout", + p.outputLayout, + "PartitioningScheme", + "List", + "outputLayout"); + from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); - to_json_key( + "hashColumn", + p.hashColumn, + "PartitioningScheme", + "VariableReferenceExpression", + "hashColumn"); + from_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( + "replicateNullsAndAny", + p.replicateNullsAndAny, + "PartitioningScheme", + "bool", + "replicateNullsAndAny"); + from_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); + "bucketToPartition", + p.bucketToPartition, + "PartitioningScheme", + "List", + "bucketToPartition"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +ExchangeNode::ExchangeNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; +} + +void to_json(json& j, const ExchangeNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.ExchangeNode"; + to_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); to_json_key( - j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); to_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); to_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); to_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); to_json_key( j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); } -void from_json(const json& j, OperatorStats& p) { - from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); - from_json_key( - j, - "stageExecutionId", - p.stageExecutionId, - "OperatorStats", - "int", - "stageExecutionId"); - from_json_key( - j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); +void from_json(const json& j, ExchangeNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); from_json_key( - j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); + j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); from_json_key( j, - "planNodeId", - p.planNodeId, - "OperatorStats", - "PlanNodeId", - "planNodeId"); + "partitioningScheme", + p.partitioningScheme, + "ExchangeNode", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "operatorType", - p.operatorType, - "OperatorStats", - "String", - "operatorType"); + "sources", + p.sources, + "ExchangeNode", + "List>", + "sources"); from_json_key( j, - "totalDrivers", - p.totalDrivers, - "OperatorStats", - "int64_t", - "totalDrivers"); + "inputs", + p.inputs, + "ExchangeNode", + "List>", + "inputs"); from_json_key( j, - "addInputCalls", - p.addInputCalls, - "OperatorStats", - "int64_t", - "addInputCalls"); - from_json_key( - j, - "addInputWall", - p.addInputWall, - "OperatorStats", - "Duration", - "addInputWall"); - from_json_key( - j, - "addInputCpu", - p.addInputCpu, - "OperatorStats", - "Duration", - "addInputCpu"); - from_json_key( - j, - "addInputAllocation", - p.addInputAllocation, - "OperatorStats", - "DataSize", - "addInputAllocation"); - from_json_key( - j, - "rawInputDataSize", - p.rawInputDataSize, - "OperatorStats", - "DataSize", - "rawInputDataSize"); - from_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "OperatorStats", - "int64_t", - "rawInputPositions"); - from_json_key( - j, - "inputDataSize", - p.inputDataSize, - "OperatorStats", - "DataSize", - "inputDataSize"); - from_json_key( - j, - "inputPositions", - p.inputPositions, - "OperatorStats", - "int64_t", - "inputPositions"); - from_json_key( - j, - "sumSquaredInputPositions", - p.sumSquaredInputPositions, - "OperatorStats", - "double", - "sumSquaredInputPositions"); - from_json_key( - j, - "getOutputCalls", - p.getOutputCalls, - "OperatorStats", - "int64_t", - "getOutputCalls"); - from_json_key( - j, - "getOutputWall", - p.getOutputWall, - "OperatorStats", - "Duration", - "getOutputWall"); - from_json_key( - j, - "getOutputCpu", - p.getOutputCpu, - "OperatorStats", - "Duration", - "getOutputCpu"); - from_json_key( - j, - "getOutputAllocation", - p.getOutputAllocation, - "OperatorStats", - "DataSize", - "getOutputAllocation"); - from_json_key( - j, - "outputDataSize", - p.outputDataSize, - "OperatorStats", - "DataSize", - "outputDataSize"); - from_json_key( - j, - "outputPositions", - p.outputPositions, - "OperatorStats", - "int64_t", - "outputPositions"); - from_json_key( - j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "OperatorStats", - "DataSize", - "physicalWrittenDataSize"); - from_json_key( - j, - "additionalCpu", - p.additionalCpu, - "OperatorStats", - "Duration", - "additionalCpu"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "ExchangeNode", + "bool", + "ensureSourceOrdering"); from_json_key( j, - "blockedWall", - p.blockedWall, - "OperatorStats", - "Duration", - "blockedWall"); - from_json_key( + "orderingScheme", + p.orderingScheme, + "ExchangeNode", + "OrderingScheme", + "orderingScheme"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +RemoteSourceNode::RemoteSourceNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; +} + +void to_json(json& j, const RemoteSourceNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; + to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + to_json_key( j, - "finishCalls", - p.finishCalls, - "OperatorStats", - "int64_t", - "finishCalls"); - from_json_key( - j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); - from_json_key( - j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); - from_json_key( + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); + to_json_key( j, - "finishAllocation", - p.finishAllocation, - "OperatorStats", - "DataSize", - "finishAllocation"); - from_json_key( + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); + to_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "OperatorStats", - "DataSize", - "userMemoryReservation"); - from_json_key( + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); + to_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "OperatorStats", - "DataSize", - "revocableMemoryReservation"); - from_json_key( + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); + to_json_key( j, - "systemMemoryReservation", - p.systemMemoryReservation, - "OperatorStats", - "DataSize", - "systemMemoryReservation"); + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); +} + +void from_json(const json& j, RemoteSourceNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); from_json_key( j, - "peakUserMemoryReservation", - p.peakUserMemoryReservation, - "OperatorStats", - "DataSize", - "peakUserMemoryReservation"); + "sourceFragmentIds", + p.sourceFragmentIds, + "RemoteSourceNode", + "List", + "sourceFragmentIds"); from_json_key( j, - "peakSystemMemoryReservation", - p.peakSystemMemoryReservation, - "OperatorStats", - "DataSize", - "peakSystemMemoryReservation"); + "outputVariables", + p.outputVariables, + "RemoteSourceNode", + "List", + "outputVariables"); from_json_key( j, - "peakTotalMemoryReservation", - p.peakTotalMemoryReservation, - "OperatorStats", - "DataSize", - "peakTotalMemoryReservation"); + "ensureSourceOrdering", + p.ensureSourceOrdering, + "RemoteSourceNode", + "bool", + "ensureSourceOrdering"); from_json_key( j, - "spilledDataSize", - p.spilledDataSize, - "OperatorStats", - "DataSize", - "spilledDataSize"); + "orderingScheme", + p.orderingScheme, + "RemoteSourceNode", + "OrderingScheme", + "orderingScheme"); from_json_key( j, - "blockedReason", - p.blockedReason, - "OperatorStats", - "BlockedReason", - "blockedReason"); - from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + "exchangeType", + p.exchangeType, + "RemoteSourceNode", + "ExchangeNodeType", + "exchangeType"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Parameter& p) { + j = json::object(); + to_json_key(j, "name", p.name, "Parameter", "String", "name"); + to_json_key(j, "type", p.type, "Parameter", "TypeSignature", "type"); +} + +void from_json(const json& j, Parameter& p) { + from_json_key(j, "name", p.name, "Parameter", "String", "name"); + from_json_key(j, "type", p.type, "Parameter", "TypeSignature", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Bound_enum_table[] = + { // NOLINT: cert-err58-cpp + {Bound::BELOW, "BELOW"}, + {Bound::EXACTLY, "EXACTLY"}, + {Bound::ABOVE, "ABOVE"}}; +void to_json(json& j, const Bound& e) { + static_assert(std::is_enum::value, "Bound must be an enum!"); + const auto* it = std::find_if( + std::begin(Bound_enum_table), + std::end(Bound_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) + ->second; +} +void from_json(const json& j, Bound& e) { + static_assert(std::is_enum::value, "Bound must be an enum!"); + const auto* it = std::find_if( + std::begin(Bound_enum_table), + std::end(Bound_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { + +void to_json(json& j, const Block& p) { + j = p.data; +} + +void from_json(const json& j, Block& p) { + p.data = std::string(j); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Marker& p) { + j = json::object(); + to_json_key(j, "type", p.type, "Marker", "Type", "type"); + to_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); + to_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); +} + +void from_json(const json& j, Marker& p) { + from_json_key(j, "type", p.type, "Marker", "Type", "type"); + from_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); + from_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const Range& p) { + j = json::object(); + to_json_key(j, "low", p.low, "Range", "Marker", "low"); + to_json_key(j, "high", p.high, "Range", "Marker", "high"); +} + +void from_json(const json& j, Range& p) { + from_json_key(j, "low", p.low, "Range", "Marker", "low"); + from_json_key(j, "high", p.high, "Range", "Marker", "high"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +SortedRangeSet::SortedRangeSet() noexcept { + _type = "sortable"; +} + +void to_json(json& j, const SortedRangeSet& p) { + j = json::object(); + j["@type"] = "sortable"; + to_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); + to_json_key(j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); +} + +void from_json(const json& j, SortedRangeSet& p) { + p._type = j["@type"]; + from_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); from_json_key( - j, - "runtimeStats", - p.runtimeStats, - "OperatorStats", - "RuntimeStats", - "runtimeStats"); + j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair TaskState_enum_table[] = + { // NOLINT: cert-err58-cpp + {TaskState::PLANNED, "PLANNED"}, + {TaskState::RUNNING, "RUNNING"}, + {TaskState::FINISHED, "FINISHED"}, + {TaskState::CANCELED, "CANCELED"}, + {TaskState::ABORTED, "ABORTED"}, + {TaskState::FAILED, "FAILED"}}; +void to_json(json& j, const TaskState& e) { + static_assert(std::is_enum::value, "TaskState must be an enum!"); + const auto* it = std::find_if( + std::begin(TaskState_enum_table), + std::end(TaskState_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(TaskState_enum_table)) + ? it + : std::begin(TaskState_enum_table)) + ->second; +} +void from_json(const json& j, TaskState& e) { + static_assert(std::is_enum::value, "TaskState must be an enum!"); + const auto* it = std::find_if( + std::begin(TaskState_enum_table), + std::end(TaskState_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(TaskState_enum_table)) + ? it + : std::begin(TaskState_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const DriverStats& p) { +void to_json(json& j, const ErrorLocation& p) { j = json::object(); - to_json_key(j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); - to_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); - to_json_key( - j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - to_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); - to_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); - to_json_key( - j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); - to_json_key( - j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); - to_json_key( - j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); - to_json_key( - j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); - to_json_key( - j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); - to_json_key( - j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); - to_json_key( - j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - to_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); to_json_key( - j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); to_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); - to_json_key( + "columnNumber", + p.columnNumber, + "ErrorLocation", + "int", + "columnNumber"); +} + +void from_json(const json& j, ErrorLocation& p) { + from_json_key( + j, "lineNumber", p.lineNumber, "ErrorLocation", "int", "lineNumber"); + from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); + "columnNumber", + p.columnNumber, + "ErrorLocation", + "int", + "columnNumber"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ErrorCause_enum_table[] = + { // NOLINT: cert-err58-cpp + {ErrorCause::UNKNOWN, "UNKNOWN"}, + {ErrorCause::LOW_PARTITION_COUNT, "LOW_PARTITION_COUNT"}, + {ErrorCause::EXCEEDS_BROADCAST_MEMORY_LIMIT, + "EXCEEDS_BROADCAST_MEMORY_LIMIT"}}; +void to_json(json& j, const ErrorCause& e) { + static_assert(std::is_enum::value, "ErrorCause must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorCause_enum_table), + std::end(ErrorCause_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ErrorCause_enum_table)) + ? it + : std::begin(ErrorCause_enum_table)) + ->second; +} +void from_json(const json& j, ErrorCause& e) { + static_assert(std::is_enum::value, "ErrorCause must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorCause_enum_table), + std::end(ErrorCause_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ErrorCause_enum_table)) + ? it + : std::begin(ErrorCause_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair ErrorType_enum_table[] = + { // NOLINT: cert-err58-cpp + {ErrorType::USER_ERROR, "USER_ERROR"}, + {ErrorType::INTERNAL_ERROR, "INTERNAL_ERROR"}, + {ErrorType::INSUFFICIENT_RESOURCES, "INSUFFICIENT_RESOURCES"}, + {ErrorType::EXTERNAL, "EXTERNAL"}}; +void to_json(json& j, const ErrorType& e) { + static_assert(std::is_enum::value, "ErrorType must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorType_enum_table), + std::end(ErrorType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(ErrorType_enum_table)) + ? it + : std::begin(ErrorType_enum_table)) + ->second; +} +void from_json(const json& j, ErrorType& e) { + static_assert(std::is_enum::value, "ErrorType must be an enum!"); + const auto* it = std::find_if( + std::begin(ErrorType_enum_table), + std::end(ErrorType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(ErrorType_enum_table)) + ? it + : std::begin(ErrorType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ErrorCode& p) { + j = json::object(); + to_json_key(j, "code", p.code, "ErrorCode", "int", "code"); + to_json_key(j, "name", p.name, "ErrorCode", "String", "name"); + to_json_key(j, "type", p.type, "ErrorCode", "ErrorType", "type"); + to_json_key(j, "retriable", p.retriable, "ErrorCode", "bool", "retriable"); +} + +void from_json(const json& j, ErrorCode& p) { + from_json_key(j, "code", p.code, "ErrorCode", "int", "code"); + from_json_key(j, "name", p.name, "ErrorCode", "String", "name"); + from_json_key(j, "type", p.type, "ErrorCode", "ErrorType", "type"); + from_json_key(j, "retriable", p.retriable, "ErrorCode", "bool", "retriable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ExecutionFailureInfo& p) { + j = json::object(); + to_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); to_json_key( - j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", - "int64_t", - "rawInputPositions"); + j, "message", p.message, "ExecutionFailureInfo", "String", "message"); to_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); to_json_key( j, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); to_json_key( - j, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", - "int64_t", - "processedInputPositions"); + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); to_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); to_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", - "int64_t", - "outputPositions"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); to_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); to_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); } -void from_json(const json& j, DriverStats& p) { - from_json_key( - j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); - from_json_key( - j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); - from_json_key( - j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); - from_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); - from_json_key( - j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); +void from_json(const json& j, ExecutionFailureInfo& p) { + from_json_key(j, "type", p.type, "ExecutionFailureInfo", "String", "type"); from_json_key( - j, - "elapsedTime", - p.elapsedTime, - "DriverStats", - "Duration", - "elapsedTime"); + j, "message", p.message, "ExecutionFailureInfo", "String", "message"); from_json_key( j, - "userMemoryReservation", - p.userMemoryReservation, - "DriverStats", - "DataSize", - "userMemoryReservation"); + "cause", + p.cause, + "ExecutionFailureInfo", + "ExecutionFailureInfo", + "cause"); from_json_key( j, - "revocableMemoryReservation", - p.revocableMemoryReservation, - "DriverStats", - "DataSize", - "revocableMemoryReservation"); + "suppressed", + p.suppressed, + "ExecutionFailureInfo", + "List", + "suppressed"); from_json_key( - j, - "systemMemoryReservation", - p.systemMemoryReservation, - "DriverStats", - "DataSize", - "systemMemoryReservation"); + j, "stack", p.stack, "ExecutionFailureInfo", "List", "stack"); from_json_key( j, - "totalScheduledTime", - p.totalScheduledTime, - "DriverStats", - "Duration", - "totalScheduledTime"); + "errorLocation", + p.errorLocation, + "ExecutionFailureInfo", + "ErrorLocation", + "errorLocation"); from_json_key( j, - "totalCpuTime", - p.totalCpuTime, - "DriverStats", - "Duration", - "totalCpuTime"); + "errorCode", + p.errorCode, + "ExecutionFailureInfo", + "ErrorCode", + "errorCode"); from_json_key( j, - "totalBlockedTime", - p.totalBlockedTime, - "DriverStats", - "Duration", - "totalBlockedTime"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); + "remoteHost", + p.remoteHost, + "ExecutionFailureInfo", + "HostAddress", + "remoteHost"); from_json_key( j, - "blockedReasons", - p.blockedReasons, - "DriverStats", - "List", - "blockedReasons"); - from_json_key( + "errorCause", + p.errorCause, + "ExecutionFailureInfo", + "ErrorCause", + "errorCause"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskStatus& p) { + j = json::object(); + to_json_key( j, - "totalAllocation", - p.totalAllocation, - "DriverStats", - "DataSize", - "totalAllocation"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); + to_json_key( + j, + "taskInstanceIdMostSignificantBits", + p.taskInstanceIdMostSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdMostSignificantBits"); + to_json_key(j, "version", p.version, "TaskStatus", "int64_t", "version"); + to_json_key(j, "state", p.state, "TaskStatus", "TaskState", "state"); + to_json_key(j, "self", p.self, "TaskStatus", "URI", "self"); + to_json_key( + j, + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); + to_json_key( + j, + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); + to_json_key( + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); + to_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); + to_json_key( + j, + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); + to_json_key( + j, + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + to_json_key( + j, + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", + "int64_t", + "physicalWrittenDataSizeInBytes"); + to_json_key( + j, + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); + to_json_key( + j, + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "systemMemoryReservationInBytes"); + to_json_key( + j, + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); + to_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); + to_json_key( + j, + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); + to_json_key( + j, + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); + to_json_key( + j, + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); + to_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); +} + +void from_json(const json& j, TaskStatus& p) { from_json_key( j, - "rawInputDataSize", - p.rawInputDataSize, - "DriverStats", - "DataSize", - "rawInputDataSize"); + "taskInstanceIdLeastSignificantBits", + p.taskInstanceIdLeastSignificantBits, + "TaskStatus", + "int64_t", + "taskInstanceIdLeastSignificantBits"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "DriverStats", + "taskInstanceIdMostSignificantBits", + p.taskInstanceIdMostSignificantBits, + "TaskStatus", "int64_t", - "rawInputPositions"); + "taskInstanceIdMostSignificantBits"); + from_json_key(j, "version", p.version, "TaskStatus", "int64_t", "version"); + from_json_key(j, "state", p.state, "TaskStatus", "TaskState", "state"); + from_json_key(j, "self", p.self, "TaskStatus", "URI", "self"); from_json_key( j, - "rawInputReadTime", - p.rawInputReadTime, - "DriverStats", - "Duration", - "rawInputReadTime"); + "completedDriverGroups", + p.completedDriverGroups, + "TaskStatus", + "List", + "completedDriverGroups"); from_json_key( j, - "processedInputDataSize", - p.processedInputDataSize, - "DriverStats", - "DataSize", - "processedInputDataSize"); + "failures", + p.failures, + "TaskStatus", + "List", + "failures"); from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "DriverStats", + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStatus", + "int", + "queuedPartitionedDrivers"); + from_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStatus", + "int", + "runningPartitionedDrivers"); + from_json_key( + j, + "outputBufferUtilization", + p.outputBufferUtilization, + "TaskStatus", + "double", + "outputBufferUtilization"); + from_json_key( + j, + "outputBufferOverutilized", + p.outputBufferOverutilized, + "TaskStatus", + "bool", + "outputBufferOverutilized"); + from_json_key( + j, + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStatus", "int64_t", - "processedInputPositions"); + "physicalWrittenDataSizeInBytes"); from_json_key( j, - "outputDataSize", - p.outputDataSize, - "DriverStats", - "DataSize", - "outputDataSize"); + "memoryReservationInBytes", + p.memoryReservationInBytes, + "TaskStatus", + "int64_t", + "memoryReservationInBytes"); from_json_key( j, - "outputPositions", - p.outputPositions, - "DriverStats", + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStatus", "int64_t", - "outputPositions"); + "systemMemoryReservationInBytes"); from_json_key( j, - "physicalWrittenDataSize", - p.physicalWrittenDataSize, - "DriverStats", - "DataSize", - "physicalWrittenDataSize"); + "peakNodeTotalMemoryReservationInBytes", + p.peakNodeTotalMemoryReservationInBytes, + "TaskStatus", + "int64_t", + "peakNodeTotalMemoryReservationInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStatus", "int64_t", "fullGcCount"); from_json_key( j, - "operatorStats", - p.operatorStats, - "DriverStats", - "List", - "operatorStats"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStatus", + "int64_t", + "fullGcTimeInMillis"); + from_json_key( + j, + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStatus", + "int64_t", + "totalCpuTimeInNanos"); + from_json_key( + j, + "taskAgeInMillis", + p.taskAgeInMillis, + "TaskStatus", + "int64_t", + "taskAgeInMillis"); + from_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "queuedPartitionedSplitsWeight"); + from_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStatus", + "int64_t", + "runningPartitionedSplitsWeight"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - throw TypeError(type + " no abstract type ConnectorTransactionHandle"); +void to_json(json& j, const HiveBucketFilter& p) { + j = json::object(); + to_json_key( + j, + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); } -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - // TPC-H transactionHandle is an array ["tpch","INSTANCE"]. - if (j.is_array()) { - type = j[0]; - } else { - type = p->getSubclassKey(j); - } - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorTransactionHandle ConnectorTransactionHandle"); - } - - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorTransactionHandle"); +void from_json(const json& j, HiveBucketFilter& p) { + from_json_key( + j, + "bucketsToKeep", + p.bucketsToKeep, + "HiveBucketFilter", + "List", + "bucketsToKeep"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "$empty") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } +EmptySplit::EmptySplit() noexcept { + _type = "$empty"; +} - throw TypeError(type + " no abstract type ConnectorSplit"); +void to_json(json& j, const EmptySplit& p) { + j = json::object(); + j["@type"] = "$empty"; + to_json_key( + j, + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); } -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorSplit"); - } +void from_json(const json& j, EmptySplit& p) { + p._type = j["@type"]; + from_json_key( + j, + "connectorId", + p.connectorId, + "EmptySplit", + "ConnectorId", + "connectorId"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (type == "$empty") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } +void to_json(json& j, const Location& p) { + j = json::object(); + to_json_key(j, "location", p.location, "Location", "String", "location"); +} - throw TypeError(type + " no abstract type ConnectorSplit"); +void from_json(const json& j, Location& p) { + from_json_key(j, "location", p.location, "Location", "String", "location"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +RemoteSplit::RemoteSplit() noexcept { + _type = "$remote"; +} -void to_json(json& j, const SplitContext& p) { +void to_json(json& j, const RemoteSplit& p) { j = json::object(); - to_json_key(j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); + j["@type"] = "$remote"; + to_json_key(j, "location", p.location, "RemoteSplit", "Location", "location"); + to_json_key( + j, + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } -void from_json(const json& j, SplitContext& p) { +void from_json(const json& j, RemoteSplit& p) { + p._type = j["@type"]; from_json_key( - j, "cacheable", p.cacheable, "SplitContext", "bool", "cacheable"); + j, "location", p.location, "RemoteSplit", "Location", "location"); + from_json_key( + j, + "remoteSourceTaskId", + p.remoteSourceTaskId, + "RemoteSplit", + "TaskId", + "remoteSourceTaskId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Split& p) { +void to_json(json& j, const VariableStatsEstimate& p) { j = json::object(); to_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); to_json_key( j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - to_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); to_json_key( j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); + to_json_key( + j, + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); } -void from_json(const json& j, Split& p) { +void from_json(const json& j, VariableStatsEstimate& p) { from_json_key( - j, "connectorId", p.connectorId, "Split", "ConnectorId", "connectorId"); + j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "Split", - "ConnectorTransactionHandle", - "transactionHandle"); + "highValue", + p.highValue, + "VariableStatsEstimate", + "double", + "highValue"); from_json_key( j, - "connectorSplit", - p.connectorSplit, - "Split", - "ConnectorSplit", - "connectorSplit"); - from_json_key(j, "lifespan", p.lifespan, "Split", "Lifespan", "lifespan"); + "nullsFraction", + p.nullsFraction, + "VariableStatsEstimate", + "double", + "nullsFraction"); from_json_key( j, - "splitContext", - p.splitContext, - "Split", - "SplitContext", - "splitContext"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); -} - -void from_json( - const json& j, - std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorMetadataUpdateHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); + "averageRowSize", + p.averageRowSize, + "VariableStatsEstimate", + "double", + "averageRowSize"); + from_json_key( + j, + "distinctValuesCount", + p.distinctValuesCount, + "VariableStatsEstimate", + "double", + "distinctValuesCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MetadataUpdates& p) { +void to_json(json& j, const PlanNodeStatsEstimate& p) { j = json::object(); to_json_key( j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + "outputRowCount", + p.outputRowCount, + "PlanNodeStatsEstimate", + "double", + "outputRowCount"); to_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); + "totalSize", + p.totalSize, + "PlanNodeStatsEstimate", + "double", + "totalSize"); + to_json_key( + j, + "confident", + p.confident, + "PlanNodeStatsEstimate", + "bool", + "confident"); + to_json_key( + j, + "variableStatistics", + p.variableStatistics, + "PlanNodeStatsEstimate", + "Map", + "variableStatistics"); } -void from_json(const json& j, MetadataUpdates& p) { +void from_json(const json& j, PlanNodeStatsEstimate& p) { from_json_key( j, - "connectorId", - p.connectorId, - "MetadataUpdates", - "ConnectorId", - "connectorId"); + "outputRowCount", + p.outputRowCount, + "PlanNodeStatsEstimate", + "double", + "outputRowCount"); from_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "MetadataUpdates", - "List>", - "metadataUpdates"); + "totalSize", + p.totalSize, + "PlanNodeStatsEstimate", + "double", + "totalSize"); + from_json_key( + j, + "confident", + p.confident, + "PlanNodeStatsEstimate", + "bool", + "confident"); + from_json_key( + j, + "variableStatistics", + p.variableStatistics, + "PlanNodeStatsEstimate", + "Map", + "variableStatistics"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const SourceLocation& p) { - j = json::object(); - to_json_key(j, "line", p.line, "SourceLocation", "int", "line"); - to_json_key(j, "column", p.column, "SourceLocation", "int", "column"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair BoundType_enum_table[] = + { // NOLINT: cert-err58-cpp + {BoundType::UNBOUNDED_PRECEDING, "UNBOUNDED_PRECEDING"}, + {BoundType::PRECEDING, "PRECEDING"}, + {BoundType::CURRENT_ROW, "CURRENT_ROW"}, + {BoundType::FOLLOWING, "FOLLOWING"}, + {BoundType::UNBOUNDED_FOLLOWING, "UNBOUNDED_FOLLOWING"}}; +void to_json(json& j, const BoundType& e) { + static_assert(std::is_enum::value, "BoundType must be an enum!"); + const auto* it = std::find_if( + std::begin(BoundType_enum_table), + std::end(BoundType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(BoundType_enum_table)) + ? it + : std::begin(BoundType_enum_table)) + ->second; } - -void from_json(const json& j, SourceLocation& p) { - from_json_key(j, "line", p.line, "SourceLocation", "int", "line"); - from_json_key(j, "column", p.column, "SourceLocation", "int", "column"); +void from_json(const json& j, BoundType& e) { + static_assert(std::is_enum::value, "BoundType must be an enum!"); + const auto* it = std::find_if( + std::begin(BoundType_enum_table), + std::end(BoundType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(BoundType_enum_table)) + ? it + : std::begin(BoundType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const HiveBucketFilter& p) { - j = json::object(); - to_json_key( - j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); -} - -void from_json(const json& j, HiveBucketFilter& p) { - from_json_key( - j, - "bucketsToKeep", - p.bucketsToKeep, - "HiveBucketFilter", - "List", - "bucketsToKeep"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "equatable") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "sortable") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "allOrNone") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ValueSet "); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair WindowType_enum_table[] = { + // NOLINT: cert-err58-cpp + {WindowType::RANGE, "RANGE"}, + {WindowType::ROWS, "ROWS"}, +}; +void to_json(json& j, const WindowType& e) { + static_assert(std::is_enum::value, "WindowType must be an enum!"); + const auto* it = std::find_if( + std::begin(WindowType_enum_table), + std::end(WindowType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(WindowType_enum_table)) + ? it + : std::begin(WindowType_enum_table)) + ->second; } +void from_json(const json& j, WindowType& e) { + static_assert(std::is_enum::value, "WindowType must be an enum!"); + const auto* it = std::find_if( + std::begin(WindowType_enum_table), + std::end(WindowType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(WindowType_enum_table)) + ? it + : std::begin(WindowType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ValueSet ValueSet"); - } - - if (type == "equatable") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "sortable") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "allOrNone") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } +void to_json(json& j, const Frame& p) { + j = json::object(); + to_json_key(j, "type", p.type, "Frame", "WindowType", "type"); + to_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); + to_json_key( + j, + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); + to_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); + to_json_key( + j, + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); + to_json_key( + j, + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); + to_json_key( + j, + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); +} - throw TypeError(type + " no abstract type ValueSet "); +void from_json(const json& j, Frame& p) { + from_json_key(j, "type", p.type, "Frame", "WindowType", "type"); + from_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); + from_json_key( + j, + "startValue", + p.startValue, + "Frame", + "VariableReferenceExpression", + "startValue"); + from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); + from_json_key( + j, + "endValue", + p.endValue, + "Frame", + "VariableReferenceExpression", + "endValue"); + from_json_key( + j, + "originalStartValue", + p.originalStartValue, + "Frame", + "String", + "originalStartValue"); + from_json_key( + j, + "originalEndValue", + p.originalEndValue, + "Frame", + "String", + "originalEndValue"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Domain& p) { +void to_json(json& j, const Specification& p) { j = json::object(); - to_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); - to_json_key(j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); + to_json_key( + j, + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); + to_json_key( + j, + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); } -void from_json(const json& j, Domain& p) { - from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); +void from_json(const json& j, Specification& p) { from_json_key( - j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); + j, + "partitionBy", + p.partitionBy, + "Specification", + "List", + "partitionBy"); + from_json_key( + j, + "orderingScheme", + p.orderingScheme, + "Specification", + "OrderingScheme", + "orderingScheme"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { +void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { return; } String type = p->_type; - if (type == "call") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "constant") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "special") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "lambda") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "variable") { - j = *std::static_pointer_cast(p); + if (type == "$static") { + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type RowExpression "); + throw TypeError(type + " no abstract type FunctionHandle "); } -void from_json(const json& j, std::shared_ptr& p) { +void from_json(const json& j, std::shared_ptr& p) { String type; try { type = p->getSubclassKey(j); } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " RowExpression RowExpression"); + throw ParseError(std::string(e.what()) + " FunctionHandle FunctionHandle"); } - if (type == "call") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "constant") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "special") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "lambda") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "variable") { - std::shared_ptr k = - std::make_shared(); + if (type == "$static") { + std::shared_ptr k = + std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = std::static_pointer_cast(k); return; } - throw TypeError(type + " no abstract type RowExpression "); + throw TypeError(type + " no abstract type FunctionHandle "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +CallExpression::CallExpression() noexcept { + _type = "call"; +} -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Form_enum_table[] = - { // NOLINT: cert-err58-cpp - {Form::IF, "IF"}, - {Form::NULL_IF, "NULL_IF"}, - {Form::SWITCH, "SWITCH"}, - {Form::WHEN, "WHEN"}, - {Form::IS_NULL, "IS_NULL"}, - {Form::COALESCE, "COALESCE"}, - {Form::IN, "IN"}, - {Form::AND, "AND"}, - {Form::OR, "OR"}, - {Form::DEREFERENCE, "DEREFERENCE"}, - {Form::ROW_CONSTRUCTOR, "ROW_CONSTRUCTOR"}, - {Form::BIND, "BIND"}}; -void to_json(json& j, const Form& e) { - static_assert(std::is_enum
::value, "Form must be an enum!"); - const auto* it = std::find_if( - std::begin(Form_enum_table), - std::end(Form_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(Form_enum_table)) ? it : std::begin(Form_enum_table)) - ->second; -} -void from_json(const json& j, Form& e) { - static_assert(std::is_enum::value, "Form must be an enum!"); - const auto* it = std::find_if( - std::begin(Form_enum_table), - std::end(Form_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(Form_enum_table)) ? it : std::begin(Form_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -SpecialFormExpression::SpecialFormExpression() noexcept { - _type = "special"; -} - -void to_json(json& j, const SpecialFormExpression& p) { +void to_json(json& j, const CallExpression& p) { j = json::object(); - j["@type"] = "special"; + j["@type"] = "call"; to_json_key( j, "sourceLocation", p.sourceLocation, - "SpecialFormExpression", + "CallExpression", "SourceLocation", "sourceLocation"); - to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); to_json_key( j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); + to_json_key( + j, + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); + to_json_key( + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); to_json_key( j, "arguments", p.arguments, - "SpecialFormExpression", + "CallExpression", "List>", "arguments"); } -void from_json(const json& j, SpecialFormExpression& p) { +void from_json(const json& j, CallExpression& p) { p._type = j["@type"]; from_json_key( j, "sourceLocation", p.sourceLocation, - "SpecialFormExpression", + "CallExpression", "SourceLocation", "sourceLocation"); - from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); from_json_key( j, - "returnType", - p.returnType, - "SpecialFormExpression", - "Type", - "returnType"); + "displayName", + p.displayName, + "CallExpression", + "String", + "displayName"); + from_json_key( + j, + "functionHandle", + p.functionHandle, + "CallExpression", + "FunctionHandle", + "functionHandle"); + from_json_key( + j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); from_json_key( j, "arguments", p.arguments, - "SpecialFormExpression", + "CallExpression", "List>", "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -VariableReferenceExpression::VariableReferenceExpression() noexcept { - _type = "variable"; + +void to_json(json& j, const Function& p) { + j = json::object(); + to_json_key( + j, + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + to_json_key( + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); } -void to_json(json& j, const VariableReferenceExpression& p) { +void from_json(const json& j, Function& p) { + from_json_key( + j, + "functionCall", + p.functionCall, + "Function", + "CallExpression", + "functionCall"); + from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + from_json_key( + j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +WindowNode::WindowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.WindowNode"; +} + +void to_json(json& j, const WindowNode& p) { j = json::object(); - j["@type"] = "variable"; + j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; to_json_key( j, "sourceLocation", p.sourceLocation, - "VariableReferenceExpression", + "WindowNode", "SourceLocation", "sourceLocation"); + to_json_key(j, "id", p.id, "WindowNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "WindowNode", "PlanNode", "source"); to_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); - to_json_key(j, "type", p.type, "VariableReferenceExpression", "Type", "type"); + j, + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); + to_json_key( + j, + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "WindowNode", + "VariableReferenceExpression", + "hashVariable"); + to_json_key( + j, + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); + to_json_key( + j, + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); } -void from_json(const json& j, VariableReferenceExpression& p) { +void from_json(const json& j, WindowNode& p) { p._type = j["@type"]; from_json_key( j, "sourceLocation", p.sourceLocation, - "VariableReferenceExpression", + "WindowNode", "SourceLocation", "sourceLocation"); + from_json_key(j, "id", p.id, "WindowNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "WindowNode", "PlanNode", "source"); from_json_key( - j, "name", p.name, "VariableReferenceExpression", "String", "name"); + j, + "specification", + p.specification, + "WindowNode", + "Specification", + "specification"); from_json_key( - j, "type", p.type, "VariableReferenceExpression", "Type", "type"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const EquiJoinClause& p) { - j = json::object(); - to_json_key( j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); - to_json_key( + "windowFunctions", + p.windowFunctions, + "WindowNode", + "Map", + "windowFunctions"); + from_json_key( j, - "right", - p.right, - "EquiJoinClause", + "hashVariable", + p.hashVariable, + "WindowNode", "VariableReferenceExpression", - "right"); -} - -void from_json(const json& j, EquiJoinClause& p) { + "hashVariable"); from_json_key( j, - "left", - p.left, - "EquiJoinClause", - "VariableReferenceExpression", - "left"); + "prePartitionedInputs", + p.prePartitionedInputs, + "WindowNode", + "List", + "prePartitionedInputs"); from_json_key( j, - "right", - p.right, - "EquiJoinClause", - "VariableReferenceExpression", - "right"); + "preSortedOrderPrefix", + p.preSortedOrderPrefix, + "WindowNode", + "int", + "preSortedOrderPrefix"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair SortOrder_enum_table[] = +static const std::pair ColumnType_enum_table[] = { // NOLINT: cert-err58-cpp - {SortOrder::ASC_NULLS_FIRST, "ASC_NULLS_FIRST"}, - {SortOrder::ASC_NULLS_LAST, "ASC_NULLS_LAST"}, - {SortOrder::DESC_NULLS_FIRST, "DESC_NULLS_FIRST"}, - {SortOrder::DESC_NULLS_LAST, "DESC_NULLS_LAST"}}; -void to_json(json& j, const SortOrder& e) { - static_assert(std::is_enum::value, "SortOrder must be an enum!"); + {ColumnType::PARTITION_KEY, "PARTITION_KEY"}, + {ColumnType::REGULAR, "REGULAR"}, + {ColumnType::SYNTHESIZED, "SYNTHESIZED"}, + {ColumnType::AGGREGATED, "AGGREGATED"}}; +void to_json(json& j, const ColumnType& e) { + static_assert(std::is_enum::value, "ColumnType must be an enum!"); const auto* it = std::find_if( - std::begin(SortOrder_enum_table), - std::end(SortOrder_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(ColumnType_enum_table), + std::end(ColumnType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(SortOrder_enum_table)) + j = ((it != std::end(ColumnType_enum_table)) ? it - : std::begin(SortOrder_enum_table)) - ->second; -} -void from_json(const json& j, SortOrder& e) { - static_assert(std::is_enum::value, "SortOrder must be an enum!"); - const auto* it = std::find_if( - std::begin(SortOrder_enum_table), - std::end(SortOrder_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(SortOrder_enum_table)) - ? it - : std::begin(SortOrder_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Ordering& p) { - j = json::object(); - to_json_key( - j, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); - to_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); -} - -void from_json(const json& j, Ordering& p) { - from_json_key( - j, - "variable", - p.variable, - "Ordering", - "VariableReferenceExpression", - "variable"); - from_json_key( - j, "sortOrder", p.sortOrder, "Ordering", "SortOrder", "sortOrder"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const OrderingScheme& p) { - j = json::object(); - to_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); -} - -void from_json(const json& j, OrderingScheme& p) { - from_json_key( - j, "orderBy", p.orderBy, "OrderingScheme", "List", "orderBy"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Specification& p) { - j = json::object(); - to_json_key( - j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); - to_json_key( - j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); -} - -void from_json(const json& j, Specification& p) { - from_json_key( - j, - "partitionBy", - p.partitionBy, - "Specification", - "List", - "partitionBy"); - from_json_key( - j, - "orderingScheme", - p.orderingScheme, - "Specification", - "OrderingScheme", - "orderingScheme"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ColumnType_enum_table[] = - { // NOLINT: cert-err58-cpp - {ColumnType::PARTITION_KEY, "PARTITION_KEY"}, - {ColumnType::REGULAR, "REGULAR"}, - {ColumnType::SYNTHESIZED, "SYNTHESIZED"}, - {ColumnType::AGGREGATED, "AGGREGATED"}}; -void to_json(json& j, const ColumnType& e) { - static_assert(std::is_enum::value, "ColumnType must be an enum!"); - const auto* it = std::find_if( - std::begin(ColumnType_enum_table), - std::end(ColumnType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ColumnType_enum_table)) - ? it - : std::begin(ColumnType_enum_table)) + : std::begin(ColumnType_enum_table)) ->second; } void from_json(const json& j, ColumnType& e) { @@ -2419,114 +2939,6 @@ void from_json(const json& j, ColumnType& e) { } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$static") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type FunctionHandle "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " FunctionHandle FunctionHandle"); - } - - if (type == "$static") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type FunctionHandle "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -CallExpression::CallExpression() noexcept { - _type = "call"; -} - -void to_json(json& j, const CallExpression& p) { - j = json::object(); - j["@type"] = "call"; - to_json_key( - j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); - to_json_key( - j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); - to_json_key( - j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - to_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); - to_json_key( - j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); -} - -void from_json(const json& j, CallExpression& p) { - p._type = j["@type"]; - from_json_key( - j, - "sourceLocation", - p.sourceLocation, - "CallExpression", - "SourceLocation", - "sourceLocation"); - from_json_key( - j, - "displayName", - p.displayName, - "CallExpression", - "String", - "displayName"); - from_json_key( - j, - "functionHandle", - p.functionHandle, - "CallExpression", - "FunctionHandle", - "functionHandle"); - from_json_key( - j, "returnType", p.returnType, "CallExpression", "Type", "returnType"); - from_json_key( - j, - "arguments", - p.arguments, - "CallExpression", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { void to_json(json& j, const Aggregation& p) { j = json::object(); @@ -2682,3513 +3094,2348 @@ void from_json(const json& j, HiveColumnHandle& p) { "partialAggregation"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const OperatorInfo& p) {} +void from_json(const json& j, OperatorInfo& p) {} +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - BucketFunctionType_enum_table[] = { // NOLINT: cert-err58-cpp - {BucketFunctionType::HIVE_COMPATIBLE, "HIVE_COMPATIBLE"}, - {BucketFunctionType::PRESTO_NATIVE, "PRESTO_NATIVE"}}; -void to_json(json& j, const BucketFunctionType& e) { +static const std::pair BlockedReason_enum_table[] = + { // NOLINT: cert-err58-cpp + {BlockedReason::WAITING_FOR_MEMORY, "WAITING_FOR_MEMORY"}}; +void to_json(json& j, const BlockedReason& e) { static_assert( - std::is_enum::value, - "BucketFunctionType must be an enum!"); + std::is_enum::value, "BlockedReason must be an enum!"); const auto* it = std::find_if( - std::begin(BucketFunctionType_enum_table), - std::end(BucketFunctionType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(BucketFunctionType_enum_table)) + std::begin(BlockedReason_enum_table), + std::end(BlockedReason_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(BlockedReason_enum_table)) ? it - : std::begin(BucketFunctionType_enum_table)) + : std::begin(BlockedReason_enum_table)) ->second; } -void from_json(const json& j, BucketFunctionType& e) { +void from_json(const json& j, BlockedReason& e) { static_assert( - std::is_enum::value, - "BucketFunctionType must be an enum!"); + std::is_enum::value, "BlockedReason must be an enum!"); const auto* it = std::find_if( - std::begin(BucketFunctionType_enum_table), - std::end(BucketFunctionType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(BlockedReason_enum_table), + std::end(BlockedReason_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(BucketFunctionType_enum_table)) + e = ((it != std::end(BlockedReason_enum_table)) ? it - : std::begin(BucketFunctionType_enum_table)) + : std::begin(BlockedReason_enum_table)) ->first; } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Order_enum_table[] = - { // NOLINT: cert-err58-cpp - {Order::ASCENDING, "ASCENDING"}, - {Order::DESCENDING, "DESCENDING"}}; -void to_json(json& j, const Order& e) { - static_assert(std::is_enum::value, "Order must be an enum!"); - const auto* it = std::find_if( - std::begin(Order_enum_table), - std::end(Order_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(Order_enum_table)) ? it : std::begin(Order_enum_table)) - ->second; +void to_json(json& j, const Duration& p) { + j = p.toString(); } -void from_json(const json& j, Order& e) { - static_assert(std::is_enum::value, "Order must be an enum!"); - const auto* it = std::find_if( - std::begin(Order_enum_table), - std::end(Order_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(Order_enum_table)) ? it : std::begin(Order_enum_table)) - ->first; + +void from_json(const json& j, Duration& p) { + p = Duration(std::string(j)); +} + +std::ostream& operator<<(std::ostream& os, const Duration& d) { + return os << d.toString(); } + } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + namespace facebook::presto::protocol { -void to_json(json& j, const SortingColumn& p) { - j = json::object(); - to_json_key( - j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); - to_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); +void to_json(nlohmann::json& j, const DataSize& p) { + j = p.toString(); } -void from_json(const json& j, SortingColumn& p) { - from_json_key( - j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); - from_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); +void from_json(const nlohmann::json& j, DataSize& p) { + p = DataSize(std::string(j)); +} + +std::ostream& operator<<(std::ostream& os, const DataSize& d) { + return os << d.toString(); } + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const HiveBucketProperty& p) { +void to_json(json& j, const OperatorStats& p) { j = json::object(); + to_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); to_json_key( j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); - to_json_key( - j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", + "stageExecutionId", + p.stageExecutionId, + "OperatorStats", "int", - "bucketCount"); + "stageExecutionId"); + to_json_key( + j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + to_json_key( + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); to_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); -} - -void from_json(const json& j, HiveBucketProperty& p) { - from_json_key( + "operatorType", + p.operatorType, + "OperatorStats", + "String", + "operatorType"); + to_json_key( j, - "bucketedBy", - p.bucketedBy, - "HiveBucketProperty", - "List", - "bucketedBy"); - from_json_key( + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); + to_json_key( j, - "bucketCount", - p.bucketCount, - "HiveBucketProperty", - "int", - "bucketCount"); - from_json_key( + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); + to_json_key( j, - "sortedBy", - p.sortedBy, - "HiveBucketProperty", - "List", - "sortedBy"); - from_json_key( + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); + to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HiveBucketProperty", - "BucketFunctionType", - "bucketFunctionType"); - from_json_key( - j, "types", p.types, "HiveBucketProperty", "List", "types"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const DwrfEncryptionMetadata& p) { - j = json::object(); + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); to_json_key( j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); to_json_key( j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); to_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); to_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", - "String", - "encryptionProvider"); -} - -void from_json(const json& j, DwrfEncryptionMetadata& p) { - from_json_key( + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); + to_json_key( j, - "fieldToKeyData", - p.fieldToKeyData, - "DwrfEncryptionMetadata", - "Map", - "fieldToKeyData"); + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); + to_json_key( + j, + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); + to_json_key( + j, + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); + to_json_key( + j, + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); + to_json_key( + j, + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); + to_json_key( + j, + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); + to_json_key( + j, + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); + to_json_key( + j, + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); + to_json_key( + j, + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); + to_json_key( + j, + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); + to_json_key( + j, + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); + to_json_key( + j, + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); + to_json_key( + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + to_json_key( + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); + to_json_key( + j, + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); + to_json_key( + j, + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); + to_json_key( + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); + to_json_key( + j, + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); + to_json_key( + j, + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); + to_json_key( + j, + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); + to_json_key( + j, + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); + to_json_key( + j, + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); + to_json_key( + j, + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + to_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + to_json_key( + j, + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); +} + +void from_json(const json& j, OperatorStats& p) { + from_json_key(j, "stageId", p.stageId, "OperatorStats", "int", "stageId"); from_json_key( j, - "extraMetadata", - p.extraMetadata, - "DwrfEncryptionMetadata", - "Map", - "extraMetadata"); + "stageExecutionId", + p.stageExecutionId, + "OperatorStats", + "int", + "stageExecutionId"); + from_json_key( + j, "pipelineId", p.pipelineId, "OperatorStats", "int", "pipelineId"); + from_json_key( + j, "operatorId", p.operatorId, "OperatorStats", "int", "operatorId"); from_json_key( j, - "encryptionAlgorithm", - p.encryptionAlgorithm, - "DwrfEncryptionMetadata", - "String", - "encryptionAlgorithm"); + "planNodeId", + p.planNodeId, + "OperatorStats", + "PlanNodeId", + "planNodeId"); from_json_key( j, - "encryptionProvider", - p.encryptionProvider, - "DwrfEncryptionMetadata", + "operatorType", + p.operatorType, + "OperatorStats", "String", - "encryptionProvider"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const EncryptionInformation& p) { - j = json::object(); - to_json_key( + "operatorType"); + from_json_key( j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} - -void from_json(const json& j, EncryptionInformation& p) { + "totalDrivers", + p.totalDrivers, + "OperatorStats", + "int64_t", + "totalDrivers"); from_json_key( j, - "dwrfEncryptionMetadata", - p.dwrfEncryptionMetadata, - "EncryptionInformation", - "DwrfEncryptionMetadata", - "dwrfEncryptionMetadata"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - HiveCompressionCodec_enum_table[] = { // NOLINT: cert-err58-cpp - {HiveCompressionCodec::NONE, "NONE"}, - {HiveCompressionCodec::SNAPPY, "SNAPPY"}, - {HiveCompressionCodec::GZIP, "GZIP"}, - {HiveCompressionCodec::LZ4, "LZ4"}, - {HiveCompressionCodec::ZSTD, "ZSTD"}}; -void to_json(json& j, const HiveCompressionCodec& e) { - static_assert( - std::is_enum::value, - "HiveCompressionCodec must be an enum!"); - const auto* it = std::find_if( - std::begin(HiveCompressionCodec_enum_table), - std::end(HiveCompressionCodec_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(HiveCompressionCodec_enum_table)) - ? it - : std::begin(HiveCompressionCodec_enum_table)) - ->second; -} -void from_json(const json& j, HiveCompressionCodec& e) { - static_assert( - std::is_enum::value, - "HiveCompressionCodec must be an enum!"); - const auto* it = std::find_if( - std::begin(HiveCompressionCodec_enum_table), - std::end(HiveCompressionCodec_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(HiveCompressionCodec_enum_table)) - ? it - : std::begin(HiveCompressionCodec_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -void to_json(json& j, const HiveStorageFormat& p) { - throw ParseError("Not implemented"); -} - -static const std::pair HiveStorageFormat_enum_table[] = - { // NOLINT: cert-err58-cpp - {HiveStorageFormat::ORC, "ORC"}, - {HiveStorageFormat::DWRF, "DWRF"}, - {HiveStorageFormat::PARQUET, "PARQUET"}, - {HiveStorageFormat::AVRO, "AVRO"}, - {HiveStorageFormat::RCBINARY, "RCBINARY"}, - {HiveStorageFormat::RCTEXT, "RCTEXT"}, - {HiveStorageFormat::SEQUENCEFILE, "SEQUENCEFILE"}, - {HiveStorageFormat::JSON, "JSON"}, - {HiveStorageFormat::TEXTFILE, "TEXTFILE"}, - {HiveStorageFormat::CSV, "CSV"}, - {HiveStorageFormat::PAGEFILE, "PAGEFILE"}}; - -void from_json(const json& j, HiveStorageFormat& e) { - static_assert( - std::is_enum::value, - "HiveStorageFormat must be an enum!"); - const auto* it = std::find_if( - std::begin(HiveStorageFormat_enum_table), - std::end(HiveStorageFormat_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(HiveStorageFormat_enum_table)) - ? it - : std::begin(HiveStorageFormat_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair WriteMode_enum_table[] = - { // NOLINT: cert-err58-cpp - {WriteMode::STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - "STAGE_AND_MOVE_TO_TARGET_DIRECTORY"}, - {WriteMode::DIRECT_TO_TARGET_NEW_DIRECTORY, - "DIRECT_TO_TARGET_NEW_DIRECTORY"}, - {WriteMode::DIRECT_TO_TARGET_EXISTING_DIRECTORY, - "DIRECT_TO_TARGET_EXISTING_DIRECTORY"}}; -void to_json(json& j, const WriteMode& e) { - static_assert(std::is_enum::value, "WriteMode must be an enum!"); - const auto* it = std::find_if( - std::begin(WriteMode_enum_table), - std::end(WriteMode_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(WriteMode_enum_table)) - ? it - : std::begin(WriteMode_enum_table)) - ->second; -} -void from_json(const json& j, WriteMode& e) { - static_assert(std::is_enum::value, "WriteMode must be an enum!"); - const auto* it = std::find_if( - std::begin(WriteMode_enum_table), - std::end(WriteMode_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(WriteMode_enum_table)) - ? it - : std::begin(WriteMode_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair TableType_enum_table[] = - { // NOLINT: cert-err58-cpp - {TableType::NEW, "NEW"}, - {TableType::EXISTING, "EXISTING"}, - {TableType::TEMPORARY, "TEMPORARY"}}; -void to_json(json& j, const TableType& e) { - static_assert(std::is_enum::value, "TableType must be an enum!"); - const auto* it = std::find_if( - std::begin(TableType_enum_table), - std::end(TableType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(TableType_enum_table)) - ? it - : std::begin(TableType_enum_table)) - ->second; -} -void from_json(const json& j, TableType& e) { - static_assert(std::is_enum::value, "TableType must be an enum!"); - const auto* it = std::find_if( - std::begin(TableType_enum_table), - std::end(TableType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(TableType_enum_table)) - ? it - : std::begin(TableType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const LocationHandle& p) { - j = json::object(); - to_json_key( - j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); - to_json_key( - j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); - to_json_key( - j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); - to_json_key( - j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); - to_json_key( - j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); -} - -void from_json(const json& j, LocationHandle& p) { - from_json_key( - j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + "addInputCalls", + p.addInputCalls, + "OperatorStats", + "int64_t", + "addInputCalls"); from_json_key( - j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + j, + "addInputWall", + p.addInputWall, + "OperatorStats", + "Duration", + "addInputWall"); from_json_key( - j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + j, + "addInputCpu", + p.addInputCpu, + "OperatorStats", + "Duration", + "addInputCpu"); from_json_key( - j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + j, + "addInputAllocation", + p.addInputAllocation, + "OperatorStats", + "DataSize", + "addInputAllocation"); from_json_key( - j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Column& p) { - j = json::object(); - to_json_key(j, "name", p.name, "Column", "String", "name"); - to_json_key(j, "type", p.type, "Column", "String", "type"); -} - -void from_json(const json& j, Column& p) { - from_json_key(j, "name", p.name, "Column", "String", "name"); - from_json_key(j, "type", p.type, "Column", "String", "type"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const StorageFormat& p) { - j = json::object(); - to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); - to_json_key( j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); - to_json_key( + "rawInputDataSize", + p.rawInputDataSize, + "OperatorStats", + "DataSize", + "rawInputDataSize"); + from_json_key( j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); -} - -void from_json(const json& j, StorageFormat& p) { - from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); + "rawInputPositions", + p.rawInputPositions, + "OperatorStats", + "int64_t", + "rawInputPositions"); from_json_key( j, - "inputFormat", - p.inputFormat, - "StorageFormat", - "String", - "inputFormat"); + "inputDataSize", + p.inputDataSize, + "OperatorStats", + "DataSize", + "inputDataSize"); from_json_key( j, - "outputFormat", - p.outputFormat, - "StorageFormat", - "String", - "outputFormat"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Storage& p) { - j = json::object(); - to_json_key( + "inputPositions", + p.inputPositions, + "OperatorStats", + "int64_t", + "inputPositions"); + from_json_key( j, - "storageFormat", - p.storageFormat, - "Storage", - "StorageFormat", - "storageFormat"); - to_json_key(j, "location", p.location, "Storage", "String", "location"); - to_json_key( - j, - "bucketProperty", - p.bucketProperty, - "Storage", - "HiveBucketProperty", - "bucketProperty"); - to_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); - to_json_key( - j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); - to_json_key( - j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); -} - -void from_json(const json& j, Storage& p) { + "sumSquaredInputPositions", + p.sumSquaredInputPositions, + "OperatorStats", + "double", + "sumSquaredInputPositions"); from_json_key( j, - "storageFormat", - p.storageFormat, - "Storage", - "StorageFormat", - "storageFormat"); - from_json_key(j, "location", p.location, "Storage", "String", "location"); + "getOutputCalls", + p.getOutputCalls, + "OperatorStats", + "int64_t", + "getOutputCalls"); from_json_key( j, - "bucketProperty", - p.bucketProperty, - "Storage", - "HiveBucketProperty", - "bucketProperty"); - from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + "getOutputWall", + p.getOutputWall, + "OperatorStats", + "Duration", + "getOutputWall"); from_json_key( j, - "serdeParameters", - p.serdeParameters, - "Storage", - "Map", - "serdeParameters"); + "getOutputCpu", + p.getOutputCpu, + "OperatorStats", + "Duration", + "getOutputCpu"); from_json_key( j, - "parameters", - p.parameters, - "Storage", - "Map", - "parameters"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair PrestoTableType_enum_table[] = - { // NOLINT: cert-err58-cpp - {PrestoTableType::MANAGED_TABLE, "MANAGED_TABLE"}, - {PrestoTableType::EXTERNAL_TABLE, "EXTERNAL_TABLE"}, - {PrestoTableType::VIRTUAL_VIEW, "VIRTUAL_VIEW"}, - {PrestoTableType::MATERIALIZED_VIEW, "MATERIALIZED_VIEW"}, - {PrestoTableType::TEMPORARY_TABLE, "TEMPORARY_TABLE"}, - {PrestoTableType::OTHER, "OTHER"}}; -void to_json(json& j, const PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->second; -} -void from_json(const json& j, PrestoTableType& e) { - static_assert( - std::is_enum::value, "PrestoTableType must be an enum!"); - const auto* it = std::find_if( - std::begin(PrestoTableType_enum_table), - std::end(PrestoTableType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(PrestoTableType_enum_table)) - ? it - : std::begin(PrestoTableType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Table& p) { - j = json::object(); - to_json_key( - j, "databaseName", p.databaseName, "Table", "String", "databaseName"); - to_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); - to_json_key(j, "owner", p.owner, "Table", "String", "owner"); - to_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); - to_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); - to_json_key( - j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); - to_json_key( - j, - "parameters", - p.parameters, - "Table", - "Map", - "parameters"); - to_json_key( - j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); - to_json_key( - j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); -} - -void from_json(const json& j, Table& p) { + "getOutputAllocation", + p.getOutputAllocation, + "OperatorStats", + "DataSize", + "getOutputAllocation"); from_json_key( - j, "databaseName", p.databaseName, "Table", "String", "databaseName"); - from_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); - from_json_key(j, "owner", p.owner, "Table", "String", "owner"); + j, + "outputDataSize", + p.outputDataSize, + "OperatorStats", + "DataSize", + "outputDataSize"); from_json_key( - j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); - from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + j, + "outputPositions", + p.outputPositions, + "OperatorStats", + "int64_t", + "outputPositions"); from_json_key( - j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); + j, + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "OperatorStats", + "DataSize", + "physicalWrittenDataSize"); from_json_key( j, - "partitionColumns", - p.partitionColumns, - "Table", - "List", - "partitionColumns"); + "additionalCpu", + p.additionalCpu, + "OperatorStats", + "Duration", + "additionalCpu"); from_json_key( j, - "parameters", - p.parameters, - "Table", - "Map", - "parameters"); + "blockedWall", + p.blockedWall, + "OperatorStats", + "Duration", + "blockedWall"); from_json_key( j, - "viewOriginalText", - p.viewOriginalText, - "Table", - "String", - "viewOriginalText"); + "finishCalls", + p.finishCalls, + "OperatorStats", + "int64_t", + "finishCalls"); + from_json_key( + j, "finishWall", p.finishWall, "OperatorStats", "Duration", "finishWall"); + from_json_key( + j, "finishCpu", p.finishCpu, "OperatorStats", "Duration", "finishCpu"); from_json_key( j, - "viewExpandedText", - p.viewExpandedText, - "Table", - "String", - "viewExpandedText"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const SchemaTableName& p) { - j = json::object(); - to_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); - to_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); -} - -void from_json(const json& j, SchemaTableName& p) { - from_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); - from_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -void to_json(json& j, const HivePageSinkMetadata& p) { - j = json::object(); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); -} - -void from_json(const json& j, HivePageSinkMetadata& p) { - from_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HivePageSinkMetadata", - "SchemaTableName", - "schemaTableName"); - from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -HiveInsertTableHandle::HiveInsertTableHandle() noexcept { - _type = "hive"; -} - -void to_json(json& j, const HiveInsertTableHandle& p) { - j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); - to_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); - to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); - to_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - to_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - to_json_key( - j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); - to_json_key( - j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); - to_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - to_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - to_json_key( - j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - to_json_key( - j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); - to_json_key( - j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); -} - -void from_json(const json& j, HiveInsertTableHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "schemaName", - p.schemaName, - "HiveInsertTableHandle", - "String", - "schemaName"); - from_json_key( - j, - "tableName", - p.tableName, - "HiveInsertTableHandle", - "String", - "tableName"); - from_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveInsertTableHandle", - "List", - "inputColumns"); - from_json_key( - j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveInsertTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); - from_json_key( - j, - "locationHandle", - p.locationHandle, - "HiveInsertTableHandle", - "LocationHandle", - "locationHandle"); - from_json_key( - j, - "bucketProperty", - p.bucketProperty, - "HiveInsertTableHandle", - "HiveBucketProperty", - "bucketProperty"); - from_json_key( - j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveInsertTableHandle", - "List", - "preferredOrderingColumns"); - from_json_key( - j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - from_json_key( - j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - from_json_key( - j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveInsertTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - from_json_key( - j, - "compressionCodec", - p.compressionCodec, - "HiveInsertTableHandle", - "HiveCompressionCodec", - "compressionCodec"); - from_json_key( - j, - "encryptionInformation", - p.encryptionInformation, - "HiveInsertTableHandle", - "EncryptionInformation", - "encryptionInformation"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const NodeVersion& p) { - j = json::object(); - to_json_key(j, "version", p.version, "NodeVersion", "String", "version"); -} - -void from_json(const json& j, NodeVersion& p) { - from_json_key(j, "version", p.version, "NodeVersion", "String", "version"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const ServerInfo& p) { - j = json::object(); - to_json_key( - j, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); - to_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); - to_json_key( - j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); - to_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); - to_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); -} - -void from_json(const json& j, ServerInfo& p) { - from_json_key( - j, - "nodeVersion", - p.nodeVersion, - "ServerInfo", - "NodeVersion", - "nodeVersion"); - from_json_key( - j, "environment", p.environment, "ServerInfo", "String", "environment"); - from_json_key( - j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); - from_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); - from_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == ".AggregationNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".DistinctLimitNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".FilterNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".LimitNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SortNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.OutputNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".ProjectNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".TableScanNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".TopNNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == ".ValuesNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type PlanNode "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " PlanNode PlanNode"); - } - - if (type == ".AggregationNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.GroupIdNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".DistinctLimitNode") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode") { - std::shared_ptr k = - std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.ExchangeNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".FilterNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.JoinNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".LimitNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SortNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.OutputNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".ProjectNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RowNumberNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.RemoteSourceNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.SemiJoinNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".TableScanNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.TableWriterNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".TopNNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.UnnestNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == ".ValuesNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.AssignUniqueId") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.MergeJoinNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "com.facebook.presto.sql.planner.plan.WindowNode") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type PlanNode "); + "finishAllocation", + p.finishAllocation, + "OperatorStats", + "DataSize", + "finishAllocation"); + from_json_key( + j, + "userMemoryReservation", + p.userMemoryReservation, + "OperatorStats", + "DataSize", + "userMemoryReservation"); + from_json_key( + j, + "revocableMemoryReservation", + p.revocableMemoryReservation, + "OperatorStats", + "DataSize", + "revocableMemoryReservation"); + from_json_key( + j, + "systemMemoryReservation", + p.systemMemoryReservation, + "OperatorStats", + "DataSize", + "systemMemoryReservation"); + from_json_key( + j, + "peakUserMemoryReservation", + p.peakUserMemoryReservation, + "OperatorStats", + "DataSize", + "peakUserMemoryReservation"); + from_json_key( + j, + "peakSystemMemoryReservation", + p.peakSystemMemoryReservation, + "OperatorStats", + "DataSize", + "peakSystemMemoryReservation"); + from_json_key( + j, + "peakTotalMemoryReservation", + p.peakTotalMemoryReservation, + "OperatorStats", + "DataSize", + "peakTotalMemoryReservation"); + from_json_key( + j, + "spilledDataSize", + p.spilledDataSize, + "OperatorStats", + "DataSize", + "spilledDataSize"); + from_json_key( + j, + "blockedReason", + p.blockedReason, + "OperatorStats", + "BlockedReason", + "blockedReason"); + from_json_key(j, "info", p.info, "OperatorStats", "OperatorInfo", "info"); + from_json_key( + j, + "runtimeStats", + p.runtimeStats, + "OperatorStats", + "RuntimeStats", + "runtimeStats"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Step_enum_table[] = +static const std::pair BufferState_enum_table[] = { // NOLINT: cert-err58-cpp - {Step::SINGLE, "SINGLE"}, - {Step::PARTIAL, "PARTIAL"}, - {Step::FINAL, "FINAL"}}; -void to_json(json& j, const Step& e) { - static_assert(std::is_enum::value, "Step must be an enum!"); + {BufferState::OPEN, "OPEN"}, + {BufferState::NO_MORE_BUFFERS, "NO_MORE_BUFFERS"}, + {BufferState::NO_MORE_PAGES, "NO_MORE_PAGES"}, + {BufferState::FLUSHING, "FLUSHING"}, + {BufferState::FINISHED, "FINISHED"}, + {BufferState::FAILED, "FAILED"}}; +void to_json(json& j, const BufferState& e) { + static_assert( + std::is_enum::value, "BufferState must be an enum!"); const auto* it = std::find_if( - std::begin(Step_enum_table), - std::end(Step_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(BufferState_enum_table), + std::end(BufferState_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + j = ((it != std::end(BufferState_enum_table)) + ? it + : std::begin(BufferState_enum_table)) ->second; } -void from_json(const json& j, Step& e) { - static_assert(std::is_enum::value, "Step must be an enum!"); +void from_json(const json& j, BufferState& e) { + static_assert( + std::is_enum::value, "BufferState must be an enum!"); const auto* it = std::find_if( - std::begin(Step_enum_table), - std::end(Step_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(BufferState_enum_table), + std::end(BufferState_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + e = ((it != std::end(BufferState_enum_table)) + ? it + : std::begin(BufferState_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TopNNode::TopNNode() noexcept { - _type = ".TopNNode"; -} -void to_json(json& j, const TopNNode& p) { +void to_json(json& j, const PageBufferInfo& p) { j = json::object(); - j["@type"] = ".TopNNode"; - to_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); - to_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); to_json_key( - j, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); -} - -void from_json(const json& j, TopNNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); - from_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); - from_json_key( - j, - "orderingScheme", - p.orderingScheme, - "TopNNode", - "OrderingScheme", - "orderingScheme"); - from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { - _type = "hive"; -} - -void to_json(json& j, const HiveMetadataUpdateHandle& p) { - j = json::object(); - j["@type"] = "hive"; + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); to_json_key( j, - "requestId", - p.requestId, - "HiveMetadataUpdateHandle", - "UUID", - "requestId"); + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", + "int64_t", + "bufferedPages"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "HiveMetadataUpdateHandle", - "SchemaTableName", - "schemaTableName"); + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", + "int64_t", + "bufferedBytes"); to_json_key( - j, - "partitionName", - p.partitionName, - "HiveMetadataUpdateHandle", - "String", - "partitionName"); + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); to_json_key( - j, - "fileName", - p.fileName, - "HiveMetadataUpdateHandle", - "String", - "fileName"); + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); } -void from_json(const json& j, HiveMetadataUpdateHandle& p) { - p._type = j["@type"]; - from_json_key( - j, - "requestId", - p.requestId, - "HiveMetadataUpdateHandle", - "UUID", - "requestId"); - from_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HiveMetadataUpdateHandle", - "SchemaTableName", - "schemaTableName"); +void from_json(const json& j, PageBufferInfo& p) { from_json_key( - j, - "partitionName", - p.partitionName, - "HiveMetadataUpdateHandle", - "String", - "partitionName"); + j, "partition", p.partition, "PageBufferInfo", "int", "partition"); from_json_key( j, - "fileName", - p.fileName, - "HiveMetadataUpdateHandle", - "String", - "fileName"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const DistributionSnapshot& p) { - j = json::object(); - to_json_key( - j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); - to_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); - to_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); - to_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); - to_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); - to_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); - to_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); - to_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); - to_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); - to_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); - to_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); - to_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); - to_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); - to_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); - to_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); -} - -void from_json(const json& j, DistributionSnapshot& p) { - from_json_key( - j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); - from_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); - from_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); - from_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); - from_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); - from_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); - from_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); - from_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); - from_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); - from_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); - from_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); - from_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); - from_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); - from_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); - from_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); + "bufferedPages", + p.bufferedPages, + "PageBufferInfo", + "int64_t", + "bufferedPages"); + from_json_key( + j, + "bufferedBytes", + p.bufferedBytes, + "PageBufferInfo", + "int64_t", + "bufferedBytes"); + from_json_key( + j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + from_json_key( + j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const PipelineStats& p) { +void to_json(json& j, const BufferInfo& p) { j = json::object(); to_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); to_json_key( j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); to_json_key( - j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); to_json_key( j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} + +void from_json(const json& j, BufferInfo& p) { + from_json_key( + j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); + from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + from_json_key( + j, + "bufferedPages", + p.bufferedPages, + "BufferInfo", + "int", + "bufferedPages"); + from_json_key( + j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + from_json_key( + j, + "pageBufferInfo", + p.pageBufferInfo, + "BufferInfo", + "PageBufferInfo", + "pageBufferInfo"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const OutputBufferInfo& p) { + j = json::object(); + to_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); + to_json_key(j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); to_json_key( j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", "bool", - "inputPipeline"); + "canAddBuffers"); to_json_key( j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", + "canAddPages", + p.canAddPages, + "OutputBufferInfo", "bool", - "outputPipeline"); + "canAddPages"); to_json_key( j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", + "int64_t", + "totalBufferedBytes"); to_json_key( j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", + "int64_t", + "totalBufferedPages"); to_json_key( j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", + "int64_t", + "totalRowsSent"); to_json_key( j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", "int64_t", - "queuedPartitionedSplitsWeight"); + "totalPagesSent"); to_json_key( j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); - to_json_key( + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); +} + +void from_json(const json& j, OutputBufferInfo& p) { + from_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); + from_json_key( + j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); + from_json_key( j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); - to_json_key( + "canAddBuffers", + p.canAddBuffers, + "OutputBufferInfo", + "bool", + "canAddBuffers"); + from_json_key( j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", + "canAddPages", + p.canAddPages, + "OutputBufferInfo", + "bool", + "canAddPages"); + from_json_key( + j, + "totalBufferedBytes", + p.totalBufferedBytes, + "OutputBufferInfo", "int64_t", - "runningPartitionedSplitsWeight"); - to_json_key( + "totalBufferedBytes"); + from_json_key( j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); - to_json_key( + "totalBufferedPages", + p.totalBufferedPages, + "OutputBufferInfo", + "int64_t", + "totalBufferedPages"); + from_json_key( j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); - to_json_key( + "totalRowsSent", + p.totalRowsSent, + "OutputBufferInfo", + "int64_t", + "totalRowsSent"); + from_json_key( j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", + "totalPagesSent", + p.totalPagesSent, + "OutputBufferInfo", "int64_t", - "userMemoryReservationInBytes"); + "totalPagesSent"); + from_json_key( + j, + "buffers", + p.buffers, + "OutputBufferInfo", + "List", + "buffers"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const DriverStats& p) { + j = json::object(); + to_json_key(j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); + to_json_key( + j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); + to_json_key( + j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); + to_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); + to_json_key( + j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); to_json_key( j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); + "elapsedTime", + p.elapsedTime, + "DriverStats", + "Duration", + "elapsedTime"); to_json_key( j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); to_json_key( j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); to_json_key( j, - "elapsedTime", - p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", - "elapsedTime"); + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); to_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); to_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); to_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); to_json_key( - j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); to_json_key( j, "blockedReasons", p.blockedReasons, - "PipelineStats", + "DriverStats", "List", "blockedReasons"); to_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); to_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); to_json_key( j, "rawInputPositions", p.rawInputPositions, - "PipelineStats", + "DriverStats", "int64_t", "rawInputPositions"); to_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); + to_json_key( + j, + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); to_json_key( j, "processedInputPositions", p.processedInputPositions, - "PipelineStats", + "DriverStats", "int64_t", "processedInputPositions"); to_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); to_json_key( j, "outputPositions", p.outputPositions, - "PipelineStats", + "DriverStats", "int64_t", "outputPositions"); to_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); to_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", + "operatorStats", + p.operatorStats, + "DriverStats", "List", - "operatorSummaries"); - to_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + "operatorStats"); } -void from_json(const json& j, PipelineStats& p) { - from_json_key( - j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); - from_json_key( - j, - "firstStartTime", - p.firstStartTime, - "PipelineStats", - "DateTime", - "firstStartTime"); - from_json_key( - j, - "lastStartTime", - p.lastStartTime, - "PipelineStats", - "DateTime", - "lastStartTime"); - from_json_key( - j, - "lastEndTime", - p.lastEndTime, - "PipelineStats", - "DateTime", - "lastEndTime"); - from_json_key( - j, - "inputPipeline", - p.inputPipeline, - "PipelineStats", - "bool", - "inputPipeline"); - from_json_key( - j, - "outputPipeline", - p.outputPipeline, - "PipelineStats", - "bool", - "outputPipeline"); - from_json_key( - j, - "totalDrivers", - p.totalDrivers, - "PipelineStats", - "int", - "totalDrivers"); - from_json_key( - j, - "queuedDrivers", - p.queuedDrivers, - "PipelineStats", - "int", - "queuedDrivers"); - from_json_key( - j, - "queuedPartitionedDrivers", - p.queuedPartitionedDrivers, - "PipelineStats", - "int", - "queuedPartitionedDrivers"); - from_json_key( - j, - "queuedPartitionedSplitsWeight", - p.queuedPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "queuedPartitionedSplitsWeight"); - from_json_key( - j, - "runningDrivers", - p.runningDrivers, - "PipelineStats", - "int", - "runningDrivers"); - from_json_key( - j, - "runningPartitionedDrivers", - p.runningPartitionedDrivers, - "PipelineStats", - "int", - "runningPartitionedDrivers"); - from_json_key( - j, - "runningPartitionedSplitsWeight", - p.runningPartitionedSplitsWeight, - "PipelineStats", - "int64_t", - "runningPartitionedSplitsWeight"); - from_json_key( - j, - "blockedDrivers", - p.blockedDrivers, - "PipelineStats", - "int", - "blockedDrivers"); - from_json_key( - j, - "completedDrivers", - p.completedDrivers, - "PipelineStats", - "int", - "completedDrivers"); - from_json_key( - j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "userMemoryReservationInBytes"); +void from_json(const json& j, DriverStats& p) { from_json_key( - j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "revocableMemoryReservationInBytes"); + j, "lifespan", p.lifespan, "DriverStats", "Lifespan", "lifespan"); from_json_key( - j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "PipelineStats", - "int64_t", - "systemMemoryReservationInBytes"); + j, "createTime", p.createTime, "DriverStats", "DateTime", "createTime"); from_json_key( - j, - "queuedTime", - p.queuedTime, - "PipelineStats", - "DistributionSnapshot", - "queuedTime"); + j, "startTime", p.startTime, "DriverStats", "DateTime", "startTime"); + from_json_key(j, "endTime", p.endTime, "DriverStats", "DateTime", "endTime"); + from_json_key( + j, "queuedTime", p.queuedTime, "DriverStats", "Duration", "queuedTime"); from_json_key( j, "elapsedTime", p.elapsedTime, - "PipelineStats", - "DistributionSnapshot", + "DriverStats", + "Duration", "elapsedTime"); from_json_key( j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "PipelineStats", - "int64_t", - "totalScheduledTimeInNanos"); + "userMemoryReservation", + p.userMemoryReservation, + "DriverStats", + "DataSize", + "userMemoryReservation"); from_json_key( j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "PipelineStats", - "int64_t", - "totalCpuTimeInNanos"); + "revocableMemoryReservation", + p.revocableMemoryReservation, + "DriverStats", + "DataSize", + "revocableMemoryReservation"); from_json_key( j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "PipelineStats", - "int64_t", - "totalBlockedTimeInNanos"); + "systemMemoryReservation", + p.systemMemoryReservation, + "DriverStats", + "DataSize", + "systemMemoryReservation"); from_json_key( j, - "fullyBlocked", - p.fullyBlocked, - "PipelineStats", - "bool", - "fullyBlocked"); + "totalScheduledTime", + p.totalScheduledTime, + "DriverStats", + "Duration", + "totalScheduledTime"); + from_json_key( + j, + "totalCpuTime", + p.totalCpuTime, + "DriverStats", + "Duration", + "totalCpuTime"); + from_json_key( + j, + "totalBlockedTime", + p.totalBlockedTime, + "DriverStats", + "Duration", + "totalBlockedTime"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "DriverStats", "bool", "fullyBlocked"); from_json_key( j, "blockedReasons", p.blockedReasons, - "PipelineStats", + "DriverStats", "List", "blockedReasons"); from_json_key( j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "PipelineStats", - "int64_t", - "totalAllocationInBytes"); + "totalAllocation", + p.totalAllocation, + "DriverStats", + "DataSize", + "totalAllocation"); from_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "rawInputDataSizeInBytes"); + "rawInputDataSize", + p.rawInputDataSize, + "DriverStats", + "DataSize", + "rawInputDataSize"); from_json_key( j, "rawInputPositions", p.rawInputPositions, - "PipelineStats", + "DriverStats", "int64_t", "rawInputPositions"); from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "PipelineStats", - "int64_t", - "processedInputDataSizeInBytes"); + "rawInputReadTime", + p.rawInputReadTime, + "DriverStats", + "Duration", + "rawInputReadTime"); + from_json_key( + j, + "processedInputDataSize", + p.processedInputDataSize, + "DriverStats", + "DataSize", + "processedInputDataSize"); from_json_key( j, "processedInputPositions", p.processedInputPositions, - "PipelineStats", + "DriverStats", "int64_t", "processedInputPositions"); from_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "PipelineStats", - "int64_t", - "outputDataSizeInBytes"); + "outputDataSize", + p.outputDataSize, + "DriverStats", + "DataSize", + "outputDataSize"); from_json_key( j, "outputPositions", p.outputPositions, - "PipelineStats", + "DriverStats", "int64_t", "outputPositions"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "PipelineStats", - "int64_t", - "physicalWrittenDataSizeInBytes"); + "physicalWrittenDataSize", + p.physicalWrittenDataSize, + "DriverStats", + "DataSize", + "physicalWrittenDataSize"); from_json_key( j, - "operatorSummaries", - p.operatorSummaries, - "PipelineStats", + "operatorStats", + p.operatorStats, + "DriverStats", "List", - "operatorSummaries"); + "operatorStats"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const DistributionSnapshot& p) { + j = json::object(); + to_json_key( + j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); + to_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); + to_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); + to_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); + to_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); + to_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); + to_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); + to_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); + to_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); + to_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); + to_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); + to_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); + to_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); + to_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); + to_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); +} + +void from_json(const json& j, DistributionSnapshot& p) { from_json_key( - j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); + j, "maxError", p.maxError, "DistributionSnapshot", "double", "maxError"); + from_json_key(j, "count", p.count, "DistributionSnapshot", "double", "count"); + from_json_key(j, "total", p.total, "DistributionSnapshot", "double", "total"); + from_json_key(j, "p01", p.p01, "DistributionSnapshot", "int64_t", "p01"); + from_json_key(j, "p05", p.p05, "DistributionSnapshot", "int64_t", "p05"); + from_json_key(j, "p10", p.p10, "DistributionSnapshot", "int64_t", "p10"); + from_json_key(j, "p25", p.p25, "DistributionSnapshot", "int64_t", "p25"); + from_json_key(j, "p50", p.p50, "DistributionSnapshot", "int64_t", "p50"); + from_json_key(j, "p75", p.p75, "DistributionSnapshot", "int64_t", "p75"); + from_json_key(j, "p90", p.p90, "DistributionSnapshot", "int64_t", "p90"); + from_json_key(j, "p95", p.p95, "DistributionSnapshot", "int64_t", "p95"); + from_json_key(j, "p99", p.p99, "DistributionSnapshot", "int64_t", "p99"); + from_json_key(j, "min", p.min, "DistributionSnapshot", "int64_t", "min"); + from_json_key(j, "max", p.max, "DistributionSnapshot", "int64_t", "max"); + from_json_key(j, "avg", p.avg, "DistributionSnapshot", "double", "avg"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TaskStats& p) { +void to_json(json& j, const PipelineStats& p) { j = json::object(); to_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); to_json_key( j, "firstStartTime", p.firstStartTime, - "TaskStats", + "PipelineStats", "DateTime", "firstStartTime"); to_json_key( j, "lastStartTime", p.lastStartTime, - "TaskStats", + "PipelineStats", "DateTime", "lastStartTime"); to_json_key( - j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); - to_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + j, + "lastEndTime", + p.lastEndTime, + "PipelineStats", + "DateTime", + "lastEndTime"); to_json_key( j, - "elapsedTimeInNanos", - p.elapsedTimeInNanos, - "TaskStats", - "int64_t", - "elapsedTimeInNanos"); + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); to_json_key( j, - "queuedTimeInNanos", - p.queuedTimeInNanos, - "TaskStats", - "int64_t", - "queuedTimeInNanos"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); to_json_key( - j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + j, + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); to_json_key( - j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + j, + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); to_json_key( j, "queuedPartitionedDrivers", p.queuedPartitionedDrivers, - "TaskStats", + "PipelineStats", "int", "queuedPartitionedDrivers"); to_json_key( j, "queuedPartitionedSplitsWeight", p.queuedPartitionedSplitsWeight, - "TaskStats", + "PipelineStats", "int64_t", "queuedPartitionedSplitsWeight"); to_json_key( j, "runningDrivers", p.runningDrivers, - "TaskStats", + "PipelineStats", "int", "runningDrivers"); to_json_key( j, "runningPartitionedDrivers", p.runningPartitionedDrivers, - "TaskStats", + "PipelineStats", "int", "runningPartitionedDrivers"); to_json_key( j, "runningPartitionedSplitsWeight", p.runningPartitionedSplitsWeight, - "TaskStats", + "PipelineStats", "int64_t", "runningPartitionedSplitsWeight"); to_json_key( j, "blockedDrivers", p.blockedDrivers, - "TaskStats", + "PipelineStats", "int", "blockedDrivers"); to_json_key( j, "completedDrivers", p.completedDrivers, - "TaskStats", - "int", - "completedDrivers"); - to_json_key( - j, - "cumulativeUserMemory", - p.cumulativeUserMemory, - "TaskStats", - "double", - "cumulativeUserMemory"); - to_json_key( - j, - "cumulativeTotalMemory", - p.cumulativeTotalMemory, - "TaskStats", - "double", - "cumulativeTotalMemory"); + "PipelineStats", + "int", + "completedDrivers"); to_json_key( j, "userMemoryReservationInBytes", p.userMemoryReservationInBytes, - "TaskStats", + "PipelineStats", "int64_t", "userMemoryReservationInBytes"); to_json_key( j, "revocableMemoryReservationInBytes", p.revocableMemoryReservationInBytes, - "TaskStats", + "PipelineStats", "int64_t", "revocableMemoryReservationInBytes"); to_json_key( j, "systemMemoryReservationInBytes", p.systemMemoryReservationInBytes, - "TaskStats", + "PipelineStats", "int64_t", "systemMemoryReservationInBytes"); to_json_key( j, - "peakTotalMemoryInBytes", - p.peakTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakTotalMemoryInBytes"); - to_json_key( - j, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); to_json_key( j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); to_json_key( j, "totalScheduledTimeInNanos", p.totalScheduledTimeInNanos, - "TaskStats", + "PipelineStats", "int64_t", "totalScheduledTimeInNanos"); to_json_key( j, "totalCpuTimeInNanos", p.totalCpuTimeInNanos, - "TaskStats", + "PipelineStats", "int64_t", "totalCpuTimeInNanos"); to_json_key( j, "totalBlockedTimeInNanos", p.totalBlockedTimeInNanos, - "TaskStats", + "PipelineStats", "int64_t", "totalBlockedTimeInNanos"); to_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + j, + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); to_json_key( j, "blockedReasons", p.blockedReasons, - "TaskStats", + "PipelineStats", "List", "blockedReasons"); to_json_key( j, "totalAllocationInBytes", p.totalAllocationInBytes, - "TaskStats", + "PipelineStats", "int64_t", "totalAllocationInBytes"); to_json_key( j, "rawInputDataSizeInBytes", p.rawInputDataSizeInBytes, - "TaskStats", + "PipelineStats", "int64_t", "rawInputDataSizeInBytes"); to_json_key( j, "rawInputPositions", p.rawInputPositions, - "TaskStats", + "PipelineStats", "int64_t", "rawInputPositions"); to_json_key( j, "processedInputDataSizeInBytes", p.processedInputDataSizeInBytes, - "TaskStats", + "PipelineStats", "int64_t", "processedInputDataSizeInBytes"); to_json_key( j, "processedInputPositions", p.processedInputPositions, - "TaskStats", + "PipelineStats", "int64_t", "processedInputPositions"); to_json_key( j, "outputDataSizeInBytes", p.outputDataSizeInBytes, - "TaskStats", + "PipelineStats", "int64_t", "outputDataSizeInBytes"); to_json_key( j, "outputPositions", p.outputPositions, - "TaskStats", + "PipelineStats", "int64_t", "outputPositions"); to_json_key( j, "physicalWrittenDataSizeInBytes", p.physicalWrittenDataSizeInBytes, - "TaskStats", + "PipelineStats", "int64_t", "physicalWrittenDataSizeInBytes"); - to_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); - to_json_key( - j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", - "int64_t", - "fullGcTimeInMillis"); to_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); to_json_key( - j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } -void from_json(const json& j, TaskStats& p) { +void from_json(const json& j, PipelineStats& p) { from_json_key( - j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); + j, "pipelineId", p.pipelineId, "PipelineStats", "int", "pipelineId"); from_json_key( j, "firstStartTime", p.firstStartTime, - "TaskStats", + "PipelineStats", "DateTime", "firstStartTime"); from_json_key( j, "lastStartTime", p.lastStartTime, - "TaskStats", + "PipelineStats", "DateTime", "lastStartTime"); from_json_key( - j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); - from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + j, + "lastEndTime", + p.lastEndTime, + "PipelineStats", + "DateTime", + "lastEndTime"); from_json_key( j, - "elapsedTimeInNanos", - p.elapsedTimeInNanos, - "TaskStats", - "int64_t", - "elapsedTimeInNanos"); + "inputPipeline", + p.inputPipeline, + "PipelineStats", + "bool", + "inputPipeline"); from_json_key( j, - "queuedTimeInNanos", - p.queuedTimeInNanos, - "TaskStats", - "int64_t", - "queuedTimeInNanos"); + "outputPipeline", + p.outputPipeline, + "PipelineStats", + "bool", + "outputPipeline"); from_json_key( - j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + j, + "totalDrivers", + p.totalDrivers, + "PipelineStats", + "int", + "totalDrivers"); from_json_key( - j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + j, + "queuedDrivers", + p.queuedDrivers, + "PipelineStats", + "int", + "queuedDrivers"); from_json_key( j, "queuedPartitionedDrivers", p.queuedPartitionedDrivers, - "TaskStats", + "PipelineStats", "int", "queuedPartitionedDrivers"); from_json_key( j, "queuedPartitionedSplitsWeight", p.queuedPartitionedSplitsWeight, - "TaskStats", + "PipelineStats", "int64_t", "queuedPartitionedSplitsWeight"); from_json_key( j, "runningDrivers", p.runningDrivers, - "TaskStats", + "PipelineStats", "int", "runningDrivers"); from_json_key( j, "runningPartitionedDrivers", p.runningPartitionedDrivers, - "TaskStats", + "PipelineStats", "int", "runningPartitionedDrivers"); from_json_key( j, "runningPartitionedSplitsWeight", p.runningPartitionedSplitsWeight, - "TaskStats", + "PipelineStats", "int64_t", "runningPartitionedSplitsWeight"); from_json_key( j, "blockedDrivers", p.blockedDrivers, - "TaskStats", - "int", - "blockedDrivers"); - from_json_key( - j, - "completedDrivers", - p.completedDrivers, - "TaskStats", + "PipelineStats", "int", - "completedDrivers"); - from_json_key( - j, - "cumulativeUserMemory", - p.cumulativeUserMemory, - "TaskStats", - "double", - "cumulativeUserMemory"); - from_json_key( - j, - "cumulativeTotalMemory", - p.cumulativeTotalMemory, - "TaskStats", - "double", - "cumulativeTotalMemory"); - from_json_key( - j, - "userMemoryReservationInBytes", - p.userMemoryReservationInBytes, - "TaskStats", - "int64_t", - "userMemoryReservationInBytes"); - from_json_key( - j, - "revocableMemoryReservationInBytes", - p.revocableMemoryReservationInBytes, - "TaskStats", - "int64_t", - "revocableMemoryReservationInBytes"); - from_json_key( - j, - "systemMemoryReservationInBytes", - p.systemMemoryReservationInBytes, - "TaskStats", - "int64_t", - "systemMemoryReservationInBytes"); - from_json_key( - j, - "peakTotalMemoryInBytes", - p.peakTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakTotalMemoryInBytes"); - from_json_key( - j, - "peakUserMemoryInBytes", - p.peakUserMemoryInBytes, - "TaskStats", - "int64_t", - "peakUserMemoryInBytes"); - from_json_key( - j, - "peakNodeTotalMemoryInBytes", - p.peakNodeTotalMemoryInBytes, - "TaskStats", - "int64_t", - "peakNodeTotalMemoryInBytes"); - from_json_key( - j, - "totalScheduledTimeInNanos", - p.totalScheduledTimeInNanos, - "TaskStats", - "int64_t", - "totalScheduledTimeInNanos"); - from_json_key( - j, - "totalCpuTimeInNanos", - p.totalCpuTimeInNanos, - "TaskStats", - "int64_t", - "totalCpuTimeInNanos"); - from_json_key( - j, - "totalBlockedTimeInNanos", - p.totalBlockedTimeInNanos, - "TaskStats", - "int64_t", - "totalBlockedTimeInNanos"); - from_json_key( - j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); - from_json_key( - j, - "blockedReasons", - p.blockedReasons, - "TaskStats", - "List", - "blockedReasons"); - from_json_key( - j, - "totalAllocationInBytes", - p.totalAllocationInBytes, - "TaskStats", - "int64_t", - "totalAllocationInBytes"); + "blockedDrivers"); from_json_key( j, - "rawInputDataSizeInBytes", - p.rawInputDataSizeInBytes, - "TaskStats", - "int64_t", - "rawInputDataSizeInBytes"); + "completedDrivers", + p.completedDrivers, + "PipelineStats", + "int", + "completedDrivers"); from_json_key( j, - "rawInputPositions", - p.rawInputPositions, - "TaskStats", + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "PipelineStats", "int64_t", - "rawInputPositions"); + "userMemoryReservationInBytes"); from_json_key( j, - "processedInputDataSizeInBytes", - p.processedInputDataSizeInBytes, - "TaskStats", + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "PipelineStats", "int64_t", - "processedInputDataSizeInBytes"); + "revocableMemoryReservationInBytes"); from_json_key( j, - "processedInputPositions", - p.processedInputPositions, - "TaskStats", + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "PipelineStats", "int64_t", - "processedInputPositions"); + "systemMemoryReservationInBytes"); from_json_key( j, - "outputDataSizeInBytes", - p.outputDataSizeInBytes, - "TaskStats", - "int64_t", - "outputDataSizeInBytes"); + "queuedTime", + p.queuedTime, + "PipelineStats", + "DistributionSnapshot", + "queuedTime"); from_json_key( j, - "outputPositions", - p.outputPositions, - "TaskStats", - "int64_t", - "outputPositions"); + "elapsedTime", + p.elapsedTime, + "PipelineStats", + "DistributionSnapshot", + "elapsedTime"); from_json_key( j, - "physicalWrittenDataSizeInBytes", - p.physicalWrittenDataSizeInBytes, - "TaskStats", + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "PipelineStats", "int64_t", - "physicalWrittenDataSizeInBytes"); - from_json_key( - j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); + "totalScheduledTimeInNanos"); from_json_key( j, - "fullGcTimeInMillis", - p.fullGcTimeInMillis, - "TaskStats", + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "PipelineStats", "int64_t", - "fullGcTimeInMillis"); + "totalCpuTimeInNanos"); from_json_key( j, - "pipelines", - p.pipelines, - "TaskStats", - "List", - "pipelines"); + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "PipelineStats", + "int64_t", + "totalBlockedTimeInNanos"); from_json_key( j, - "runtimeStats", - p.runtimeStats, - "TaskStats", - "RuntimeStats", - "runtimeStats"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorTableHandle"); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + " ConnectorTableHandle ConnectorTableHandle"); - } - - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorTableHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TableHandle& p) { - j = json::object(); - to_json_key( - j, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); - to_json_key( - j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); - to_json_key( - j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); - to_json_key( - j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); -} - -void from_json(const json& j, TableHandle& p) { + "fullyBlocked", + p.fullyBlocked, + "PipelineStats", + "bool", + "fullyBlocked"); from_json_key( j, - "connectorId", - p.connectorId, - "TableHandle", - "ConnectorId", - "connectorId"); + "blockedReasons", + p.blockedReasons, + "PipelineStats", + "List", + "blockedReasons"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "TableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "totalAllocationInBytes", + p.totalAllocationInBytes, + "PipelineStats", + "int64_t", + "totalAllocationInBytes"); from_json_key( j, - "transaction", - p.transaction, - "TableHandle", - "ConnectorTransactionHandle", - "transaction"); + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "rawInputDataSizeInBytes"); from_json_key( j, - "connectorTableLayout", - p.connectorTableLayout, - "TableHandle", - "ConnectorTableLayoutHandle", - "connectorTableLayout"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -DeleteHandle::DeleteHandle() noexcept { - _type = "DeleteHandle"; -} - -void to_json(json& j, const DeleteHandle& p) { - j = json::object(); - j["@type"] = "DeleteHandle"; - to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "DeleteHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, DeleteHandle& p) { - p._type = j["@type"]; - from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + "rawInputPositions", + p.rawInputPositions, + "PipelineStats", + "int64_t", + "rawInputPositions"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "DeleteHandle", - "SchemaTableName", - "schemaTableName"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ExchangeNodeType_enum_table[] = { - // NOLINT: cert-err58-cpp - {ExchangeNodeType::GATHER, "GATHER"}, - {ExchangeNodeType::REPARTITION, "REPARTITION"}, - {ExchangeNodeType::REPLICATE, "REPLICATE"}, -}; -void to_json(json& j, const ExchangeNodeType& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ExchangeNodeType_enum_table)) - ? it - : std::begin(ExchangeNodeType_enum_table)) - ->second; -} -void from_json(const json& j, ExchangeNodeType& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeType_enum_table), - std::end(ExchangeNodeType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ExchangeNodeType_enum_table)) - ? it - : std::begin(ExchangeNodeType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair ExchangeNodeScope_enum_table[] = - { // NOLINT: cert-err58-cpp - {ExchangeNodeScope::LOCAL, "LOCAL"}, - {ExchangeNodeScope::REMOTE_STREAMING, "REMOTE_STREAMING"}, - {ExchangeNodeScope::REMOTE_MATERIALIZED, "REMOTE_MATERIALIZED"}}; -void to_json(json& j, const ExchangeNodeScope& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeScope must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeScope_enum_table), - std::end(ExchangeNodeScope_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(ExchangeNodeScope_enum_table)) - ? it - : std::begin(ExchangeNodeScope_enum_table)) - ->second; -} -void from_json(const json& j, ExchangeNodeScope& e) { - static_assert( - std::is_enum::value, - "ExchangeNodeScope must be an enum!"); - const auto* it = std::find_if( - std::begin(ExchangeNodeScope_enum_table), - std::end(ExchangeNodeScope_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(ExchangeNodeScope_enum_table)) - ? it - : std::begin(ExchangeNodeScope_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "$remote") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); - return; - } - if (getConnectorKey(type) == "tpch") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError(std::string(e.what()) + " ConnectorPartitioningHandle"); - } - - if (type == "$remote") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "hive") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - if (getConnectorKey(type) == "tpch") { - auto k = std::make_shared(); - j.get_to(*k); - p = k; - return; - } - - throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PartitioningHandle& p) { - j = json::object(); - to_json_key( + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "PipelineStats", + "int64_t", + "processedInputDataSizeInBytes"); + from_json_key( j, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); - to_json_key( + "processedInputPositions", + p.processedInputPositions, + "PipelineStats", + "int64_t", + "processedInputPositions"); + from_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - to_json_key( + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "PipelineStats", + "int64_t", + "outputDataSizeInBytes"); + from_json_key( j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); -} - -void from_json(const json& j, PartitioningHandle& p) { + "outputPositions", + p.outputPositions, + "PipelineStats", + "int64_t", + "outputPositions"); from_json_key( j, - "connectorId", - p.connectorId, - "PartitioningHandle", - "ConnectorId", - "connectorId"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "PipelineStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "PartitioningHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "operatorSummaries", + p.operatorSummaries, + "PipelineStats", + "List", + "operatorSummaries"); from_json_key( - j, - "connectorHandle", - p.connectorHandle, - "PartitioningHandle", - "ConnectorPartitioningHandle", - "connectorHandle"); + j, "drivers", p.drivers, "PipelineStats", "List", "drivers"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Partitioning& p) { +void to_json(json& j, const TaskStats& p) { j = json::object(); to_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); to_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); -} - -void from_json(const json& j, Partitioning& p) { - from_json_key( - j, "handle", p.handle, "Partitioning", "PartitioningHandle", "handle"); - from_json_key( + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); + to_json_key( j, - "arguments", - p.arguments, - "Partitioning", - "List>", - "arguments"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PartitioningScheme& p) { - j = json::object(); + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); + to_json_key( + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + to_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); + to_json_key( + j, + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", + "int64_t", + "elapsedTimeInNanos"); + to_json_key( + j, + "queuedTimeInNanos", + p.queuedTimeInNanos, + "TaskStats", + "int64_t", + "queuedTimeInNanos"); + to_json_key( + j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + to_json_key( + j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); + to_json_key( + j, + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", + "int", + "queuedPartitionedDrivers"); + to_json_key( + j, + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "queuedPartitionedSplitsWeight"); + to_json_key( + j, + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); + to_json_key( + j, + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); + to_json_key( + j, + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "runningPartitionedSplitsWeight"); + to_json_key( + j, + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); + to_json_key( + j, + "completedDrivers", + p.completedDrivers, + "TaskStats", + "int", + "completedDrivers"); + to_json_key( + j, + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); + to_json_key( + j, + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); + to_json_key( + j, + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", + "int64_t", + "userMemoryReservationInBytes"); + to_json_key( + j, + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", + "int64_t", + "revocableMemoryReservationInBytes"); + to_json_key( + j, + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", + "int64_t", + "systemMemoryReservationInBytes"); + to_json_key( + j, + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); + to_json_key( + j, + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); + to_json_key( + j, + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); + to_json_key( + j, + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", + "int64_t", + "totalScheduledTimeInNanos"); + to_json_key( + j, + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", + "int64_t", + "totalCpuTimeInNanos"); + to_json_key( + j, + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", + "int64_t", + "totalBlockedTimeInNanos"); + to_json_key( + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + to_json_key( + j, + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); + to_json_key( + j, + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", + "int64_t", + "totalAllocationInBytes"); + to_json_key( + j, + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", + "int64_t", + "rawInputDataSizeInBytes"); + to_json_key( + j, + "rawInputPositions", + p.rawInputPositions, + "TaskStats", + "int64_t", + "rawInputPositions"); + to_json_key( + j, + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", + "int64_t", + "processedInputDataSizeInBytes"); + to_json_key( + j, + "processedInputPositions", + p.processedInputPositions, + "TaskStats", + "int64_t", + "processedInputPositions"); to_json_key( j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", + "int64_t", + "outputDataSizeInBytes"); to_json_key( j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + "outputPositions", + p.outputPositions, + "TaskStats", + "int64_t", + "outputPositions"); to_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + to_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); to_json_key( j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", + "int64_t", + "fullGcTimeInMillis"); to_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); + to_json_key( + j, + "runtimeStats", + p.runtimeStats, + "TaskStats", + "RuntimeStats", + "runtimeStats"); } -void from_json(const json& j, PartitioningScheme& p) { +void from_json(const json& j, TaskStats& p) { from_json_key( - j, - "partitioning", - p.partitioning, - "PartitioningScheme", - "Partitioning", - "partitioning"); + j, "createTime", p.createTime, "TaskStats", "DateTime", "createTime"); from_json_key( j, - "outputLayout", - p.outputLayout, - "PartitioningScheme", - "List", - "outputLayout"); + "firstStartTime", + p.firstStartTime, + "TaskStats", + "DateTime", + "firstStartTime"); from_json_key( j, - "hashColumn", - p.hashColumn, - "PartitioningScheme", - "VariableReferenceExpression", - "hashColumn"); + "lastStartTime", + p.lastStartTime, + "TaskStats", + "DateTime", + "lastStartTime"); from_json_key( - j, - "replicateNullsAndAny", - p.replicateNullsAndAny, - "PartitioningScheme", - "bool", - "replicateNullsAndAny"); + j, "lastEndTime", p.lastEndTime, "TaskStats", "DateTime", "lastEndTime"); + from_json_key(j, "endTime", p.endTime, "TaskStats", "DateTime", "endTime"); from_json_key( j, - "bucketToPartition", - p.bucketToPartition, - "PartitioningScheme", - "List", - "bucketToPartition"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -ExchangeNode::ExchangeNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.ExchangeNode"; -} - -void to_json(json& j, const ExchangeNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.ExchangeNode"; - to_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); - to_json_key( - j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); - to_json_key( - j, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( - j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); - to_json_key( - j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); - to_json_key( - j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); - to_json_key( + "elapsedTimeInNanos", + p.elapsedTimeInNanos, + "TaskStats", + "int64_t", + "elapsedTimeInNanos"); + from_json_key( j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); -} - -void from_json(const json& j, ExchangeNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "ExchangeNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "ExchangeNode", "ExchangeNodeType", "type"); + "queuedTimeInNanos", + p.queuedTimeInNanos, + "TaskStats", + "int64_t", + "queuedTimeInNanos"); from_json_key( - j, "scope", p.scope, "ExchangeNode", "ExchangeNodeScope", "scope"); + j, "totalDrivers", p.totalDrivers, "TaskStats", "int", "totalDrivers"); + from_json_key( + j, "queuedDrivers", p.queuedDrivers, "TaskStats", "int", "queuedDrivers"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "ExchangeNode", - "PartitioningScheme", - "partitioningScheme"); + "queuedPartitionedDrivers", + p.queuedPartitionedDrivers, + "TaskStats", + "int", + "queuedPartitionedDrivers"); from_json_key( j, - "sources", - p.sources, - "ExchangeNode", - "List>", - "sources"); + "queuedPartitionedSplitsWeight", + p.queuedPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "queuedPartitionedSplitsWeight"); from_json_key( j, - "inputs", - p.inputs, - "ExchangeNode", - "List>", - "inputs"); + "runningDrivers", + p.runningDrivers, + "TaskStats", + "int", + "runningDrivers"); from_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "ExchangeNode", - "bool", - "ensureSourceOrdering"); + "runningPartitionedDrivers", + p.runningPartitionedDrivers, + "TaskStats", + "int", + "runningPartitionedDrivers"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "ExchangeNode", - "OrderingScheme", - "orderingScheme"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -RemoteSourceNode::RemoteSourceNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; -} - -void to_json(json& j, const RemoteSourceNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RemoteSourceNode"; - to_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); - to_json_key( + "runningPartitionedSplitsWeight", + p.runningPartitionedSplitsWeight, + "TaskStats", + "int64_t", + "runningPartitionedSplitsWeight"); + from_json_key( j, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); - to_json_key( + "blockedDrivers", + p.blockedDrivers, + "TaskStats", + "int", + "blockedDrivers"); + from_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); - to_json_key( + "completedDrivers", + p.completedDrivers, + "TaskStats", + "int", + "completedDrivers"); + from_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); - to_json_key( + "cumulativeUserMemory", + p.cumulativeUserMemory, + "TaskStats", + "double", + "cumulativeUserMemory"); + from_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); - to_json_key( + "cumulativeTotalMemory", + p.cumulativeTotalMemory, + "TaskStats", + "double", + "cumulativeTotalMemory"); + from_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); -} - -void from_json(const json& j, RemoteSourceNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "RemoteSourceNode", "PlanNodeId", "id"); + "userMemoryReservationInBytes", + p.userMemoryReservationInBytes, + "TaskStats", + "int64_t", + "userMemoryReservationInBytes"); from_json_key( j, - "sourceFragmentIds", - p.sourceFragmentIds, - "RemoteSourceNode", - "List", - "sourceFragmentIds"); + "revocableMemoryReservationInBytes", + p.revocableMemoryReservationInBytes, + "TaskStats", + "int64_t", + "revocableMemoryReservationInBytes"); from_json_key( j, - "outputVariables", - p.outputVariables, - "RemoteSourceNode", - "List", - "outputVariables"); + "systemMemoryReservationInBytes", + p.systemMemoryReservationInBytes, + "TaskStats", + "int64_t", + "systemMemoryReservationInBytes"); from_json_key( j, - "ensureSourceOrdering", - p.ensureSourceOrdering, - "RemoteSourceNode", - "bool", - "ensureSourceOrdering"); + "peakTotalMemoryInBytes", + p.peakTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakTotalMemoryInBytes"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "RemoteSourceNode", - "OrderingScheme", - "orderingScheme"); + "peakUserMemoryInBytes", + p.peakUserMemoryInBytes, + "TaskStats", + "int64_t", + "peakUserMemoryInBytes"); from_json_key( j, - "exchangeType", - p.exchangeType, - "RemoteSourceNode", - "ExchangeNodeType", - "exchangeType"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -FilterNode::FilterNode() noexcept { - _type = ".FilterNode"; -} - -void to_json(json& j, const FilterNode& p) { - j = json::object(); - j["@type"] = ".FilterNode"; - to_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); - to_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); -} - -void from_json(const json& j, FilterNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); + "peakNodeTotalMemoryInBytes", + p.peakNodeTotalMemoryInBytes, + "TaskStats", + "int64_t", + "peakNodeTotalMemoryInBytes"); from_json_key( - j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair DistributionType_enum_table[] = - { // NOLINT: cert-err58-cpp - {DistributionType::PARTITIONED, "PARTITIONED"}, - {DistributionType::REPLICATED, "REPLICATED"}}; -void to_json(json& j, const DistributionType& e) { - static_assert( - std::is_enum::value, - "DistributionType must be an enum!"); - const auto* it = std::find_if( - std::begin(DistributionType_enum_table), - std::end(DistributionType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(DistributionType_enum_table)) - ? it - : std::begin(DistributionType_enum_table)) - ->second; -} -void from_json(const json& j, DistributionType& e) { - static_assert( - std::is_enum::value, - "DistributionType must be an enum!"); - const auto* it = std::find_if( - std::begin(DistributionType_enum_table), - std::end(DistributionType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(DistributionType_enum_table)) - ? it - : std::begin(DistributionType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair JoinNodeType_enum_table[] = - { // NOLINT: cert-err58-cpp - {JoinNodeType::INNER, "INNER"}, - {JoinNodeType::LEFT, "LEFT"}, - {JoinNodeType::RIGHT, "RIGHT"}, - {JoinNodeType::FULL, "FULL"}}; -void to_json(json& j, const JoinNodeType& e) { - static_assert( - std::is_enum::value, "JoinNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(JoinNodeType_enum_table), - std::end(JoinNodeType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(JoinNodeType_enum_table)) - ? it - : std::begin(JoinNodeType_enum_table)) - ->second; -} -void from_json(const json& j, JoinNodeType& e) { - static_assert( - std::is_enum::value, "JoinNodeType must be an enum!"); - const auto* it = std::find_if( - std::begin(JoinNodeType_enum_table), - std::end(JoinNodeType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(JoinNodeType_enum_table)) - ? it - : std::begin(JoinNodeType_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -JoinNode::JoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.JoinNode"; -} - -void to_json(json& j, const JoinNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.JoinNode"; - to_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); - to_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); - to_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); - to_json_key( j, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); - to_json_key( + "totalScheduledTimeInNanos", + p.totalScheduledTimeInNanos, + "TaskStats", + "int64_t", + "totalScheduledTimeInNanos"); + from_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); - to_json_key( + "totalCpuTimeInNanos", + p.totalCpuTimeInNanos, + "TaskStats", + "int64_t", + "totalCpuTimeInNanos"); + from_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); - to_json_key( + "totalBlockedTimeInNanos", + p.totalBlockedTimeInNanos, + "TaskStats", + "int64_t", + "totalBlockedTimeInNanos"); + from_json_key( + j, "fullyBlocked", p.fullyBlocked, "TaskStats", "bool", "fullyBlocked"); + from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); - to_json_key( + "blockedReasons", + p.blockedReasons, + "TaskStats", + "List", + "blockedReasons"); + from_json_key( + j, + "totalAllocationInBytes", + p.totalAllocationInBytes, + "TaskStats", + "int64_t", + "totalAllocationInBytes"); + from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); - to_json_key( + "rawInputDataSizeInBytes", + p.rawInputDataSizeInBytes, + "TaskStats", + "int64_t", + "rawInputDataSizeInBytes"); + from_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "DistributionType", - "distributionType"); - to_json_key( + "rawInputPositions", + p.rawInputPositions, + "TaskStats", + "int64_t", + "rawInputPositions"); + from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); -} - -void from_json(const json& j, JoinNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); - from_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); - from_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); + "processedInputDataSizeInBytes", + p.processedInputDataSizeInBytes, + "TaskStats", + "int64_t", + "processedInputDataSizeInBytes"); from_json_key( j, - "criteria", - p.criteria, - "JoinNode", - "List", - "criteria"); + "processedInputPositions", + p.processedInputPositions, + "TaskStats", + "int64_t", + "processedInputPositions"); from_json_key( j, - "outputVariables", - p.outputVariables, - "JoinNode", - "List", - "outputVariables"); + "outputDataSizeInBytes", + p.outputDataSizeInBytes, + "TaskStats", + "int64_t", + "outputDataSizeInBytes"); from_json_key( j, - "filter", - p.filter, - "JoinNode", - "std::shared_ptr", - "filter"); + "outputPositions", + p.outputPositions, + "TaskStats", + "int64_t", + "outputPositions"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "JoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "physicalWrittenDataSizeInBytes", + p.physicalWrittenDataSizeInBytes, + "TaskStats", + "int64_t", + "physicalWrittenDataSizeInBytes"); + from_json_key( + j, "fullGcCount", p.fullGcCount, "TaskStats", "int", "fullGcCount"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "JoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "fullGcTimeInMillis", + p.fullGcTimeInMillis, + "TaskStats", + "int64_t", + "fullGcTimeInMillis"); from_json_key( j, - "distributionType", - p.distributionType, - "JoinNode", - "DistributionType", - "distributionType"); + "pipelines", + p.pipelines, + "TaskStats", + "List", + "pipelines"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "JoinNode", - "Map", - "dynamicFilters"); + "runtimeStats", + p.runtimeStats, + "TaskStats", + "RuntimeStats", + "runtimeStats"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -MergeJoinNode::MergeJoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); } -void to_json(json& j, const MergeJoinNode& p) { +void from_json( + const json& j, + std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError(std::string(e.what()) + " ConnectorMetadataUpdateHandle"); + } + + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorMetadataUpdateHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MetadataUpdates& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; - to_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); - to_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); - to_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); - to_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); to_json_key( j, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); to_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} + +void from_json(const json& j, MetadataUpdates& p) { + from_json_key( + j, + "connectorId", + p.connectorId, + "MetadataUpdates", + "ConnectorId", + "connectorId"); + from_json_key( + j, + "metadataUpdates", + p.metadataUpdates, + "MetadataUpdates", + "List>", + "metadataUpdates"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskInfo& p) { + j = json::object(); + to_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); + to_json_key( + j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); to_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); to_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); to_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); + "noMoreSplits", + p.noMoreSplits, + "TaskInfo", + "List", + "noMoreSplits"); + to_json_key(j, "stats", p.stats, "TaskInfo", "TaskStats", "stats"); + to_json_key(j, "needsPlan", p.needsPlan, "TaskInfo", "bool", "needsPlan"); + to_json_key( + j, + "metadataUpdates", + p.metadataUpdates, + "TaskInfo", + "MetadataUpdates", + "metadataUpdates"); + to_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); } -void from_json(const json& j, MergeJoinNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); - from_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); - from_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); - from_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); +void from_json(const json& j, TaskInfo& p) { + from_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); from_json_key( - j, - "criteria", - p.criteria, - "MergeJoinNode", - "List", - "criteria"); + j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); from_json_key( j, - "outputVariables", - p.outputVariables, - "MergeJoinNode", - "List", - "outputVariables"); + "lastHeartbeat", + p.lastHeartbeat, + "TaskInfo", + "DateTime", + "lastHeartbeat"); from_json_key( j, - "filter", - p.filter, - "MergeJoinNode", - "std::shared_ptr", - "filter"); + "outputBuffers", + p.outputBuffers, + "TaskInfo", + "OutputBufferInfo", + "outputBuffers"); from_json_key( j, - "leftHashVariable", - p.leftHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "leftHashVariable"); + "noMoreSplits", + p.noMoreSplits, + "TaskInfo", + "List", + "noMoreSplits"); + from_json_key(j, "stats", p.stats, "TaskInfo", "TaskStats", "stats"); + from_json_key(j, "needsPlan", p.needsPlan, "TaskInfo", "bool", "needsPlan"); from_json_key( j, - "rightHashVariable", - p.rightHashVariable, - "MergeJoinNode", - "VariableReferenceExpression", - "rightHashVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - SystemPartitioning_enum_table[] = { // NOLINT: cert-err58-cpp - {SystemPartitioning::SINGLE, "SINGLE"}, - {SystemPartitioning::FIXED, "FIXED"}, - {SystemPartitioning::SOURCE, "SOURCE"}, - {SystemPartitioning::SCALED, "SCALED"}, - {SystemPartitioning::COORDINATOR_ONLY, "COORDINATOR_ONLY"}, - {SystemPartitioning::ARBITRARY, "ARBITRARY"}}; -void to_json(json& j, const SystemPartitioning& e) { - static_assert( - std::is_enum::value, - "SystemPartitioning must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitioning_enum_table), - std::end(SystemPartitioning_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(SystemPartitioning_enum_table)) - ? it - : std::begin(SystemPartitioning_enum_table)) - ->second; -} -void from_json(const json& j, SystemPartitioning& e) { - static_assert( - std::is_enum::value, - "SystemPartitioning must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitioning_enum_table), - std::end(SystemPartitioning_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(SystemPartitioning_enum_table)) - ? it - : std::begin(SystemPartitioning_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - SystemPartitionFunction_enum_table[] = { // NOLINT: cert-err58-cpp - {SystemPartitionFunction::SINGLE, "SINGLE"}, - {SystemPartitionFunction::HASH, "HASH"}, - {SystemPartitionFunction::ROUND_ROBIN, "ROUND_ROBIN"}, - {SystemPartitionFunction::BROADCAST, "BROADCAST"}, - {SystemPartitionFunction::UNKNOWN, "UNKNOWN"}}; -void to_json(json& j, const SystemPartitionFunction& e) { - static_assert( - std::is_enum::value, - "SystemPartitionFunction must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitionFunction_enum_table), - std::end(SystemPartitionFunction_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(SystemPartitionFunction_enum_table)) - ? it - : std::begin(SystemPartitionFunction_enum_table)) - ->second; -} -void from_json(const json& j, SystemPartitionFunction& e) { - static_assert( - std::is_enum::value, - "SystemPartitionFunction must be an enum!"); - const auto* it = std::find_if( - std::begin(SystemPartitionFunction_enum_table), - std::end(SystemPartitionFunction_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(SystemPartitionFunction_enum_table)) - ? it - : std::begin(SystemPartitionFunction_enum_table)) - ->first; + "metadataUpdates", + p.metadataUpdates, + "TaskInfo", + "MetadataUpdates", + "metadataUpdates"); + from_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SystemPartitioningHandle::SystemPartitioningHandle() noexcept { +RemoteTransactionHandle::RemoteTransactionHandle() noexcept { _type = "$remote"; } -void to_json(json& j, const SystemPartitioningHandle& p) { +void to_json(json& j, const RemoteTransactionHandle& p) { j = json::object(); j["@type"] = "$remote"; to_json_key( - j, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); - to_json_key( - j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } -void from_json(const json& j, SystemPartitioningHandle& p) { +void from_json(const json& j, RemoteTransactionHandle& p) { p._type = j["@type"]; from_json_key( - j, - "partitioning", - p.partitioning, - "SystemPartitioningHandle", - "SystemPartitioning", - "partitioning"); - from_json_key( - j, - "function", - p.function, - "SystemPartitioningHandle", - "SystemPartitionFunction", - "function"); + j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); } } // namespace facebook::presto::protocol /* @@ -6205,467 +5452,754 @@ void from_json(const json& j, SystemPartitioningHandle& p) { * limitations under the License. */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { +void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { return; } String type = p->_type; if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } -void from_json(const json& j, std::shared_ptr& p) { +void from_json(const json& j, std::shared_ptr& p) { String type; try { type = p->getSubclassKey(j); } catch (json::parse_error& e) { throw ParseError( std::string(e.what()) + - " ConnectorOutputTableHandle ConnectorOutputTableHandle"); + " ConnectorTableLayoutHandle ConnectorTableLayoutHandle"); + } + + if (getConnectorKey(type) == "hive") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorTableHandle"); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + " ConnectorTableHandle ConnectorTableHandle"); } if (getConnectorKey(type) == "hive") { - std::shared_ptr k = - std::make_shared(); + auto k = std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = k; return; } - throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + + throw TypeError(type + " no abstract type ConnectorTableHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const OutputTableHandle& p) { +void to_json(json& j, const TableHandle& p) { j = json::object(); to_json_key( j, "connectorId", p.connectorId, - "OutputTableHandle", + "TableHandle", "ConnectorId", "connectorId"); - to_json_key( - j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); to_json_key( j, "connectorHandle", p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", + "TableHandle", + "ConnectorTableHandle", "connectorHandle"); + to_json_key( + j, + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); + to_json_key( + j, + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } -void from_json(const json& j, OutputTableHandle& p) { +void from_json(const json& j, TableHandle& p) { from_json_key( j, "connectorId", p.connectorId, - "OutputTableHandle", + "TableHandle", "ConnectorId", "connectorId"); - from_json_key( - j, - "transactionHandle", - p.transactionHandle, - "OutputTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); from_json_key( j, "connectorHandle", p.connectorHandle, - "OutputTableHandle", - "ConnectorOutputTableHandle", + "TableHandle", + "ConnectorTableHandle", "connectorHandle"); + from_json_key( + j, + "transaction", + p.transaction, + "TableHandle", + "ConnectorTransactionHandle", + "transaction"); + from_json_key( + j, + "connectorTableLayout", + p.connectorTableLayout, + "TableHandle", + "ConnectorTableLayoutHandle", + "connectorTableLayout"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -CreateHandle::CreateHandle() noexcept { - _type = "CreateHandle"; -} -void to_json(json& j, const CreateHandle& p) { +void to_json(json& j, const DeleteScanInfo& p) { j = json::object(); - j["@type"] = "CreateHandle"; - to_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } -void from_json(const json& j, CreateHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); +void from_json(const json& j, DeleteScanInfo& p) { + from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "CreateHandle", - "SchemaTableName", - "schemaTableName"); + "tableHandle", + p.tableHandle, + "DeleteScanInfo", + "TableHandle", + "tableHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const BucketConversion& p) { +void to_json(json& j, const Language& p) { + j = json::object(); + to_json_key(j, "language", p.language, "Language", "String", "language"); +} + +void from_json(const json& j, Language& p) { + from_json_key(j, "language", p.language, "Language", "String", "language"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair NullCallClause_enum_table[] = + { // NOLINT: cert-err58-cpp + {NullCallClause::RETURNS_NULL_ON_NULL_INPUT, + "RETURNS_NULL_ON_NULL_INPUT"}, + {NullCallClause::CALLED_ON_NULL_INPUT, "CALLED_ON_NULL_INPUT"}}; +void to_json(json& j, const NullCallClause& e) { + static_assert( + std::is_enum::value, "NullCallClause must be an enum!"); + const auto* it = std::find_if( + std::begin(NullCallClause_enum_table), + std::end(NullCallClause_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(NullCallClause_enum_table)) + ? it + : std::begin(NullCallClause_enum_table)) + ->second; +} +void from_json(const json& j, NullCallClause& e) { + static_assert( + std::is_enum::value, "NullCallClause must be an enum!"); + const auto* it = std::find_if( + std::begin(NullCallClause_enum_table), + std::end(NullCallClause_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(NullCallClause_enum_table)) + ? it + : std::begin(NullCallClause_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Determinism_enum_table[] = { + // NOLINT: cert-err58-cpp + {Determinism::DETERMINISTIC, "DETERMINISTIC"}, + {Determinism::NOT_DETERMINISTIC, "NOT_DETERMINISTIC"}, +}; +void to_json(json& j, const Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->second; +} +void from_json(const json& j, Determinism& e) { + static_assert( + std::is_enum::value, "Determinism must be an enum!"); + const auto* it = std::find_if( + std::begin(Determinism_enum_table), + std::end(Determinism_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Determinism_enum_table)) + ? it + : std::begin(Determinism_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const RoutineCharacteristics& p) { j = json::object(); to_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); to_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); to_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } -void from_json(const json& j, BucketConversion& p) { - from_json_key( - j, - "tableBucketCount", - p.tableBucketCount, - "BucketConversion", - "int", - "tableBucketCount"); +void from_json(const json& j, RoutineCharacteristics& p) { from_json_key( j, - "partitionBucketCount", - p.partitionBucketCount, - "BucketConversion", - "int", - "partitionBucketCount"); + "language", + p.language, + "RoutineCharacteristics", + "Language", + "language"); from_json_key( j, - "bucketColumnHandles", - p.bucketColumnHandles, - "BucketConversion", - "List", - "bucketColumnHandles"); + "determinism", + p.determinism, + "RoutineCharacteristics", + "Determinism", + "determinism"); + from_json_key( + j, + "nullCallClause", + p.nullCallClause, + "RoutineCharacteristics", + "NullCallClause", + "nullCallClause"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveOutputTableHandle::HiveOutputTableHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveOutputTableHandle& p) { +void to_json(json& j, const MemoryAllocation& p) { j = json::object(); - j["@type"] = "hive"; + to_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); to_json_key( j, - "schemaName", - p.schemaName, - "HiveOutputTableHandle", - "String", - "schemaName"); - to_json_key( + "allocation", + p.allocation, + "MemoryAllocation", + "int64_t", + "allocation"); +} + +void from_json(const json& j, MemoryAllocation& p) { + from_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); + from_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); + "allocation", + p.allocation, + "MemoryAllocation", + "int64_t", + "allocation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MemoryPoolInfo& p) { + j = json::object(); to_json_key( - j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); to_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); to_json_key( j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); to_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); - to_json_key( + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); +} + +void from_json(const json& j, MemoryPoolInfo& p) { + from_json_key( + j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); + from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); - to_json_key( + "reservedBytes", + p.reservedBytes, + "MemoryPoolInfo", + "int64_t", + "reservedBytes"); + from_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); - to_json_key( + "reservedRevocableBytes", + p.reservedRevocableBytes, + "MemoryPoolInfo", + "int64_t", + "reservedRevocableBytes"); + from_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); - to_json_key( + "queryMemoryReservations", + p.queryMemoryReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryReservations"); + from_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); - to_json_key( + "queryMemoryAllocations", + p.queryMemoryAllocations, + "MemoryPoolInfo", + "Map>", + "queryMemoryAllocations"); + from_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + "queryMemoryRevocableReservations", + p.queryMemoryRevocableReservations, + "MemoryPoolInfo", + "Map", + "queryMemoryRevocableReservations"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const MemoryInfo& p) { + j = json::object(); to_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); to_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); } -void from_json(const json& j, HiveOutputTableHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, MemoryInfo& p) { from_json_key( j, - "schemaName", - p.schemaName, - "HiveOutputTableHandle", - "String", - "schemaName"); + "totalNodeMemory", + p.totalNodeMemory, + "MemoryInfo", + "DataSize", + "totalNodeMemory"); from_json_key( j, - "tableName", - p.tableName, - "HiveOutputTableHandle", - "String", - "tableName"); - from_json_key( + "pools", + p.pools, + "MemoryInfo", + "Map", + "pools"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const PlanCostEstimate& p) { + j = json::object(); + to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + to_json_key( + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + to_json_key( j, - "inputColumns", - p.inputColumns, - "HiveOutputTableHandle", - "List", - "inputColumns"); - from_json_key( + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); + to_json_key( j, - "pageSinkMetadata", - p.pageSinkMetadata, - "HiveOutputTableHandle", - "HivePageSinkMetadata", - "pageSinkMetadata"); + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); +} + +void from_json(const json& j, PlanCostEstimate& p) { + from_json_key( + j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + from_json_key( + j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); from_json_key( j, - "locationHandle", - p.locationHandle, - "HiveOutputTableHandle", - "LocationHandle", - "locationHandle"); + "maxMemoryWhenOutputting", + p.maxMemoryWhenOutputting, + "PlanCostEstimate", + "double", + "maxMemoryWhenOutputting"); from_json_key( j, - "tableStorageFormat", - p.tableStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "tableStorageFormat"); - from_json_key( + "networkCost", + p.networkCost, + "PlanCostEstimate", + "double", + "networkCost"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const StatsAndCosts& p) { + j = json::object(); + to_json_key( j, - "partitionStorageFormat", - p.partitionStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "partitionStorageFormat"); - from_json_key( + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); + to_json_key( j, - "actualStorageFormat", - p.actualStorageFormat, - "HiveOutputTableHandle", - "HiveStorageFormat", - "actualStorageFormat"); + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); +} + +void from_json(const json& j, StatsAndCosts& p) { from_json_key( j, - "compressionCodec", - p.compressionCodec, - "HiveOutputTableHandle", - "HiveCompressionCodec", - "compressionCodec"); + "stats", + p.stats, + "StatsAndCosts", + "Map", + "stats"); from_json_key( j, - "partitionedBy", - p.partitionedBy, - "HiveOutputTableHandle", - "List", - "partitionedBy"); - from_json_key( + "costs", + p.costs, + "StatsAndCosts", + "Map", + "costs"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const SchemaTableName& p) { + j = json::object(); + to_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); + to_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); +} + +void from_json(const json& j, SchemaTableName& p) { + from_json_key(j, "schema", p.schema, "SchemaTableName", "String", "schema"); + from_json_key(j, "table", p.table, "SchemaTableName", "String", "table"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorInsertTableHandle ConnectorInsertTableHandle"); + } + + if (getConnectorKey(type) == "hive") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const InsertTableHandle& p) { + j = json::object(); + to_json_key( j, - "bucketProperty", - p.bucketProperty, - "HiveOutputTableHandle", - "HiveBucketProperty", - "bucketProperty"); - from_json_key( + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); + to_json_key( j, - "preferredOrderingColumns", - p.preferredOrderingColumns, - "HiveOutputTableHandle", - "List", - "preferredOrderingColumns"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( + j, + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); +} + +void from_json(const json& j, InsertTableHandle& p) { from_json_key( j, - "tableOwner", - p.tableOwner, - "HiveOutputTableHandle", - "String", - "tableOwner"); + "connectorId", + p.connectorId, + "InsertTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "additionalTableParameters", - p.additionalTableParameters, - "HiveOutputTableHandle", - "Map", - "additionalTableParameters"); + "transactionHandle", + p.transactionHandle, + "InsertTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "encryptionInformation", - p.encryptionInformation, - "HiveOutputTableHandle", - "EncryptionInformation", - "encryptionInformation"); + "connectorHandle", + p.connectorHandle, + "InsertTableHandle", + "ConnectorInsertTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AllOrNoneValueSet::AllOrNoneValueSet() noexcept { - _type = "allOrNone"; +InsertHandle::InsertHandle() noexcept { + _type = "InsertHandle"; } -void to_json(json& j, const AllOrNoneValueSet& p) { +void to_json(json& j, const InsertHandle& p) { j = json::object(); - j["@type"] = "allOrNone"; - to_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); - to_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); + j["@type"] = "InsertHandle"; + to_json_key( + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, AllOrNoneValueSet& p) { +void from_json(const json& j, InsertHandle& p) { p._type = j["@type"]; - from_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); - from_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); + from_json_key( + j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "InsertHandle", + "SchemaTableName", + "schemaTableName"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Order_enum_table[] = + { // NOLINT: cert-err58-cpp + {Order::ASCENDING, "ASCENDING"}, + {Order::DESCENDING, "DESCENDING"}}; +void to_json(json& j, const Order& e) { + static_assert(std::is_enum::value, "Order must be an enum!"); + const auto* it = std::find_if( + std::begin(Order_enum_table), + std::end(Order_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Order_enum_table)) ? it : std::begin(Order_enum_table)) + ->second; +} +void from_json(const json& j, Order& e) { + static_assert(std::is_enum::value, "Order must be an enum!"); + const auto* it = std::find_if( + std::begin(Order_enum_table), + std::end(Order_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Order_enum_table)) ? it : std::begin(Order_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -DistinctLimitNode::DistinctLimitNode() noexcept { - _type = ".DistinctLimitNode"; -} -void to_json(json& j, const DistinctLimitNode& p) { +void to_json(json& j, const SortingColumn& p) { j = json::object(); - j["@type"] = ".DistinctLimitNode"; - to_json_key(j, "id", p.id, "DistinctLimitNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); - to_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); - to_json_key(j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); - to_json_key( - j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); to_json_key( - j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); + j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); + to_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); } -void from_json(const json& j, DistinctLimitNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "DistinctLimitNode", "PlanNodeId", "id"); - from_json_key( - j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); - from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); - from_json_key( - j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); - from_json_key( - j, - "distinctVariables", - p.distinctVariables, - "DistinctLimitNode", - "List", - "distinctVariables"); +void from_json(const json& j, SortingColumn& p) { from_json_key( - j, - "hashVariable", - p.hashVariable, - "DistinctLimitNode", - "VariableReferenceExpression", - "hashVariable"); + j, "columnName", p.columnName, "SortingColumn", "String", "columnName"); + from_json_key(j, "order", p.order, "SortingColumn", "Order", "order"); } } // namespace facebook::presto::protocol /* @@ -6727,2031 +6261,2621 @@ void from_json(const json& j, std::shared_ptr& p) { } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +TableScanNode::TableScanNode() noexcept { + _type = ".TableScanNode"; +} -void to_json(json& j, const HiveBucketHandle& p) { +void to_json(json& j, const TableScanNode& p) { j = json::object(); + j["@type"] = ".TableScanNode"; + to_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + to_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); to_json_key( j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); to_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); +} + +void from_json(const json& j, TableScanNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); + from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); + from_json_key( + j, + "outputVariables", + p.outputVariables, + "TableScanNode", + "List", + "outputVariables"); + from_json_key( + j, + "assignments", + p.assignments, + "TableScanNode", + "Map>", + "assignments"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +FilterNode::FilterNode() noexcept { + _type = ".FilterNode"; +} + +void to_json(json& j, const FilterNode& p) { + j = json::object(); + j["@type"] = ".FilterNode"; + to_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); + to_json_key( + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); +} + +void from_json(const json& j, FilterNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "FilterNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "FilterNode", "PlanNode", "source"); + from_json_key( + j, "predicate", p.predicate, "FilterNode", "RowExpression", "predicate"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (type == "CreateHandle") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "InsertHandle") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "DeleteHandle") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ExecutionWriterTarget "); +} + +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ExecutionWriterTarget ExecutionWriterTarget"); + } + + if (type == "CreateHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "InsertHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "DeleteHandle") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ExecutionWriterTarget "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const AnalyzeTableHandle& p) { + j = json::object(); + to_json_key( + j, + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); + to_json_key( + j, + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + to_json_key( + j, + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); +} + +void from_json(const json& j, AnalyzeTableHandle& p) { + from_json_key( + j, + "connectorId", + p.connectorId, + "AnalyzeTableHandle", + "ConnectorId", + "connectorId"); + from_json_key( + j, + "transactionHandle", + p.transactionHandle, + "AnalyzeTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); + from_json_key( + j, + "connectorHandle", + p.connectorHandle, + "AnalyzeTableHandle", + "ConnectorTableHandle", + "connectorHandle"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TableWriteInfo& p) { + j = json::object(); + to_json_key( + j, + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); + to_json_key( + j, + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); to_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } -void from_json(const json& j, HiveBucketHandle& p) { +void from_json(const json& j, TableWriteInfo& p) { from_json_key( j, - "columns", - p.columns, - "HiveBucketHandle", - "List", - "columns"); + "writerTarget", + p.writerTarget, + "TableWriteInfo", + "ExecutionWriterTarget", + "writerTarget"); from_json_key( j, - "tableBucketCount", - p.tableBucketCount, - "HiveBucketHandle", - "int", - "tableBucketCount"); + "analyzeTableHandle", + p.analyzeTableHandle, + "TableWriteInfo", + "AnalyzeTableHandle", + "analyzeTableHandle"); from_json_key( j, - "readBucketCount", - p.readBucketCount, - "HiveBucketHandle", - "int", - "readBucketCount"); + "deleteScanInfo", + p.deleteScanInfo, + "TableWriteInfo", + "DeleteScanInfo", + "deleteScanInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveTableLayoutHandle::HiveTableLayoutHandle() noexcept { - _type = "hive"; +EnforceSingleRowNode::EnforceSingleRowNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; } -void to_json(json& j, const HiveTableLayoutHandle& p) { +void to_json(json& j, const EnforceSingleRowNode& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); - to_json_key( - j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); - to_json_key( - j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); - to_json_key( - j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); + j["@type"] = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; + to_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); to_json_key( - j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); + j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); +} + +void from_json(const json& j, EnforceSingleRowNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); + from_json_key( + j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const EquiJoinClause& p) { + j = json::object(); to_json_key( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); to_json_key( j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); - to_json_key( + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); +} + +void from_json(const json& j, EquiJoinClause& p) { + from_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); - to_json_key( + "left", + p.left, + "EquiJoinClause", + "VariableReferenceExpression", + "left"); + from_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "right", + p.right, + "EquiJoinClause", + "VariableReferenceExpression", + "right"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair DistributionType_enum_table[] = + { // NOLINT: cert-err58-cpp + {DistributionType::PARTITIONED, "PARTITIONED"}, + {DistributionType::REPLICATED, "REPLICATED"}}; +void to_json(json& j, const DistributionType& e) { + static_assert( + std::is_enum::value, + "DistributionType must be an enum!"); + const auto* it = std::find_if( + std::begin(DistributionType_enum_table), + std::end(DistributionType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(DistributionType_enum_table)) + ? it + : std::begin(DistributionType_enum_table)) + ->second; +} +void from_json(const json& j, DistributionType& e) { + static_assert( + std::is_enum::value, + "DistributionType must be an enum!"); + const auto* it = std::find_if( + std::begin(DistributionType_enum_table), + std::end(DistributionType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(DistributionType_enum_table)) + ? it + : std::begin(DistributionType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair JoinNodeType_enum_table[] = + { // NOLINT: cert-err58-cpp + {JoinNodeType::INNER, "INNER"}, + {JoinNodeType::LEFT, "LEFT"}, + {JoinNodeType::RIGHT, "RIGHT"}, + {JoinNodeType::FULL, "FULL"}}; +void to_json(json& j, const JoinNodeType& e) { + static_assert( + std::is_enum::value, "JoinNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(JoinNodeType_enum_table), + std::end(JoinNodeType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(JoinNodeType_enum_table)) + ? it + : std::begin(JoinNodeType_enum_table)) + ->second; +} +void from_json(const json& j, JoinNodeType& e) { + static_assert( + std::is_enum::value, "JoinNodeType must be an enum!"); + const auto* it = std::find_if( + std::begin(JoinNodeType_enum_table), + std::end(JoinNodeType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(JoinNodeType_enum_table)) + ? it + : std::begin(JoinNodeType_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +JoinNode::JoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.JoinNode"; +} + +void to_json(json& j, const JoinNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.JoinNode"; + to_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); + to_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); + to_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); to_json_key( j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); to_json_key( - j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + j, + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); to_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); to_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", - "String", - "layoutString"); + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); to_json_key( j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", - "List", - "requestedColumns"); + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); to_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "distributionType", + p.distributionType, + "JoinNode", + "DistributionType", + "distributionType"); to_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, HiveTableLayoutHandle& p) { +void from_json(const json& j, JoinNode& p) { p._type = j["@type"]; + from_json_key(j, "id", p.id, "JoinNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "JoinNode", "JoinNodeType", "type"); + from_json_key(j, "left", p.left, "JoinNode", "PlanNode", "left"); + from_json_key(j, "right", p.right, "JoinNode", "PlanNode", "right"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "HiveTableLayoutHandle", - "SchemaTableName", - "schemaTableName"); + "criteria", + p.criteria, + "JoinNode", + "List", + "criteria"); from_json_key( j, - "tablePath", - p.tablePath, - "HiveTableLayoutHandle", - "String", - "tablePath"); + "outputVariables", + p.outputVariables, + "JoinNode", + "List", + "outputVariables"); from_json_key( j, - "partitionColumns", - p.partitionColumns, - "HiveTableLayoutHandle", - "List", - "partitionColumns"); + "filter", + p.filter, + "JoinNode", + "std::shared_ptr", + "filter"); from_json_key( j, - "dataColumns", - p.dataColumns, - "HiveTableLayoutHandle", - "List", - "dataColumns"); + "leftHashVariable", + p.leftHashVariable, + "JoinNode", + "VariableReferenceExpression", + "leftHashVariable"); from_json_key( j, - "tableParameters", - p.tableParameters, - "HiveTableLayoutHandle", - "Map", - "tableParameters"); + "rightHashVariable", + p.rightHashVariable, + "JoinNode", + "VariableReferenceExpression", + "rightHashVariable"); from_json_key( j, - "domainPredicate", - p.domainPredicate, - "HiveTableLayoutHandle", - "TupleDomain", - "domainPredicate"); + "distributionType", + p.distributionType, + "JoinNode", + "DistributionType", + "distributionType"); from_json_key( j, - "remainingPredicate", - p.remainingPredicate, - "HiveTableLayoutHandle", - "RowExpression", - "remainingPredicate"); + "dynamicFilters", + p.dynamicFilters, + "JoinNode", + "Map", + "dynamicFilters"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +MergeJoinNode::MergeJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; +} + +void to_json(json& j, const MergeJoinNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.MergeJoinNode"; + to_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); + to_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); + to_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); + to_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); + to_json_key( + j, + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); + to_json_key( + j, + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); + to_json_key( + j, + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); + to_json_key( + j, + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + to_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "rightHashVariable"); +} + +void from_json(const json& j, MergeJoinNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "MergeJoinNode", "PlanNodeId", "id"); + from_json_key(j, "type", p.type, "MergeJoinNode", "JoinNode.Type", "type"); + from_json_key(j, "left", p.left, "MergeJoinNode", "PlanNode", "left"); + from_json_key(j, "right", p.right, "MergeJoinNode", "PlanNode", "right"); from_json_key( j, - "predicateColumns", - p.predicateColumns, - "HiveTableLayoutHandle", - "Map", - "predicateColumns"); + "criteria", + p.criteria, + "MergeJoinNode", + "List", + "criteria"); from_json_key( j, - "partitionColumnPredicate", - p.partitionColumnPredicate, - "HiveTableLayoutHandle", - "TupleDomain>", - "partitionColumnPredicate"); + "outputVariables", + p.outputVariables, + "MergeJoinNode", + "List", + "outputVariables"); from_json_key( j, - "bucketHandle", - p.bucketHandle, - "HiveTableLayoutHandle", - "HiveBucketHandle", - "bucketHandle"); + "filter", + p.filter, + "MergeJoinNode", + "std::shared_ptr", + "filter"); from_json_key( j, - "bucketFilter", - p.bucketFilter, - "HiveTableLayoutHandle", - "HiveBucketFilter", - "bucketFilter"); + "leftHashVariable", + p.leftHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "leftHashVariable"); + from_json_key( + j, + "rightHashVariable", + p.rightHashVariable, + "MergeJoinNode", + "VariableReferenceExpression", + "rightHashVariable"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const BucketConversion& p) { + j = json::object(); + to_json_key( + j, + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); + to_json_key( + j, + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", + "int", + "partitionBucketCount"); + to_json_key( + j, + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", + "List", + "bucketColumnHandles"); +} + +void from_json(const json& j, BucketConversion& p) { from_json_key( j, - "pushdownFilterEnabled", - p.pushdownFilterEnabled, - "HiveTableLayoutHandle", - "bool", - "pushdownFilterEnabled"); + "tableBucketCount", + p.tableBucketCount, + "BucketConversion", + "int", + "tableBucketCount"); from_json_key( j, - "layoutString", - p.layoutString, - "HiveTableLayoutHandle", - "String", - "layoutString"); + "partitionBucketCount", + p.partitionBucketCount, + "BucketConversion", + "int", + "partitionBucketCount"); from_json_key( j, - "requestedColumns", - p.requestedColumns, - "HiveTableLayoutHandle", + "bucketColumnHandles", + p.bucketColumnHandles, + "BucketConversion", "List", - "requestedColumns"); - from_json_key( + "bucketColumnHandles"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair CacheQuotaScope_enum_table[] = + { // NOLINT: cert-err58-cpp + {CacheQuotaScope::GLOBAL, "GLOBAL"}, + {CacheQuotaScope::SCHEMA, "SCHEMA"}, + {CacheQuotaScope::TABLE, "TABLE"}, + {CacheQuotaScope::PARTITION, "PARTITION"}}; +void to_json(json& j, const CacheQuotaScope& e) { + static_assert( + std::is_enum::value, "CacheQuotaScope must be an enum!"); + const auto* it = std::find_if( + std::begin(CacheQuotaScope_enum_table), + std::end(CacheQuotaScope_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(CacheQuotaScope_enum_table)) + ? it + : std::begin(CacheQuotaScope_enum_table)) + ->second; +} +void from_json(const json& j, CacheQuotaScope& e) { + static_assert( + std::is_enum::value, "CacheQuotaScope must be an enum!"); + const auto* it = std::find_if( + std::begin(CacheQuotaScope_enum_table), + std::end(CacheQuotaScope_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(CacheQuotaScope_enum_table)) + ? it + : std::begin(CacheQuotaScope_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const CacheQuotaRequirement& p) { + j = json::object(); + to_json_key( j, - "partialAggregationsPushedDown", - p.partialAggregationsPushedDown, - "HiveTableLayoutHandle", - "bool", - "partialAggregationsPushedDown"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); + to_json_key( + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); +} + +void from_json(const json& j, CacheQuotaRequirement& p) { from_json_key( j, - "appendRowNumber", - p.appendRowNumber, - "HiveTableLayoutHandle", - "bool", - "appendRowNumber"); + "cacheQuotaScope", + p.cacheQuotaScope, + "CacheQuotaRequirement", + "CacheQuotaScope", + "cacheQuotaScope"); + from_json_key( + j, "quota", p.quota, "CacheQuotaRequirement", "DataSize", "quota"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair BufferType_enum_table[] = - { // NOLINT: cert-err58-cpp - {BufferType::PARTITIONED, "PARTITIONED"}, - {BufferType::BROADCAST, "BROADCAST"}, - {BufferType::ARBITRARY, "ARBITRARY"}, - {BufferType::DISCARDING, "DISCARDING"}, - {BufferType::SPOOLING, "SPOOLING"}}; -void to_json(json& j, const BufferType& e) { - static_assert(std::is_enum::value, "BufferType must be an enum!"); +static const std::pair + NodeSelectionStrategy_enum_table[] = { // NOLINT: cert-err58-cpp + {NodeSelectionStrategy::HARD_AFFINITY, "HARD_AFFINITY"}, + {NodeSelectionStrategy::SOFT_AFFINITY, "SOFT_AFFINITY"}, + {NodeSelectionStrategy::NO_PREFERENCE, "NO_PREFERENCE"}}; +void to_json(json& j, const NodeSelectionStrategy& e) { + static_assert( + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(BufferType_enum_table), - std::end(BufferType_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(NodeSelectionStrategy_enum_table), + std::end(NodeSelectionStrategy_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(BufferType_enum_table)) + j = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(BufferType_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->second; } -void from_json(const json& j, BufferType& e) { - static_assert(std::is_enum::value, "BufferType must be an enum!"); +void from_json(const json& j, NodeSelectionStrategy& e) { + static_assert( + std::is_enum::value, + "NodeSelectionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(BufferType_enum_table), - std::end(BufferType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(NodeSelectionStrategy_enum_table), + std::end(NodeSelectionStrategy_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(BufferType_enum_table)) + e = ((it != std::end(NodeSelectionStrategy_enum_table)) ? it - : std::begin(BufferType_enum_table)) + : std::begin(NodeSelectionStrategy_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const OutputBuffers& p) { +void to_json(json& j, const Column& p) { + j = json::object(); + to_json_key(j, "name", p.name, "Column", "String", "name"); + to_json_key(j, "type", p.type, "Column", "String", "type"); +} + +void from_json(const json& j, Column& p) { + from_json_key(j, "name", p.name, "Column", "String", "name"); + from_json_key(j, "type", p.type, "Column", "String", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TableToPartitionMapping& p) { + j = json::object(); + to_json_key( + j, + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); + to_json_key( + j, + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); +} + +void from_json(const json& j, TableToPartitionMapping& p) { + from_json_key( + j, + "tableToPartitionColumns", + p.tableToPartitionColumns, + "TableToPartitionMapping", + "Map", + "tableToPartitionColumns"); + from_json_key( + j, + "partitionSchemaDifference", + p.partitionSchemaDifference, + "TableToPartitionMapping", + "Map", + "partitionSchemaDifference"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const HiveFileSplit& p) { + j = json::object(); + to_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); + to_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); + to_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); + to_json_key( + j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); + to_json_key( + j, + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); + to_json_key( + j, + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); + to_json_key( + j, + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); +} + +void from_json(const json& j, HiveFileSplit& p) { + from_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); + from_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); + from_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); + from_json_key( + j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); + from_json_key( + j, + "fileModifiedTime", + p.fileModifiedTime, + "HiveFileSplit", + "int64_t", + "fileModifiedTime"); + from_json_key( + j, + "extraFileInfo", + p.extraFileInfo, + "HiveFileSplit", + "String", + "extraFileInfo"); + from_json_key( + j, + "customSplitInfo", + p.customSplitInfo, + "HiveFileSplit", + "Map", + "customSplitInfo"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const HivePartitionKey& p) { j = json::object(); - to_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); - to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); - to_json_key( - j, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); - to_json_key( - j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); + to_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); + to_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); } -void from_json(const json& j, OutputBuffers& p) { - from_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); - from_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); - from_json_key( - j, - "noMoreBufferIds", - p.noMoreBufferIds, - "OutputBuffers", - "bool", - "noMoreBufferIds"); - from_json_key( - j, - "buffers", - p.buffers, - "OutputBuffers", - "Map", - "buffers"); +void from_json(const json& j, HivePartitionKey& p) { + from_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); + from_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Assignments& p) { +void to_json(json& j, const StorageFormat& p) { j = json::object(); + to_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); to_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); + to_json_key( + j, + "outputFormat", + p.outputFormat, + "StorageFormat", + "String", + "outputFormat"); } -void from_json(const json& j, Assignments& p) { +void from_json(const json& j, StorageFormat& p) { + from_json_key(j, "serDe", p.serDe, "StorageFormat", "String", "serDe"); from_json_key( j, - "assignments", - p.assignments, - "Assignments", - "Map>", - "assignments"); + "inputFormat", + p.inputFormat, + "StorageFormat", + "String", + "inputFormat"); + from_json_key( + j, + "outputFormat", + p.outputFormat, + "StorageFormat", + "String", + "outputFormat"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveTransactionHandle::HiveTransactionHandle() noexcept { - _type = "hive"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const HiveTransactionHandle& p) { - j = json::object(); - j["@type"] = "hive"; - to_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + BucketFunctionType_enum_table[] = { // NOLINT: cert-err58-cpp + {BucketFunctionType::HIVE_COMPATIBLE, "HIVE_COMPATIBLE"}, + {BucketFunctionType::PRESTO_NATIVE, "PRESTO_NATIVE"}}; +void to_json(json& j, const BucketFunctionType& e) { + static_assert( + std::is_enum::value, + "BucketFunctionType must be an enum!"); + const auto* it = std::find_if( + std::begin(BucketFunctionType_enum_table), + std::end(BucketFunctionType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(BucketFunctionType_enum_table)) + ? it + : std::begin(BucketFunctionType_enum_table)) + ->second; } - -void from_json(const json& j, HiveTransactionHandle& p) { - p._type = j["@type"]; - from_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); +void from_json(const json& j, BucketFunctionType& e) { + static_assert( + std::is_enum::value, + "BucketFunctionType must be an enum!"); + const auto* it = std::find_if( + std::begin(BucketFunctionType_enum_table), + std::end(BucketFunctionType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(BucketFunctionType_enum_table)) + ? it + : std::begin(BucketFunctionType_enum_table)) + ->first; } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -TableWriterNode::TableWriterNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.TableWriterNode"; -} -void to_json(json& j, const TableWriterNode& p) { +void to_json(json& j, const HiveBucketProperty& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterNode"; - to_json_key(j, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); - to_json_key( - j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterNode", - "VariableReferenceExpression", - "rowCountVariable"); - to_json_key( - j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); - to_json_key( - j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); - to_json_key( - j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); to_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", "List", - "columnNames"); + "bucketedBy"); to_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); to_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); to_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); -} - -void from_json(const json& j, TableWriterNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); - from_json_key( - j, - "rowCountVariable", - p.rowCountVariable, - "TableWriterNode", - "VariableReferenceExpression", - "rowCountVariable"); - from_json_key( - j, - "fragmentVariable", - p.fragmentVariable, - "TableWriterNode", - "VariableReferenceExpression", - "fragmentVariable"); - from_json_key( - j, - "tableCommitContextVariable", - p.tableCommitContextVariable, - "TableWriterNode", - "VariableReferenceExpression", - "tableCommitContextVariable"); - from_json_key( - j, - "columns", - p.columns, - "TableWriterNode", - "List", - "columns"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + to_json_key(j, "types", p.types, "HiveBucketProperty", "List", "types"); +} + +void from_json(const json& j, HiveBucketProperty& p) { from_json_key( j, - "columnNames", - p.columnNames, - "TableWriterNode", + "bucketedBy", + p.bucketedBy, + "HiveBucketProperty", "List", - "columnNames"); + "bucketedBy"); from_json_key( j, - "notNullColumnVariables", - p.notNullColumnVariables, - "TableWriterNode", - "List", - "notNullColumnVariables"); + "bucketCount", + p.bucketCount, + "HiveBucketProperty", + "int", + "bucketCount"); from_json_key( j, - "partitioningScheme", - p.partitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "partitioningScheme"); + "sortedBy", + p.sortedBy, + "HiveBucketProperty", + "List", + "sortedBy"); from_json_key( j, - "preferredShufflePartitioningScheme", - p.preferredShufflePartitioningScheme, - "TableWriterNode", - "PartitioningScheme", - "preferredShufflePartitioningScheme"); -} -} // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -namespace facebook::presto::protocol { - -void to_json(json& j, const Block& p) { - j = p.data; -} - -void from_json(const json& j, Block& p) { - p.data = std::string(j); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const ValueEntry& p) { - j = json::object(); - to_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); - to_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); -} - -void from_json(const json& j, ValueEntry& p) { - from_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); - from_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); + "bucketFunctionType", + p.bucketFunctionType, + "HiveBucketProperty", + "BucketFunctionType", + "bucketFunctionType"); + from_json_key( + j, "types", p.types, "HiveBucketProperty", "List", "types"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EquatableValueSet::EquatableValueSet() noexcept { - _type = "equatable"; -} -void to_json(json& j, const EquatableValueSet& p) { +void to_json(json& j, const Storage& p) { j = json::object(); - j["@type"] = "equatable"; - to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); to_json_key( - j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + to_json_key(j, "location", p.location, "Storage", "String", "location"); to_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + to_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + to_json_key( + j, + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + to_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); } -void from_json(const json& j, EquatableValueSet& p) { - p._type = j["@type"]; - from_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); +void from_json(const json& j, Storage& p) { from_json_key( - j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); + j, + "storageFormat", + p.storageFormat, + "Storage", + "StorageFormat", + "storageFormat"); + from_json_key(j, "location", p.location, "Storage", "String", "location"); from_json_key( j, - "entries", - p.entries, - "EquatableValueSet", - "List", - "entries"); + "bucketProperty", + p.bucketProperty, + "Storage", + "HiveBucketProperty", + "bucketProperty"); + from_json_key(j, "skewed", p.skewed, "Storage", "bool", "skewed"); + from_json_key( + j, + "serdeParameters", + p.serdeParameters, + "Storage", + "Map", + "serdeParameters"); + from_json_key( + j, + "parameters", + p.parameters, + "Storage", + "Map", + "parameters"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveSplit::HiveSplit() noexcept { + _type = "hive"; +} -void to_json(json& j, const PageBufferInfo& p) { +void to_json(json& j, const HiveSplit& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); + to_json_key(j, "database", p.database, "HiveSplit", "String", "database"); + to_json_key(j, "table", p.table, "HiveSplit", "String", "table"); to_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", - "int64_t", - "bufferedPages"); + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); to_json_key( j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", - "int64_t", - "bufferedBytes"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); + to_json_key( + j, + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); + to_json_key( + j, + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", + "int", + "readBucketNumber"); + to_json_key( + j, + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); + to_json_key( + j, + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); + to_json_key( + j, + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", + "int", + "partitionDataColumnCount"); + to_json_key( + j, + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); + to_json_key( + j, + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); + to_json_key( + j, + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); + to_json_key( + j, + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); + to_json_key( + j, + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); to_json_key( - j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + j, + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); to_json_key( - j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); + j, + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); } -void from_json(const json& j, PageBufferInfo& p) { +void from_json(const json& j, HiveSplit& p) { + p._type = j["@type"]; from_json_key( - j, "partition", p.partition, "PageBufferInfo", "int", "partition"); + j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); + from_json_key(j, "database", p.database, "HiveSplit", "String", "database"); + from_json_key(j, "table", p.table, "HiveSplit", "String", "table"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "PageBufferInfo", - "int64_t", - "bufferedPages"); + "partitionName", + p.partitionName, + "HiveSplit", + "String", + "partitionName"); + from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); from_json_key( j, - "bufferedBytes", - p.bufferedBytes, - "PageBufferInfo", - "int64_t", - "bufferedBytes"); + "partitionKeys", + p.partitionKeys, + "HiveSplit", + "List", + "partitionKeys"); from_json_key( - j, "rowsAdded", p.rowsAdded, "PageBufferInfo", "int64_t", "rowsAdded"); + j, + "addresses", + p.addresses, + "HiveSplit", + "List", + "addresses"); from_json_key( - j, "pagesAdded", p.pagesAdded, "PageBufferInfo", "int64_t", "pagesAdded"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const BufferInfo& p) { - j = json::object(); - to_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - to_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); - to_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", + "readBucketNumber", + p.readBucketNumber, + "HiveSplit", "int", - "bufferedPages"); - to_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); - to_json_key( + "readBucketNumber"); + from_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); -} - -void from_json(const json& j, BufferInfo& p) { + "tableBucketNumber", + p.tableBucketNumber, + "HiveSplit", + "int", + "tableBucketNumber"); from_json_key( - j, "bufferId", p.bufferId, "BufferInfo", "OutputBufferId", "bufferId"); - from_json_key(j, "finished", p.finished, "BufferInfo", "bool", "finished"); + j, + "nodeSelectionStrategy", + p.nodeSelectionStrategy, + "HiveSplit", + "NodeSelectionStrategy", + "nodeSelectionStrategy"); from_json_key( j, - "bufferedPages", - p.bufferedPages, - "BufferInfo", + "partitionDataColumnCount", + p.partitionDataColumnCount, + "HiveSplit", "int", - "bufferedPages"); + "partitionDataColumnCount"); from_json_key( - j, "pagesSent", p.pagesSent, "BufferInfo", "int64_t", "pagesSent"); + j, + "tableToPartitionMapping", + p.tableToPartitionMapping, + "HiveSplit", + "TableToPartitionMapping", + "tableToPartitionMapping"); from_json_key( j, - "pageBufferInfo", - p.pageBufferInfo, - "BufferInfo", - "PageBufferInfo", - "pageBufferInfo"); + "bucketConversion", + p.bucketConversion, + "HiveSplit", + "BucketConversion", + "bucketConversion"); + from_json_key( + j, + "s3SelectPushdownEnabled", + p.s3SelectPushdownEnabled, + "HiveSplit", + "bool", + "s3SelectPushdownEnabled"); + from_json_key( + j, + "cacheQuota", + p.cacheQuota, + "HiveSplit", + "CacheQuotaRequirement", + "cacheQuota"); + from_json_key( + j, + "encryptionMetadata", + p.encryptionMetadata, + "HiveSplit", + "EncryptionInformation", + "encryptionMetadata"); + from_json_key( + j, + "redundantColumnDomains", + p.redundantColumnDomains, + "HiveSplit", + "List>", + "redundantColumnDomains"); + from_json_key( + j, + "splitWeight", + p.splitWeight, + "HiveSplit", + "SplitWeight", + "splitWeight"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +AssignUniqueId::AssignUniqueId() noexcept { + _type = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; +} -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair FunctionKind_enum_table[] = - { // NOLINT: cert-err58-cpp - {FunctionKind::SCALAR, "SCALAR"}, - {FunctionKind::AGGREGATE, "AGGREGATE"}, - {FunctionKind::WINDOW, "WINDOW"}}; -void to_json(json& j, const FunctionKind& e) { - static_assert( - std::is_enum::value, "FunctionKind must be an enum!"); - const auto* it = std::find_if( - std::begin(FunctionKind_enum_table), - std::end(FunctionKind_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(FunctionKind_enum_table)) - ? it - : std::begin(FunctionKind_enum_table)) - ->second; +void to_json(json& j, const AssignUniqueId& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; + to_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + to_json_key( + j, + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); } -void from_json(const json& j, FunctionKind& e) { - static_assert( - std::is_enum::value, "FunctionKind must be an enum!"); - const auto* it = std::find_if( - std::begin(FunctionKind_enum_table), - std::end(FunctionKind_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(FunctionKind_enum_table)) - ? it - : std::begin(FunctionKind_enum_table)) - ->first; + +void from_json(const json& j, AssignUniqueId& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + from_json_key( + j, + "idVariable", + p.idVariable, + "AssignUniqueId", + "VariableReferenceExpression", + "idVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +TpchColumnHandle::TpchColumnHandle() noexcept { + _type = "tpch"; +} -void to_json(json& j, const LongVariableConstraint& p) { +void to_json(json& j, const TpchColumnHandle& p) { j = json::object(); - to_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); + j["@type"] = "tpch"; to_json_key( j, - "expression", - p.expression, - "LongVariableConstraint", + "columnName", + p.columnName, + "TpchColumnHandle", "String", - "expression"); + "columnName"); + to_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); } -void from_json(const json& j, LongVariableConstraint& p) { - from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); +void from_json(const json& j, TpchColumnHandle& p) { + p._type = j["@type"]; from_json_key( j, - "expression", - p.expression, - "LongVariableConstraint", + "columnName", + p.columnName, + "TpchColumnHandle", "String", - "expression"); + "columnName"); + from_json_key(j, "type", p.type, "TpchColumnHandle", "Type", "type"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair PrestoTableType_enum_table[] = + { // NOLINT: cert-err58-cpp + {PrestoTableType::MANAGED_TABLE, "MANAGED_TABLE"}, + {PrestoTableType::EXTERNAL_TABLE, "EXTERNAL_TABLE"}, + {PrestoTableType::VIRTUAL_VIEW, "VIRTUAL_VIEW"}, + {PrestoTableType::MATERIALIZED_VIEW, "MATERIALIZED_VIEW"}, + {PrestoTableType::TEMPORARY_TABLE, "TEMPORARY_TABLE"}, + {PrestoTableType::OTHER, "OTHER"}}; +void to_json(json& j, const PrestoTableType& e) { + static_assert( + std::is_enum::value, "PrestoTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(PrestoTableType_enum_table), + std::end(PrestoTableType_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_enum_table)) + ->second; +} +void from_json(const json& j, PrestoTableType& e) { + static_assert( + std::is_enum::value, "PrestoTableType must be an enum!"); + const auto* it = std::find_if( + std::begin(PrestoTableType_enum_table), + std::end(PrestoTableType_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(PrestoTableType_enum_table)) + ? it + : std::begin(PrestoTableType_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const TypeVariableConstraint& p) { +void to_json(json& j, const Table& p) { j = json::object(); - to_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); + to_json_key( + j, "databaseName", p.databaseName, "Table", "String", "databaseName"); + to_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); + to_json_key(j, "owner", p.owner, "Table", "String", "owner"); + to_json_key( + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + to_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + to_json_key( + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); to_json_key( j, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); to_json_key( j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); to_json_key( j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", + "viewOriginalText", + p.viewOriginalText, + "Table", "String", - "variadicBound"); + "viewOriginalText"); to_json_key( j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); } -void from_json(const json& j, TypeVariableConstraint& p) { - from_json_key(j, "name", p.name, "TypeVariableConstraint", "String", "name"); +void from_json(const json& j, Table& p) { + from_json_key( + j, "databaseName", p.databaseName, "Table", "String", "databaseName"); + from_json_key(j, "tableName", p.tableName, "Table", "String", "tableName"); + from_json_key(j, "owner", p.owner, "Table", "String", "owner"); + from_json_key( + j, "tableType", p.tableType, "Table", "PrestoTableType", "tableType"); + from_json_key(j, "storage", p.storage, "Table", "Storage", "storage"); + from_json_key( + j, "dataColumns", p.dataColumns, "Table", "List", "dataColumns"); from_json_key( j, - "comparableRequired", - p.comparableRequired, - "TypeVariableConstraint", - "bool", - "comparableRequired"); + "partitionColumns", + p.partitionColumns, + "Table", + "List", + "partitionColumns"); from_json_key( j, - "orderableRequired", - p.orderableRequired, - "TypeVariableConstraint", - "bool", - "orderableRequired"); + "parameters", + p.parameters, + "Table", + "Map", + "parameters"); from_json_key( j, - "variadicBound", - p.variadicBound, - "TypeVariableConstraint", + "viewOriginalText", + p.viewOriginalText, + "Table", "String", - "variadicBound"); + "viewOriginalText"); from_json_key( j, - "nonDecimalNumericRequired", - p.nonDecimalNumericRequired, - "TypeVariableConstraint", - "bool", - "nonDecimalNumericRequired"); + "viewExpandedText", + p.viewExpandedText, + "Table", + "String", + "viewExpandedText"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -void to_json(json& j, const Signature& p) { +void to_json(json& j, const HivePageSinkMetadata& p) { j = json::object(); - to_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); - to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); to_json_key( j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); + "schemaTableName", + p.schemaTableName, + "HivePageSinkMetadata", + "SchemaTableName", + "schemaTableName"); + to_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); +} + +void from_json(const json& j, HivePageSinkMetadata& p) { + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "HivePageSinkMetadata", + "SchemaTableName", + "schemaTableName"); + from_json_key(j, "table", p.table, "HivePageSinkMetadata", "Table", "table"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveMetadataUpdateHandle::HiveMetadataUpdateHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveMetadataUpdateHandle& p) { + j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); to_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); + "schemaTableName", + p.schemaTableName, + "HiveMetadataUpdateHandle", + "SchemaTableName", + "schemaTableName"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "partitionName", + p.partitionName, + "HiveMetadataUpdateHandle", + "String", + "partitionName"); to_json_key( j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); } -void from_json(const json& j, Signature& p) { - from_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); - from_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); - from_json_key( - j, - "typeVariableConstraints", - p.typeVariableConstraints, - "Signature", - "List", - "typeVariableConstraints"); - from_json_key( - j, - "longVariableConstraints", - p.longVariableConstraints, - "Signature", - "List", - "longVariableConstraints"); +void from_json(const json& j, HiveMetadataUpdateHandle& p) { + p._type = j["@type"]; from_json_key( j, - "returnType", - p.returnType, - "Signature", - "TypeSignature", - "returnType"); + "requestId", + p.requestId, + "HiveMetadataUpdateHandle", + "UUID", + "requestId"); from_json_key( j, - "argumentTypes", - p.argumentTypes, - "Signature", - "List", - "argumentTypes"); + "schemaTableName", + p.schemaTableName, + "HiveMetadataUpdateHandle", + "SchemaTableName", + "schemaTableName"); from_json_key( j, - "variableArity", - p.variableArity, - "Signature", - "bool", - "variableArity"); + "partitionName", + p.partitionName, + "HiveMetadataUpdateHandle", + "String", + "partitionName"); + from_json_key( + j, + "fileName", + p.fileName, + "HiveMetadataUpdateHandle", + "String", + "fileName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { - _type = "$static"; + +void to_json(json& j, const NodeVersion& p) { + j = json::object(); + to_json_key(j, "version", p.version, "NodeVersion", "String", "version"); } -void to_json(json& j, const BuiltInFunctionHandle& p) { +void from_json(const json& j, NodeVersion& p) { + from_json_key(j, "version", p.version, "NodeVersion", "String", "version"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const ServerInfo& p) { j = json::object(); - j["@type"] = "$static"; to_json_key( j, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); + to_json_key( + j, "environment", p.environment, "ServerInfo", "String", "environment"); + to_json_key( + j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); + to_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); + to_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); } -void from_json(const json& j, BuiltInFunctionHandle& p) { - p._type = j["@type"]; +void from_json(const json& j, ServerInfo& p) { from_json_key( j, - "signature", - p.signature, - "BuiltInFunctionHandle", - "Signature", - "signature"); + "nodeVersion", + p.nodeVersion, + "ServerInfo", + "NodeVersion", + "nodeVersion"); + from_json_key( + j, "environment", p.environment, "ServerInfo", "String", "environment"); + from_json_key( + j, "coordinator", p.coordinator, "ServerInfo", "bool", "coordinator"); + from_json_key(j, "starting", p.starting, "ServerInfo", "bool", "starting"); + from_json_key(j, "uptime", p.uptime, "ServerInfo", "Duration", "uptime"); } } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { - _type = "lambda"; +void to_json(json& j, const std::shared_ptr& p) { + if (p == nullptr) { + return; + } + String type = p->_type; + + if (getConnectorKey(type) == "hive") { + j = *std::static_pointer_cast(p); + return; + } + + throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); } -void to_json(json& j, const LambdaDefinitionExpression& p) { +void from_json(const json& j, std::shared_ptr& p) { + String type; + try { + type = p->getSubclassKey(j); + } catch (json::parse_error& e) { + throw ParseError( + std::string(e.what()) + + " ConnectorOutputTableHandle ConnectorOutputTableHandle"); + } + + if (getConnectorKey(type) == "hive") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + + throw TypeError(type + " no abstract type ConnectorOutputTableHandle "); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const OutputTableHandle& p) { j = json::object(); - j["@type"] = "lambda"; to_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); to_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); to_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); - to_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); } -void from_json(const json& j, LambdaDefinitionExpression& p) { - p._type = j["@type"]; +void from_json(const json& j, OutputTableHandle& p) { from_json_key( j, - "sourceLocation", - p.sourceLocation, - "LambdaDefinitionExpression", - "SourceLocation", - "sourceLocation"); + "connectorId", + p.connectorId, + "OutputTableHandle", + "ConnectorId", + "connectorId"); from_json_key( j, - "argumentTypes", - p.argumentTypes, - "LambdaDefinitionExpression", - "List", - "argumentTypes"); + "transactionHandle", + p.transactionHandle, + "OutputTableHandle", + "ConnectorTransactionHandle", + "transactionHandle"); from_json_key( j, - "arguments", - p.arguments, - "LambdaDefinitionExpression", - "List", - "arguments"); - from_json_key( - j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); + "connectorHandle", + p.connectorHandle, + "OutputTableHandle", + "ConnectorOutputTableHandle", + "connectorHandle"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair BoundType_enum_table[] = - { // NOLINT: cert-err58-cpp - {BoundType::UNBOUNDED_PRECEDING, "UNBOUNDED_PRECEDING"}, - {BoundType::PRECEDING, "PRECEDING"}, - {BoundType::CURRENT_ROW, "CURRENT_ROW"}, - {BoundType::FOLLOWING, "FOLLOWING"}, - {BoundType::UNBOUNDED_FOLLOWING, "UNBOUNDED_FOLLOWING"}}; -void to_json(json& j, const BoundType& e) { - static_assert(std::is_enum::value, "BoundType must be an enum!"); - const auto* it = std::find_if( - std::begin(BoundType_enum_table), - std::end(BoundType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(BoundType_enum_table)) - ? it - : std::begin(BoundType_enum_table)) - ->second; -} -void from_json(const json& j, BoundType& e) { - static_assert(std::is_enum::value, "BoundType must be an enum!"); - const auto* it = std::find_if( - std::begin(BoundType_enum_table), - std::end(BoundType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(BoundType_enum_table)) - ? it - : std::begin(BoundType_enum_table)) - ->first; +CreateHandle::CreateHandle() noexcept { + _type = "CreateHandle"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair WindowType_enum_table[] = { - // NOLINT: cert-err58-cpp - {WindowType::RANGE, "RANGE"}, - {WindowType::ROWS, "ROWS"}, -}; -void to_json(json& j, const WindowType& e) { - static_assert(std::is_enum::value, "WindowType must be an enum!"); - const auto* it = std::find_if( - std::begin(WindowType_enum_table), - std::end(WindowType_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(WindowType_enum_table)) - ? it - : std::begin(WindowType_enum_table)) - ->second; -} -void from_json(const json& j, WindowType& e) { - static_assert(std::is_enum::value, "WindowType must be an enum!"); - const auto* it = std::find_if( - std::begin(WindowType_enum_table), - std::end(WindowType_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(WindowType_enum_table)) - ? it - : std::begin(WindowType_enum_table)) - ->first; +void to_json(json& j, const CreateHandle& p) { + j = json::object(); + j["@type"] = "CreateHandle"; + to_json_key( + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "SchemaTableName", + "schemaTableName"); +} + +void from_json(const json& j, CreateHandle& p) { + p._type = j["@type"]; + from_json_key( + j, "handle", p.handle, "CreateHandle", "OutputTableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "CreateHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Frame& p) { +void to_json(json& j, const NodeStatus& p) { j = json::object(); - to_json_key(j, "type", p.type, "Frame", "WindowType", "type"); - to_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); + to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); to_json_key( j, - "startValue", - p.startValue, - "Frame", - "VariableReferenceExpression", - "startValue"); - to_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); to_json_key( - j, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); + j, "environment", p.environment, "NodeStatus", "String", "environment"); + to_json_key( + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); to_json_key( j, - "originalStartValue", - p.originalStartValue, - "Frame", + "externalAddress", + p.externalAddress, + "NodeStatus", "String", - "originalStartValue"); + "externalAddress"); to_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", + "internalAddress", + p.internalAddress, + "NodeStatus", "String", - "originalEndValue"); + "internalAddress"); + to_json_key( + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); + to_json_key( + j, + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); + to_json_key( + j, + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + to_json_key( + j, + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + to_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } -void from_json(const json& j, Frame& p) { - from_json_key(j, "type", p.type, "Frame", "WindowType", "type"); - from_json_key(j, "startType", p.startType, "Frame", "BoundType", "startType"); +void from_json(const json& j, NodeStatus& p) { + from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); from_json_key( j, - "startValue", - p.startValue, - "Frame", - "VariableReferenceExpression", - "startValue"); - from_json_key(j, "endType", p.endType, "Frame", "BoundType", "endType"); + "nodeVersion", + p.nodeVersion, + "NodeStatus", + "NodeVersion", + "nodeVersion"); from_json_key( - j, - "endValue", - p.endValue, - "Frame", - "VariableReferenceExpression", - "endValue"); + j, "environment", p.environment, "NodeStatus", "String", "environment"); + from_json_key( + j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); + from_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); from_json_key( j, - "originalStartValue", - p.originalStartValue, - "Frame", + "externalAddress", + p.externalAddress, + "NodeStatus", "String", - "originalStartValue"); + "externalAddress"); from_json_key( j, - "originalEndValue", - p.originalEndValue, - "Frame", + "internalAddress", + p.internalAddress, + "NodeStatus", "String", - "originalEndValue"); + "internalAddress"); + from_json_key( + j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); + from_json_key( + j, "processors", p.processors, "NodeStatus", "int", "processors"); + from_json_key( + j, + "processCpuLoad", + p.processCpuLoad, + "NodeStatus", + "double", + "processCpuLoad"); + from_json_key( + j, + "systemCpuLoad", + p.systemCpuLoad, + "NodeStatus", + "double", + "systemCpuLoad"); + from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + from_json_key( + j, + "heapAvailable", + p.heapAvailable, + "NodeStatus", + "int64_t", + "heapAvailable"); + from_json_key( + j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveTableHandle::HiveTableHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const Function& p) { +void to_json(json& j, const HiveTableHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( - j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - to_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); to_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + to_json_key( + j, + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); } -void from_json(const json& j, Function& p) { +void from_json(const json& j, HiveTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "functionCall", - p.functionCall, - "Function", - "CallExpression", - "functionCall"); - from_json_key(j, "frame", p.frame, "Function", "Frame", "frame"); + j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); from_json_key( - j, "ignoreNulls", p.ignoreNulls, "Function", "bool", "ignoreNulls"); + j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + from_json_key( + j, + "analyzePartitionValues", + p.analyzePartitionValues, + "HiveTableHandle", + "List>", + "analyzePartitionValues"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveTransactionHandle::HiveTransactionHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const VariableStatsEstimate& p) { +void to_json(json& j, const HiveTransactionHandle& p) { j = json::object(); - to_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); - to_json_key( - j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); - to_json_key( - j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - to_json_key( - j, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", - "double", - "averageRowSize"); - to_json_key( - j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); + j["@type"] = "hive"; + to_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); +} + +void from_json(const json& j, HiveTransactionHandle& p) { + p._type = j["@type"]; + from_json_key(j, "uuid", p.uuid, "HiveTransactionHandle", "UUID", "uuid"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +namespace facebook::presto::protocol { +TableWriterNode::TableWriterNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.TableWriterNode"; } -void from_json(const json& j, VariableStatsEstimate& p) { - from_json_key( - j, "lowValue", p.lowValue, "VariableStatsEstimate", "double", "lowValue"); - from_json_key( +void to_json(json& j, const TableWriterNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.TableWriterNode"; + to_json_key(j, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); + to_json_key( j, - "highValue", - p.highValue, - "VariableStatsEstimate", - "double", - "highValue"); - from_json_key( + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); + to_json_key( j, - "nullsFraction", - p.nullsFraction, - "VariableStatsEstimate", - "double", - "nullsFraction"); - from_json_key( + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); + to_json_key( j, - "averageRowSize", - p.averageRowSize, - "VariableStatsEstimate", - "double", - "averageRowSize"); - from_json_key( + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); + to_json_key( j, - "distinctValuesCount", - p.distinctValuesCount, - "VariableStatsEstimate", - "double", - "distinctValuesCount"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PlanNodeStatsEstimate& p) { - j = json::object(); + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); to_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); to_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); to_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); to_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); } -void from_json(const json& j, PlanNodeStatsEstimate& p) { +void from_json(const json& j, TableWriterNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TableWriterNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "TableWriterNode", "PlanNode", "source"); from_json_key( j, - "outputRowCount", - p.outputRowCount, - "PlanNodeStatsEstimate", - "double", - "outputRowCount"); + "rowCountVariable", + p.rowCountVariable, + "TableWriterNode", + "VariableReferenceExpression", + "rowCountVariable"); from_json_key( j, - "totalSize", - p.totalSize, - "PlanNodeStatsEstimate", - "double", - "totalSize"); + "fragmentVariable", + p.fragmentVariable, + "TableWriterNode", + "VariableReferenceExpression", + "fragmentVariable"); from_json_key( j, - "confident", - p.confident, - "PlanNodeStatsEstimate", - "bool", - "confident"); + "tableCommitContextVariable", + p.tableCommitContextVariable, + "TableWriterNode", + "VariableReferenceExpression", + "tableCommitContextVariable"); from_json_key( j, - "variableStatistics", - p.variableStatistics, - "PlanNodeStatsEstimate", - "Map", - "variableStatistics"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const PlanCostEstimate& p) { - j = json::object(); - to_json_key(j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); - to_json_key( - j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); - to_json_key( - j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); - to_json_key( - j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); -} - -void from_json(const json& j, PlanCostEstimate& p) { + "columns", + p.columns, + "TableWriterNode", + "List", + "columns"); from_json_key( - j, "cpuCost", p.cpuCost, "PlanCostEstimate", "double", "cpuCost"); + j, + "columnNames", + p.columnNames, + "TableWriterNode", + "List", + "columnNames"); from_json_key( - j, "maxMemory", p.maxMemory, "PlanCostEstimate", "double", "maxMemory"); + j, + "notNullColumnVariables", + p.notNullColumnVariables, + "TableWriterNode", + "List", + "notNullColumnVariables"); from_json_key( j, - "maxMemoryWhenOutputting", - p.maxMemoryWhenOutputting, - "PlanCostEstimate", - "double", - "maxMemoryWhenOutputting"); + "partitioningScheme", + p.partitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "partitioningScheme"); from_json_key( j, - "networkCost", - p.networkCost, - "PlanCostEstimate", - "double", - "networkCost"); + "preferredShufflePartitioningScheme", + p.preferredShufflePartitioningScheme, + "TableWriterNode", + "PartitioningScheme", + "preferredShufflePartitioningScheme"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +DistinctLimitNode::DistinctLimitNode() noexcept { + _type = ".DistinctLimitNode"; +} -void to_json(json& j, const StatsAndCosts& p) { +void to_json(json& j, const DistinctLimitNode& p) { j = json::object(); + j["@type"] = ".DistinctLimitNode"; + to_json_key(j, "id", p.id, "DistinctLimitNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); + to_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); + to_json_key(j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); to_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); to_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); } -void from_json(const json& j, StatsAndCosts& p) { +void from_json(const json& j, DistinctLimitNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "DistinctLimitNode", "PlanNodeId", "id"); + from_json_key( + j, "source", p.source, "DistinctLimitNode", "PlanNode", "source"); + from_json_key(j, "limit", p.limit, "DistinctLimitNode", "int64_t", "limit"); + from_json_key( + j, "partial", p.partial, "DistinctLimitNode", "bool", "partial"); from_json_key( j, - "stats", - p.stats, - "StatsAndCosts", - "Map", - "stats"); + "distinctVariables", + p.distinctVariables, + "DistinctLimitNode", + "List", + "distinctVariables"); from_json_key( j, - "costs", - p.costs, - "StatsAndCosts", - "Map", - "costs"); + "hashVariable", + p.hashVariable, + "DistinctLimitNode", + "VariableReferenceExpression", + "hashVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Bound_enum_table[] = +static const std::pair Form_enum_table[] = { // NOLINT: cert-err58-cpp - {Bound::BELOW, "BELOW"}, - {Bound::EXACTLY, "EXACTLY"}, - {Bound::ABOVE, "ABOVE"}}; -void to_json(json& j, const Bound& e) { - static_assert(std::is_enum::value, "Bound must be an enum!"); + {Form::IF, "IF"}, + {Form::NULL_IF, "NULL_IF"}, + {Form::SWITCH, "SWITCH"}, + {Form::WHEN, "WHEN"}, + {Form::IS_NULL, "IS_NULL"}, + {Form::COALESCE, "COALESCE"}, + {Form::IN, "IN"}, + {Form::AND, "AND"}, + {Form::OR, "OR"}, + {Form::DEREFERENCE, "DEREFERENCE"}, + {Form::ROW_CONSTRUCTOR, "ROW_CONSTRUCTOR"}, + {Form::BIND, "BIND"}}; +void to_json(json& j, const Form& e) { + static_assert(std::is_enum::value, "Form must be an enum!"); const auto* it = std::find_if( - std::begin(Bound_enum_table), - std::end(Bound_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(Form_enum_table), + std::end(Form_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) + j = ((it != std::end(Form_enum_table)) ? it : std::begin(Form_enum_table)) ->second; } -void from_json(const json& j, Bound& e) { - static_assert(std::is_enum::value, "Bound must be an enum!"); +void from_json(const json& j, Form& e) { + static_assert(std::is_enum::value, "Form must be an enum!"); const auto* it = std::find_if( - std::begin(Bound_enum_table), - std::end(Bound_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(Form_enum_table), + std::end(Form_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(Bound_enum_table)) ? it : std::begin(Bound_enum_table)) + e = ((it != std::end(Form_enum_table)) ? it : std::begin(Form_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { - -void to_json(json& j, const Marker& p) { - j = json::object(); - to_json_key(j, "type", p.type, "Marker", "Type", "type"); - to_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - to_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); -} - -void from_json(const json& j, Marker& p) { - from_json_key(j, "type", p.type, "Marker", "Type", "type"); - from_json_key(j, "valueBlock", p.valueBlock, "Marker", "Block", "valueBlock"); - from_json_key(j, "bound", p.bound, "Marker", "Bound", "bound"); +SpecialFormExpression::SpecialFormExpression() noexcept { + _type = "special"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const TableToPartitionMapping& p) { +void to_json(json& j, const SpecialFormExpression& p) { j = json::object(); + j["@type"] = "special"; to_json_key( j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); - to_json_key( - j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); -} - -void from_json(const json& j, TableToPartitionMapping& p) { - from_json_key( - j, - "tableToPartitionColumns", - p.tableToPartitionColumns, - "TableToPartitionMapping", - "Map", - "tableToPartitionColumns"); - from_json_key( - j, - "partitionSchemaDifference", - p.partitionSchemaDifference, - "TableToPartitionMapping", - "Map", - "partitionSchemaDifference"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TableScanNode::TableScanNode() noexcept { - _type = ".TableScanNode"; -} - -void to_json(json& j, const TableScanNode& p) { - j = json::object(); - j["@type"] = ".TableScanNode"; - to_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); - to_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + to_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); to_json_key( j, - "outputVariables", - p.outputVariables, - "TableScanNode", - "List", - "outputVariables"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); to_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } -void from_json(const json& j, TableScanNode& p) { +void from_json(const json& j, SpecialFormExpression& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "TableScanNode", "PlanNodeId", "id"); - from_json_key(j, "table", p.table, "TableScanNode", "TableHandle", "table"); from_json_key( j, - "outputVariables", - p.outputVariables, - "TableScanNode", - "List", - "outputVariables"); + "sourceLocation", + p.sourceLocation, + "SpecialFormExpression", + "SourceLocation", + "sourceLocation"); + from_json_key(j, "form", p.form, "SpecialFormExpression", "Form", "form"); from_json_key( j, - "assignments", - p.assignments, - "TableScanNode", - "Map>", - "assignments"); + "returnType", + p.returnType, + "SpecialFormExpression", + "Type", + "returnType"); + from_json_key( + j, + "arguments", + p.arguments, + "SpecialFormExpression", + "List>", + "arguments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EmptySplit::EmptySplit() noexcept { - _type = "$empty"; -} -void to_json(json& j, const EmptySplit& p) { +void to_json(json& j, const ValueEntry& p) { j = json::object(); - j["@type"] = "$empty"; - to_json_key( - j, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); + to_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); + to_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); } -void from_json(const json& j, EmptySplit& p) { - p._type = j["@type"]; - from_json_key( - j, - "connectorId", - p.connectorId, - "EmptySplit", - "ConnectorId", - "connectorId"); +void from_json(const json& j, ValueEntry& p) { + from_json_key(j, "type", p.type, "ValueEntry", "Type", "type"); + from_json_key(j, "block", p.block, "ValueEntry", "Block", "block"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ValuesNode::ValuesNode() noexcept { - _type = ".ValuesNode"; +EquatableValueSet::EquatableValueSet() noexcept { + _type = "equatable"; } -void to_json(json& j, const ValuesNode& p) { +void to_json(json& j, const EquatableValueSet& p) { j = json::object(); - j["@type"] = ".ValuesNode"; - to_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); - to_json_key( - j, - "outputVariables", - p.outputVariables, - "ValuesNode", - "List", - "outputVariables"); + j["@type"] = "equatable"; + to_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); to_json_key( - j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); + j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); to_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); } -void from_json(const json& j, ValuesNode& p) { - p._type = j["@type"]; - from_json_key( - j, "location", p.location, "ValuesNode", "SourceLocation", "location"); - from_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); - from_json_key( - j, - "outputVariables", - p.outputVariables, - "ValuesNode", - "List", - "outputVariables"); - from_json_key( - j, - "rows", - p.rows, - "ValuesNode", - "List>>", - "rows"); +void from_json(const json& j, EquatableValueSet& p) { + p._type = j["@type"]; + from_json_key(j, "type", p.type, "EquatableValueSet", "Type", "type"); + from_json_key( + j, "whiteList", p.whiteList, "EquatableValueSet", "bool", "whiteList"); from_json_key( j, - "valuesNodeLabel", - p.valuesNodeLabel, - "ValuesNode", - "String", - "valuesNodeLabel"); + "entries", + p.entries, + "EquatableValueSet", + "List", + "entries"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair RuntimeUnit_enum_table[] = - { // NOLINT: cert-err58-cpp - {RuntimeUnit::NONE, "NONE"}, - {RuntimeUnit::NANO, "NANO"}, - {RuntimeUnit::BYTE, "BYTE"}}; -void to_json(json& j, const RuntimeUnit& e) { +static const std::pair + AggregationNodeStep_enum_table[] = { // NOLINT: cert-err58-cpp + {AggregationNodeStep::PARTIAL, "PARTIAL"}, + {AggregationNodeStep::FINAL, "FINAL"}, + {AggregationNodeStep::INTERMEDIATE, "INTERMEDIATE"}, + {AggregationNodeStep::SINGLE, "SINGLE"}}; +void to_json(json& j, const AggregationNodeStep& e) { static_assert( - std::is_enum::value, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "AggregationNodeStep must be an enum!"); const auto* it = std::find_if( - std::begin(RuntimeUnit_enum_table), - std::end(RuntimeUnit_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(RuntimeUnit_enum_table)) + j = ((it != std::end(AggregationNodeStep_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(AggregationNodeStep_enum_table)) ->second; } -void from_json(const json& j, RuntimeUnit& e) { +void from_json(const json& j, AggregationNodeStep& e) { static_assert( - std::is_enum::value, "RuntimeUnit must be an enum!"); + std::is_enum::value, + "AggregationNodeStep must be an enum!"); const auto* it = std::find_if( - std::begin(RuntimeUnit_enum_table), - std::end(RuntimeUnit_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(AggregationNodeStep_enum_table), + std::end(AggregationNodeStep_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(RuntimeUnit_enum_table)) + e = ((it != std::end(AggregationNodeStep_enum_table)) ? it - : std::begin(RuntimeUnit_enum_table)) + : std::begin(AggregationNodeStep_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const RuntimeMetric& p) { +void to_json(json& j, const GroupingSetDescriptor& p) { j = json::object(); - to_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); - to_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); - to_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); - to_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); - to_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); - to_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); + to_json_key( + j, + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", + "List", + "groupingKeys"); + to_json_key( + j, + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); + to_json_key( + j, + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } -void from_json(const json& j, RuntimeMetric& p) { - from_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); - from_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); - from_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); - from_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); - from_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); - from_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); +void from_json(const json& j, GroupingSetDescriptor& p) { + from_json_key( + j, + "groupingKeys", + p.groupingKeys, + "GroupingSetDescriptor", + "List", + "groupingKeys"); + from_json_key( + j, + "groupingSetCount", + p.groupingSetCount, + "GroupingSetDescriptor", + "int", + "groupingSetCount"); + from_json_key( + j, + "globalGroupingSets", + p.globalGroupingSets, + "GroupingSetDescriptor", + "List", + "globalGroupingSets"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -AssignUniqueId::AssignUniqueId() noexcept { - _type = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; +AggregationNode::AggregationNode() noexcept { + _type = ".AggregationNode"; } -void to_json(json& j, const AssignUniqueId& p) { +void to_json(json& j, const AggregationNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.AssignUniqueId"; - to_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + j["@type"] = ".AggregationNode"; + to_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); to_json_key( j, - "idVariable", - p.idVariable, - "AssignUniqueId", + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); + to_json_key( + j, + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); + to_json_key( + j, + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); + to_json_key( + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "AggregationNode", "VariableReferenceExpression", - "idVariable"); + "hashVariable"); + to_json_key( + j, + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); } -void from_json(const json& j, AssignUniqueId& p) { +void from_json(const json& j, AggregationNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "AssignUniqueId", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "AssignUniqueId", "PlanNode", "source"); + from_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); from_json_key( j, - "idVariable", - p.idVariable, - "AssignUniqueId", + "aggregations", + p.aggregations, + "AggregationNode", + "Map", + "aggregations"); + from_json_key( + j, + "groupingSets", + p.groupingSets, + "AggregationNode", + "GroupingSetDescriptor", + "groupingSets"); + from_json_key( + j, + "preGroupedVariables", + p.preGroupedVariables, + "AggregationNode", + "List", + "preGroupedVariables"); + from_json_key( + j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + from_json_key( + j, + "hashVariable", + p.hashVariable, + "AggregationNode", "VariableReferenceExpression", - "idVariable"); + "hashVariable"); + from_json_key( + j, + "groupIdVariable", + p.groupIdVariable, + "AggregationNode", + "VariableReferenceExpression", + "groupIdVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteTransactionHandle::RemoteTransactionHandle() noexcept { - _type = "$remote"; +ConstantExpression::ConstantExpression() noexcept { + _type = "constant"; } -void to_json(json& j, const RemoteTransactionHandle& p) { +void to_json(json& j, const ConstantExpression& p) { j = json::object(); - j["@type"] = "$remote"; + j["@type"] = "constant"; to_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); + j, + "valueBlock", + p.valueBlock, + "ConstantExpression", + "Block", + "valueBlock"); + to_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); } -void from_json(const json& j, RemoteTransactionHandle& p) { +void from_json(const json& j, ConstantExpression& p) { p._type = j["@type"]; from_json_key( - j, "dummy", p.dummy, "RemoteTransactionHandle", "String", "dummy"); + j, + "valueBlock", + p.valueBlock, + "ConstantExpression", + "Block", + "valueBlock"); + from_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { +void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { return; } String type = p->_type; - if (getConnectorKey(type) == "hive") { - j = *std::static_pointer_cast(p); + if (type == "equatable") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "sortable") { + j = *std::static_pointer_cast(p); + return; + } + if (type == "allOrNone") { + j = *std::static_pointer_cast(p); return; } - throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); + throw TypeError(type + " no abstract type ValueSet "); } -void from_json(const json& j, std::shared_ptr& p) { +void from_json(const json& j, std::shared_ptr& p) { String type; try { type = p->getSubclassKey(j); } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ConnectorInsertTableHandle ConnectorInsertTableHandle"); + throw ParseError(std::string(e.what()) + " ValueSet ValueSet"); } - if (getConnectorKey(type) == "hive") { - std::shared_ptr k = - std::make_shared(); + if (type == "equatable") { + std::shared_ptr k = + std::make_shared(); j.get_to(*k); - p = std::static_pointer_cast(k); + p = std::static_pointer_cast(k); + return; + } + if (type == "sortable") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + if (type == "allOrNone") { + std::shared_ptr k = + std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); return; } - throw TypeError(type + " no abstract type ConnectorInsertTableHandle "); + throw TypeError(type + " no abstract type ValueSet "); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const InsertTableHandle& p) { +void to_json(json& j, const Domain& p) { j = json::object(); - to_json_key( - j, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); - to_json_key( - j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); - to_json_key( - j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); + to_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); + to_json_key(j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); } -void from_json(const json& j, InsertTableHandle& p) { - from_json_key( - j, - "connectorId", - p.connectorId, - "InsertTableHandle", - "ConnectorId", - "connectorId"); - from_json_key( - j, - "transactionHandle", - p.transactionHandle, - "InsertTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); +void from_json(const json& j, Domain& p) { + from_json_key(j, "values", p.values, "Domain", "ValueSet", "values"); from_json_key( - j, - "connectorHandle", - p.connectorHandle, - "InsertTableHandle", - "ConnectorInsertTableHandle", - "connectorHandle"); + j, "nullAllowed", p.nullAllowed, "Domain", "bool", "nullAllowed"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { - -void to_json(json& j, const RefreshMaterializedViewHandle& p) { - j = json::object(); - to_json_key( - j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); - to_json_key( - j, - "schemaTableName", - p.schemaTableName, - "RefreshMaterializedViewHandle", - "SchemaTableName", - "schemaTableName"); -} - -void from_json(const json& j, RefreshMaterializedViewHandle& p) { - from_json_key( - j, - "handle", - p.handle, - "RefreshMaterializedViewHandle", - "InsertTableHandle", - "handle"); - from_json_key( - j, - "schemaTableName", - p.schemaTableName, - "RefreshMaterializedViewHandle", - "SchemaTableName", - "schemaTableName"); +TpchTableHandle::TpchTableHandle() noexcept { + _type = "tpch"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const HiveFileSplit& p) { +void to_json(json& j, const TpchTableHandle& p) { j = json::object(); - to_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); - to_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); - to_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); - to_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); - to_json_key( - j, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", - "int64_t", - "fileModifiedTime"); + j["@type"] = "tpch"; to_json_key( - j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", - "String", - "extraFileInfo"); + j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); to_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); } -void from_json(const json& j, HiveFileSplit& p) { - from_json_key(j, "path", p.path, "HiveFileSplit", "String", "path"); - from_json_key(j, "start", p.start, "HiveFileSplit", "int64_t", "start"); - from_json_key(j, "length", p.length, "HiveFileSplit", "int64_t", "length"); - from_json_key( - j, "fileSize", p.fileSize, "HiveFileSplit", "int64_t", "fileSize"); - from_json_key( - j, - "fileModifiedTime", - p.fileModifiedTime, - "HiveFileSplit", - "int64_t", - "fileModifiedTime"); +void from_json(const json& j, TpchTableHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "extraFileInfo", - p.extraFileInfo, - "HiveFileSplit", - "String", - "extraFileInfo"); + j, "tableName", p.tableName, "TpchTableHandle", "String", "tableName"); from_json_key( j, - "customSplitInfo", - p.customSplitInfo, - "HiveFileSplit", - "Map", - "customSplitInfo"); + "scaleFactor", + p.scaleFactor, + "TpchTableHandle", + "double", + "scaleFactor"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - StageExecutionStrategy_enum_table[] = { // NOLINT: cert-err58-cpp - {StageExecutionStrategy::UNGROUPED_EXECUTION, "UNGROUPED_EXECUTION"}, - {StageExecutionStrategy::FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - "FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION"}, - {StageExecutionStrategy::DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - "DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION"}, - {StageExecutionStrategy::RECOVERABLE_GROUPED_EXECUTION, - "RECOVERABLE_GROUPED_EXECUTION"}}; -void to_json(json& j, const StageExecutionStrategy& e) { - static_assert( - std::is_enum::value, - "StageExecutionStrategy must be an enum!"); - const auto* it = std::find_if( - std::begin(StageExecutionStrategy_enum_table), - std::end(StageExecutionStrategy_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(StageExecutionStrategy_enum_table)) - ? it - : std::begin(StageExecutionStrategy_enum_table)) - ->second; -} -void from_json(const json& j, StageExecutionStrategy& e) { - static_assert( - std::is_enum::value, - "StageExecutionStrategy must be an enum!"); - const auto* it = std::find_if( - std::begin(StageExecutionStrategy_enum_table), - std::end(StageExecutionStrategy_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(StageExecutionStrategy_enum_table)) - ? it - : std::begin(StageExecutionStrategy_enum_table)) - ->first; +TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { + _type = "tpch"; } -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const StageExecutionDescriptor& p) { +void to_json(json& j, const TpchTableLayoutHandle& p) { j = json::object(); + j["@type"] = "tpch"; to_json_key( - j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); - to_json_key( - j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); to_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); } -void from_json(const json& j, StageExecutionDescriptor& p) { - from_json_key( - j, - "stageExecutionStrategy", - p.stageExecutionStrategy, - "StageExecutionDescriptor", - "StageExecutionStrategy", - "stageExecutionStrategy"); +void from_json(const json& j, TpchTableLayoutHandle& p) { + p._type = j["@type"]; from_json_key( - j, - "groupedExecutionScanNodes", - p.groupedExecutionScanNodes, - "StageExecutionDescriptor", - "List", - "groupedExecutionScanNodes"); + j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); from_json_key( j, - "totalLifespans", - p.totalLifespans, - "StageExecutionDescriptor", - "int", - "totalLifespans"); + "predicate", + p.predicate, + "TpchTableLayoutHandle", + "TupleDomain>", + "predicate"); } } // namespace facebook::presto::protocol /* @@ -8767,217 +8891,120 @@ void from_json(const json& j, StageExecutionDescriptor& p) { * See the License for the specific language governing permissions and * limitations under the License. */ - namespace facebook::presto::protocol { -void to_json(json& j, const PlanFragment& p) { - j = json::object(); - to_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); - to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); - to_json_key( - j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); - to_json_key( - j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); - to_json_key( - j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); - to_json_key( - j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); - to_json_key( - j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); - to_json_key( - j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); - to_json_key( - j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", - "String", - "jsonRepresentation"); +void to_json(json& j, const HiveStorageFormat& p) { + throw ParseError("Not implemented"); } -void from_json(const json& j, PlanFragment& p) { - from_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); - from_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); - from_json_key( - j, - "variables", - p.variables, - "PlanFragment", - "List", - "variables"); - from_json_key( - j, - "partitioning", - p.partitioning, - "PlanFragment", - "PartitioningHandle", - "partitioning"); - from_json_key( - j, - "tableScanSchedulingOrder", - p.tableScanSchedulingOrder, - "PlanFragment", - "List", - "tableScanSchedulingOrder"); - from_json_key( - j, - "partitioningScheme", - p.partitioningScheme, - "PlanFragment", - "PartitioningScheme", - "partitioningScheme"); - from_json_key( - j, - "stageExecutionDescriptor", - p.stageExecutionDescriptor, - "PlanFragment", - "StageExecutionDescriptor", - "stageExecutionDescriptor"); - from_json_key( - j, - "outputTableWriterFragment", - p.outputTableWriterFragment, - "PlanFragment", - "bool", - "outputTableWriterFragment"); - from_json_key( - j, - "jsonRepresentation", - p.jsonRepresentation, - "PlanFragment", - "String", - "jsonRepresentation"); +static const std::pair HiveStorageFormat_enum_table[] = + { // NOLINT: cert-err58-cpp + {HiveStorageFormat::ORC, "ORC"}, + {HiveStorageFormat::DWRF, "DWRF"}, + {HiveStorageFormat::PARQUET, "PARQUET"}, + {HiveStorageFormat::AVRO, "AVRO"}, + {HiveStorageFormat::RCBINARY, "RCBINARY"}, + {HiveStorageFormat::RCTEXT, "RCTEXT"}, + {HiveStorageFormat::SEQUENCEFILE, "SEQUENCEFILE"}, + {HiveStorageFormat::JSON, "JSON"}, + {HiveStorageFormat::TEXTFILE, "TEXTFILE"}, + {HiveStorageFormat::CSV, "CSV"}, + {HiveStorageFormat::PAGEFILE, "PAGEFILE"}}; + +void from_json(const json& j, HiveStorageFormat& e) { + static_assert( + std::is_enum::value, + "HiveStorageFormat must be an enum!"); + const auto* it = std::find_if( + std::begin(HiveStorageFormat_enum_table), + std::end(HiveStorageFormat_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(HiveStorageFormat_enum_table)) + ? it + : std::begin(HiveStorageFormat_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair SelectedRoleType_enum_table[] = - { // NOLINT: cert-err58-cpp - {SelectedRoleType::ROLE, "ROLE"}, - {SelectedRoleType::ALL, "ALL"}, - {SelectedRoleType::NONE, "NONE"}}; -void to_json(json& j, const SelectedRoleType& e) { +static const std::pair + HiveCompressionCodec_enum_table[] = { // NOLINT: cert-err58-cpp + {HiveCompressionCodec::NONE, "NONE"}, + {HiveCompressionCodec::SNAPPY, "SNAPPY"}, + {HiveCompressionCodec::GZIP, "GZIP"}, + {HiveCompressionCodec::LZ4, "LZ4"}, + {HiveCompressionCodec::ZSTD, "ZSTD"}}; +void to_json(json& j, const HiveCompressionCodec& e) { static_assert( - std::is_enum::value, - "SelectedRoleType must be an enum!"); + std::is_enum::value, + "HiveCompressionCodec must be an enum!"); const auto* it = std::find_if( - std::begin(SelectedRoleType_enum_table), - std::end(SelectedRoleType_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(HiveCompressionCodec_enum_table), + std::end(HiveCompressionCodec_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(SelectedRoleType_enum_table)) + j = ((it != std::end(HiveCompressionCodec_enum_table)) ? it - : std::begin(SelectedRoleType_enum_table)) + : std::begin(HiveCompressionCodec_enum_table)) ->second; } -void from_json(const json& j, SelectedRoleType& e) { +void from_json(const json& j, HiveCompressionCodec& e) { static_assert( - std::is_enum::value, - "SelectedRoleType must be an enum!"); + std::is_enum::value, + "HiveCompressionCodec must be an enum!"); const auto* it = std::find_if( - std::begin(SelectedRoleType_enum_table), - std::end(SelectedRoleType_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(HiveCompressionCodec_enum_table), + std::end(HiveCompressionCodec_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(SelectedRoleType_enum_table)) + e = ((it != std::end(HiveCompressionCodec_enum_table)) ? it - : std::begin(SelectedRoleType_enum_table)) + : std::begin(HiveCompressionCodec_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { - -void to_json(json& j, const SelectedRole& p) { - j = json::object(); - to_json_key(j, "type", p.type, "SelectedRole", "SelectedRoleType", "type"); - to_json_key(j, "role", p.role, "SelectedRole", "String", "role"); -} - -void from_json(const json& j, SelectedRole& p) { - from_json_key(j, "type", p.type, "SelectedRole", "SelectedRoleType", "type"); - from_json_key(j, "role", p.role, "SelectedRole", "String", "role"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Language& p) { - j = json::object(); - to_json_key(j, "language", p.language, "Language", "String", "language"); -} - -void from_json(const json& j, Language& p) { - from_json_key(j, "language", p.language, "Language", "String", "language"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Determinism_enum_table[] = { - // NOLINT: cert-err58-cpp - {Determinism::DETERMINISTIC, "DETERMINISTIC"}, - {Determinism::NOT_DETERMINISTIC, "NOT_DETERMINISTIC"}, -}; -void to_json(json& j, const Determinism& e) { - static_assert( - std::is_enum::value, "Determinism must be an enum!"); +static const std::pair WriteMode_enum_table[] = + { // NOLINT: cert-err58-cpp + {WriteMode::STAGE_AND_MOVE_TO_TARGET_DIRECTORY, + "STAGE_AND_MOVE_TO_TARGET_DIRECTORY"}, + {WriteMode::DIRECT_TO_TARGET_NEW_DIRECTORY, + "DIRECT_TO_TARGET_NEW_DIRECTORY"}, + {WriteMode::DIRECT_TO_TARGET_EXISTING_DIRECTORY, + "DIRECT_TO_TARGET_EXISTING_DIRECTORY"}}; +void to_json(json& j, const WriteMode& e) { + static_assert(std::is_enum::value, "WriteMode must be an enum!"); const auto* it = std::find_if( - std::begin(Determinism_enum_table), - std::end(Determinism_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(WriteMode_enum_table), + std::end(WriteMode_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(Determinism_enum_table)) + j = ((it != std::end(WriteMode_enum_table)) ? it - : std::begin(Determinism_enum_table)) + : std::begin(WriteMode_enum_table)) ->second; } -void from_json(const json& j, Determinism& e) { - static_assert( - std::is_enum::value, "Determinism must be an enum!"); +void from_json(const json& j, WriteMode& e) { + static_assert(std::is_enum::value, "WriteMode must be an enum!"); const auto* it = std::find_if( - std::begin(Determinism_enum_table), - std::end(Determinism_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(WriteMode_enum_table), + std::end(WriteMode_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(Determinism_enum_table)) + e = ((it != std::end(WriteMode_enum_table)) ? it - : std::begin(Determinism_enum_table)) + : std::begin(WriteMode_enum_table)) ->first; } } // namespace facebook::presto::protocol @@ -8985,2792 +9012,2589 @@ namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair NullCallClause_enum_table[] = +static const std::pair TableType_enum_table[] = { // NOLINT: cert-err58-cpp - {NullCallClause::RETURNS_NULL_ON_NULL_INPUT, - "RETURNS_NULL_ON_NULL_INPUT"}, - {NullCallClause::CALLED_ON_NULL_INPUT, "CALLED_ON_NULL_INPUT"}}; -void to_json(json& j, const NullCallClause& e) { - static_assert( - std::is_enum::value, "NullCallClause must be an enum!"); + {TableType::NEW, "NEW"}, + {TableType::EXISTING, "EXISTING"}, + {TableType::TEMPORARY, "TEMPORARY"}}; +void to_json(json& j, const TableType& e) { + static_assert(std::is_enum::value, "TableType must be an enum!"); const auto* it = std::find_if( - std::begin(NullCallClause_enum_table), - std::end(NullCallClause_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(TableType_enum_table), + std::end(TableType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(NullCallClause_enum_table)) + j = ((it != std::end(TableType_enum_table)) ? it - : std::begin(NullCallClause_enum_table)) + : std::begin(TableType_enum_table)) ->second; } -void from_json(const json& j, NullCallClause& e) { - static_assert( - std::is_enum::value, "NullCallClause must be an enum!"); +void from_json(const json& j, TableType& e) { + static_assert(std::is_enum::value, "TableType must be an enum!"); const auto* it = std::find_if( - std::begin(NullCallClause_enum_table), - std::end(NullCallClause_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(TableType_enum_table), + std::end(TableType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(NullCallClause_enum_table)) + e = ((it != std::end(TableType_enum_table)) ? it - : std::begin(NullCallClause_enum_table)) + : std::begin(TableType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const RoutineCharacteristics& p) { +void to_json(json& j, const LocationHandle& p) { + j = json::object(); + to_json_key( + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + to_json_key( + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + to_json_key( + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + to_json_key( + j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + to_json_key( + j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); +} + +void from_json(const json& j, LocationHandle& p) { + from_json_key( + j, "targetPath", p.targetPath, "LocationHandle", "String", "targetPath"); + from_json_key( + j, "writePath", p.writePath, "LocationHandle", "String", "writePath"); + from_json_key( + j, "tempPath", p.tempPath, "LocationHandle", "String", "tempPath"); + from_json_key( + j, "tableType", p.tableType, "LocationHandle", "TableType", "tableType"); + from_json_key( + j, "writeMode", p.writeMode, "LocationHandle", "WriteMode", "writeMode"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +HiveInsertTableHandle::HiveInsertTableHandle() noexcept { + _type = "hive"; +} + +void to_json(json& j, const HiveInsertTableHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); to_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); to_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); + to_json_key( + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); + to_json_key( + j, + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); + to_json_key( + j, + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); + to_json_key( + j, + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); + to_json_key( + j, + "tableStorageFormat", + p.tableStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + to_json_key( + j, + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + to_json_key( + j, + "actualStorageFormat", + p.actualStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + to_json_key( + j, + "compressionCodec", + p.compressionCodec, + "HiveInsertTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + to_json_key( + j, + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); } -void from_json(const json& j, RoutineCharacteristics& p) { +void from_json(const json& j, HiveInsertTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "language", - p.language, - "RoutineCharacteristics", - "Language", - "language"); + "schemaName", + p.schemaName, + "HiveInsertTableHandle", + "String", + "schemaName"); from_json_key( j, - "determinism", - p.determinism, - "RoutineCharacteristics", - "Determinism", - "determinism"); + "tableName", + p.tableName, + "HiveInsertTableHandle", + "String", + "tableName"); + from_json_key( + j, + "inputColumns", + p.inputColumns, + "HiveInsertTableHandle", + "List", + "inputColumns"); + from_json_key( + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveInsertTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); + from_json_key( + j, + "locationHandle", + p.locationHandle, + "HiveInsertTableHandle", + "LocationHandle", + "locationHandle"); + from_json_key( + j, + "bucketProperty", + p.bucketProperty, + "HiveInsertTableHandle", + "HiveBucketProperty", + "bucketProperty"); + from_json_key( + j, + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveInsertTableHandle", + "List", + "preferredOrderingColumns"); + from_json_key( + j, + "tableStorageFormat", + p.tableStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); + from_json_key( + j, + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + from_json_key( + j, + "actualStorageFormat", + p.actualStorageFormat, + "HiveInsertTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + from_json_key( + j, + "compressionCodec", + p.compressionCodec, + "HiveInsertTableHandle", + "HiveCompressionCodec", + "compressionCodec"); from_json_key( j, - "nullCallClause", - p.nullCallClause, - "RoutineCharacteristics", - "NullCallClause", - "nullCallClause"); + "encryptionInformation", + p.encryptionInformation, + "HiveInsertTableHandle", + "EncryptionInformation", + "encryptionInformation"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const Parameter& p) { +void to_json(json& j, const Assignments& p) { j = json::object(); - to_json_key(j, "name", p.name, "Parameter", "String", "name"); - to_json_key(j, "type", p.type, "Parameter", "TypeSignature", "type"); + to_json_key( + j, + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); } -void from_json(const json& j, Parameter& p) { - from_json_key(j, "name", p.name, "Parameter", "String", "name"); - from_json_key(j, "type", p.type, "Parameter", "TypeSignature", "type"); +void from_json(const json& j, Assignments& p) { + from_json_key( + j, + "assignments", + p.assignments, + "Assignments", + "Map>", + "assignments"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const SqlInvokedFunction& p) { +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Locality_enum_table[] = + { // NOLINT: cert-err58-cpp + {Locality::UNKNOWN, "UNKNOWN"}, + {Locality::LOCAL, "LOCAL"}, + {Locality::REMOTE, "REMOTE"}}; +void to_json(json& j, const Locality& e) { + static_assert(std::is_enum::value, "Locality must be an enum!"); + const auto* it = std::find_if( + std::begin(Locality_enum_table), + std::end(Locality_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) + ->second; +} +void from_json(const json& j, Locality& e) { + static_assert(std::is_enum::value, "Locality must be an enum!"); + const auto* it = std::find_if( + std::begin(Locality_enum_table), + std::end(Locality_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Locality_enum_table)) ? it + : std::begin(Locality_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +ProjectNode::ProjectNode() noexcept { + _type = ".ProjectNode"; +} + +void to_json(json& j, const ProjectNode& p) { j = json::object(); + j["@type"] = ".ProjectNode"; + to_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); to_json_key( j, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); - to_json_key( - j, - "description", - p.description, - "SqlInvokedFunction", - "String", - "description"); - to_json_key( - j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - to_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); - to_json_key( - j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); - to_json_key( - j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + "assignments", + p.assignments, + "ProjectNode", + "Assignments", + "assignments"); + to_json_key(j, "locality", p.locality, "ProjectNode", "Locality", "locality"); } -void from_json(const json& j, SqlInvokedFunction& p) { - from_json_key( - j, - "parameters", - p.parameters, - "SqlInvokedFunction", - "List", - "parameters"); - from_json_key( - j, - "description", - p.description, - "SqlInvokedFunction", - "String", - "description"); - from_json_key( - j, - "routineCharacteristics", - p.routineCharacteristics, - "SqlInvokedFunction", - "RoutineCharacteristics", - "routineCharacteristics"); - from_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); +void from_json(const json& j, ProjectNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); from_json_key( j, - "signature", - p.signature, - "SqlInvokedFunction", - "Signature", - "signature"); + "assignments", + p.assignments, + "ProjectNode", + "Assignments", + "assignments"); from_json_key( - j, - "functionId", - p.functionId, - "SqlInvokedFunction", - "SqlFunctionId", - "functionId"); + j, "locality", p.locality, "ProjectNode", "Locality", "locality"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const SessionRepresentation& p) { - j = json::object(); - to_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); - to_json_key( - j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); - to_json_key( - j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", - "bool", - "clientTransactionSupport"); - to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); - to_json_key( - j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); - to_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); - to_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); - to_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); - to_json_key( - j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); - to_json_key( - j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - to_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); - to_json_key( - j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); - to_json_key( - j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair Step_enum_table[] = + { // NOLINT: cert-err58-cpp + {Step::SINGLE, "SINGLE"}, + {Step::PARTIAL, "PARTIAL"}, + {Step::FINAL, "FINAL"}}; +void to_json(json& j, const Step& e) { + static_assert(std::is_enum::value, "Step must be an enum!"); + const auto* it = std::find_if( + std::begin(Step_enum_table), + std::end(Step_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + ->second; +} +void from_json(const json& j, Step& e) { + static_assert(std::is_enum::value, "Step must be an enum!"); + const auto* it = std::find_if( + std::begin(Step_enum_table), + std::end(Step_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(Step_enum_table)) ? it : std::begin(Step_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +TopNNode::TopNNode() noexcept { + _type = ".TopNNode"; +} + +void to_json(json& j, const TopNNode& p) { + j = json::object(); + j["@type"] = ".TopNNode"; + to_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); + to_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); to_json_key( j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); - to_json_key( + "orderingScheme", + p.orderingScheme, + "TopNNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "step", p.step, "TopNNode", "Step", "step"); +} + +void from_json(const json& j, TopNNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "TopNNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "TopNNode", "PlanNode", "source"); + from_json_key(j, "count", p.count, "TopNNode", "int64_t", "count"); + from_json_key( j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); + "orderingScheme", + p.orderingScheme, + "TopNNode", + "OrderingScheme", + "orderingScheme"); + from_json_key(j, "step", p.step, "TopNNode", "Step", "step"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +SemiJoinNode::SemiJoinNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; +} + +void to_json(json& j, const SemiJoinNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; + to_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); to_json_key( j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); to_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceJoinVariable"); to_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceJoinVariable"); to_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", + "VariableReferenceExpression", + "semiJoinOutput"); to_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); to_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); to_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); to_json_key( j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } -void from_json(const json& j, SessionRepresentation& p) { - from_json_key( - j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); - from_json_key( - j, - "transactionId", - p.transactionId, - "SessionRepresentation", - "TransactionId", - "transactionId"); - from_json_key( - j, - "clientTransactionSupport", - p.clientTransactionSupport, - "SessionRepresentation", - "bool", - "clientTransactionSupport"); - from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); - from_json_key( - j, - "principal", - p.principal, - "SessionRepresentation", - "String", - "principal"); - from_json_key( - j, "source", p.source, "SessionRepresentation", "String", "source"); - from_json_key( - j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); - from_json_key( - j, "schema", p.schema, "SessionRepresentation", "String", "schema"); - from_json_key( - j, - "traceToken", - p.traceToken, - "SessionRepresentation", - "String", - "traceToken"); - from_json_key( - j, - "timeZoneKey", - p.timeZoneKey, - "SessionRepresentation", - "TimeZoneKey", - "timeZoneKey"); - from_json_key( - j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); - from_json_key( - j, - "remoteUserAddress", - p.remoteUserAddress, - "SessionRepresentation", - "String", - "remoteUserAddress"); - from_json_key( - j, - "userAgent", - p.userAgent, - "SessionRepresentation", - "String", - "userAgent"); - from_json_key( - j, - "clientInfo", - p.clientInfo, - "SessionRepresentation", - "String", - "clientInfo"); - from_json_key( - j, - "clientTags", - p.clientTags, - "SessionRepresentation", - "List", - "clientTags"); - from_json_key( - j, - "resourceEstimates", - p.resourceEstimates, - "SessionRepresentation", - "ResourceEstimates", - "resourceEstimates"); +void from_json(const json& j, SemiJoinNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); from_json_key( j, - "startTime", - p.startTime, - "SessionRepresentation", - "int64_t", - "startTime"); + "filteringSource", + p.filteringSource, + "SemiJoinNode", + "PlanNode", + "filteringSource"); from_json_key( j, - "systemProperties", - p.systemProperties, - "SessionRepresentation", - "Map", - "systemProperties"); + "sourceJoinVariable", + p.sourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceJoinVariable"); from_json_key( j, - "catalogProperties", - p.catalogProperties, - "SessionRepresentation", - "Map>", - "catalogProperties"); + "filteringSourceJoinVariable", + p.filteringSourceJoinVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceJoinVariable"); from_json_key( j, - "unprocessedCatalogProperties", - p.unprocessedCatalogProperties, - "SessionRepresentation", - "Map>", - "unprocessedCatalogProperties"); + "semiJoinOutput", + p.semiJoinOutput, + "SemiJoinNode", + "VariableReferenceExpression", + "semiJoinOutput"); from_json_key( j, - "roles", - p.roles, - "SessionRepresentation", - "Map", - "roles"); + "sourceHashVariable", + p.sourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "sourceHashVariable"); from_json_key( j, - "preparedStatements", - p.preparedStatements, - "SessionRepresentation", - "Map", - "preparedStatements"); + "filteringSourceHashVariable", + p.filteringSourceHashVariable, + "SemiJoinNode", + "VariableReferenceExpression", + "filteringSourceHashVariable"); from_json_key( j, - "sessionFunctions", - p.sessionFunctions, - "SessionRepresentation", - "Map", - "sessionFunctions"); + "distributionType", + p.distributionType, + "SemiJoinNode", + "DistributionType", + "distributionType"); + from_json_key( + j, + "dynamicFilters", + p.dynamicFilters, + "SemiJoinNode", + "Map", + "dynamicFilters"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveOutputTableHandle::HiveOutputTableHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const ScheduledSplit& p) { +void to_json(json& j, const HiveOutputTableHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); + j, + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); to_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - to_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); -} - -void from_json(const json& j, ScheduledSplit& p) { - from_json_key( - j, "sequenceId", p.sequenceId, "ScheduledSplit", "int64_t", "sequenceId"); - from_json_key( + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); + to_json_key( j, - "planNodeId", - p.planNodeId, - "ScheduledSplit", - "PlanNodeId", - "planNodeId"); - from_json_key(j, "split", p.split, "ScheduledSplit", "Split", "split"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TaskSource& p) { - j = json::object(); + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); to_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); + j, + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); to_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); + j, + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); to_json_key( j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); to_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); -} - -void from_json(const json& j, TaskSource& p) { - from_json_key( - j, "planNodeId", p.planNodeId, "TaskSource", "PlanNodeId", "planNodeId"); - from_json_key( - j, "splits", p.splits, "TaskSource", "List", "splits"); - from_json_key( j, - "noMoreSplitsForLifespan", - p.noMoreSplitsForLifespan, - "TaskSource", - "List", - "noMoreSplitsForLifespan"); - from_json_key( - j, "noMoreSplits", p.noMoreSplits, "TaskSource", "bool", "noMoreSplits"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const DeleteScanInfo& p) { - j = json::object(); - to_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); to_json_key( j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); -} - -void from_json(const json& j, DeleteScanInfo& p) { - from_json_key(j, "id", p.id, "DeleteScanInfo", "PlanNodeId", "id"); - from_json_key( + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + to_json_key( j, - "tableHandle", - p.tableHandle, - "DeleteScanInfo", - "TableHandle", - "tableHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -void to_json(json& j, const std::shared_ptr& p) { - if (p == nullptr) { - return; - } - String type = p->_type; - - if (type == "CreateHandle") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "InsertHandle") { - j = *std::static_pointer_cast(p); - return; - } - if (type == "DeleteHandle") { - j = *std::static_pointer_cast(p); - return; - } - - throw TypeError(type + " no abstract type ExecutionWriterTarget "); -} - -void from_json(const json& j, std::shared_ptr& p) { - String type; - try { - type = p->getSubclassKey(j); - } catch (json::parse_error& e) { - throw ParseError( - std::string(e.what()) + - " ExecutionWriterTarget ExecutionWriterTarget"); - } - - if (type == "CreateHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "InsertHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - if (type == "DeleteHandle") { - std::shared_ptr k = std::make_shared(); - j.get_to(*k); - p = std::static_pointer_cast(k); - return; - } - - throw TypeError(type + " no abstract type ExecutionWriterTarget "); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const AnalyzeTableHandle& p) { - j = json::object(); + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); to_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); + to_json_key( + j, + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); + to_json_key( + j, + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); + to_json_key( + j, + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); to_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", + "Map", + "additionalTableParameters"); to_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); } -void from_json(const json& j, AnalyzeTableHandle& p) { +void from_json(const json& j, HiveOutputTableHandle& p) { + p._type = j["@type"]; from_json_key( j, - "connectorId", - p.connectorId, - "AnalyzeTableHandle", - "ConnectorId", - "connectorId"); + "schemaName", + p.schemaName, + "HiveOutputTableHandle", + "String", + "schemaName"); from_json_key( j, - "transactionHandle", - p.transactionHandle, - "AnalyzeTableHandle", - "ConnectorTransactionHandle", - "transactionHandle"); + "tableName", + p.tableName, + "HiveOutputTableHandle", + "String", + "tableName"); from_json_key( j, - "connectorHandle", - p.connectorHandle, - "AnalyzeTableHandle", - "ConnectorTableHandle", - "connectorHandle"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TableWriteInfo& p) { - j = json::object(); - to_json_key( - j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); - to_json_key( - j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); - to_json_key( + "inputColumns", + p.inputColumns, + "HiveOutputTableHandle", + "List", + "inputColumns"); + from_json_key( j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); -} - -void from_json(const json& j, TableWriteInfo& p) { + "pageSinkMetadata", + p.pageSinkMetadata, + "HiveOutputTableHandle", + "HivePageSinkMetadata", + "pageSinkMetadata"); from_json_key( j, - "writerTarget", - p.writerTarget, - "TableWriteInfo", - "ExecutionWriterTarget", - "writerTarget"); + "locationHandle", + p.locationHandle, + "HiveOutputTableHandle", + "LocationHandle", + "locationHandle"); from_json_key( j, - "analyzeTableHandle", - p.analyzeTableHandle, - "TableWriteInfo", - "AnalyzeTableHandle", - "analyzeTableHandle"); + "tableStorageFormat", + p.tableStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "tableStorageFormat"); from_json_key( j, - "deleteScanInfo", - p.deleteScanInfo, - "TableWriteInfo", - "DeleteScanInfo", - "deleteScanInfo"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TaskUpdateRequest& p) { - j = json::object(); - to_json_key( + "partitionStorageFormat", + p.partitionStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "partitionStorageFormat"); + from_json_key( j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); - to_json_key( + "actualStorageFormat", + p.actualStorageFormat, + "HiveOutputTableHandle", + "HiveStorageFormat", + "actualStorageFormat"); + from_json_key( j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", - "Map", - "extraCredentials"); - to_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); - to_json_key( + "compressionCodec", + p.compressionCodec, + "HiveOutputTableHandle", + "HiveCompressionCodec", + "compressionCodec"); + from_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); - to_json_key( + "partitionedBy", + p.partitionedBy, + "HiveOutputTableHandle", + "List", + "partitionedBy"); + from_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); - to_json_key( + "bucketProperty", + p.bucketProperty, + "HiveOutputTableHandle", + "HiveBucketProperty", + "bucketProperty"); + from_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); -} - -void from_json(const json& j, TaskUpdateRequest& p) { + "preferredOrderingColumns", + p.preferredOrderingColumns, + "HiveOutputTableHandle", + "List", + "preferredOrderingColumns"); from_json_key( j, - "session", - p.session, - "TaskUpdateRequest", - "SessionRepresentation", - "session"); + "tableOwner", + p.tableOwner, + "HiveOutputTableHandle", + "String", + "tableOwner"); from_json_key( j, - "extraCredentials", - p.extraCredentials, - "TaskUpdateRequest", + "additionalTableParameters", + p.additionalTableParameters, + "HiveOutputTableHandle", "Map", - "extraCredentials"); - from_json_key( - j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); + "additionalTableParameters"); from_json_key( j, - "sources", - p.sources, - "TaskUpdateRequest", - "List", - "sources"); - from_json_key( + "encryptionInformation", + p.encryptionInformation, + "HiveOutputTableHandle", + "EncryptionInformation", + "encryptionInformation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() + +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair FunctionKind_enum_table[] = + { // NOLINT: cert-err58-cpp + {FunctionKind::SCALAR, "SCALAR"}, + {FunctionKind::AGGREGATE, "AGGREGATE"}, + {FunctionKind::WINDOW, "WINDOW"}}; +void to_json(json& j, const FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->second; +} +void from_json(const json& j, FunctionKind& e) { + static_assert( + std::is_enum::value, "FunctionKind must be an enum!"); + const auto* it = std::find_if( + std::begin(FunctionKind_enum_table), + std::end(FunctionKind_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(FunctionKind_enum_table)) + ? it + : std::begin(FunctionKind_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const LongVariableConstraint& p) { + j = json::object(); + to_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); + to_json_key( j, - "outputIds", - p.outputIds, - "TaskUpdateRequest", - "OutputBuffers", - "outputIds"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); +} + +void from_json(const json& j, LongVariableConstraint& p) { + from_json_key(j, "name", p.name, "LongVariableConstraint", "String", "name"); from_json_key( j, - "tableWriteInfo", - p.tableWriteInfo, - "TaskUpdateRequest", - "TableWriteInfo", - "tableWriteInfo"); + "expression", + p.expression, + "LongVariableConstraint", + "String", + "expression"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -GroupIdNode::GroupIdNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.GroupIdNode"; -} -void to_json(json& j, const GroupIdNode& p) { +void to_json(json& j, const Signature& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.GroupIdNode"; - to_json_key(j, "id", p.id, "GroupIdNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "GroupIdNode", "PlanNode", "source"); + to_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); + to_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); to_json_key( j, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); to_json_key( j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); to_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); to_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); + to_json_key( + j, + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } -void from_json(const json& j, GroupIdNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "GroupIdNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "GroupIdNode", "PlanNode", "source"); +void from_json(const json& j, Signature& p) { + from_json_key(j, "name", p.name, "Signature", "QualifiedObjectName", "name"); + from_json_key(j, "kind", p.kind, "Signature", "FunctionKind", "kind"); from_json_key( j, - "groupingSets", - p.groupingSets, - "GroupIdNode", - "List>", - "groupingSets"); + "typeVariableConstraints", + p.typeVariableConstraints, + "Signature", + "List", + "typeVariableConstraints"); from_json_key( j, - "groupingColumns", - p.groupingColumns, - "GroupIdNode", - "Map", - "groupingColumns"); + "longVariableConstraints", + p.longVariableConstraints, + "Signature", + "List", + "longVariableConstraints"); from_json_key( j, - "aggregationArguments", - p.aggregationArguments, - "GroupIdNode", - "List", - "aggregationArguments"); + "returnType", + p.returnType, + "Signature", + "TypeSignature", + "returnType"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "GroupIdNode", - "VariableReferenceExpression", - "groupIdVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair Locality_enum_table[] = - { // NOLINT: cert-err58-cpp - {Locality::UNKNOWN, "UNKNOWN"}, - {Locality::LOCAL, "LOCAL"}, - {Locality::REMOTE, "REMOTE"}}; -void to_json(json& j, const Locality& e) { - static_assert(std::is_enum::value, "Locality must be an enum!"); - const auto* it = std::find_if( - std::begin(Locality_enum_table), - std::end(Locality_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(Locality_enum_table)) ? it - : std::begin(Locality_enum_table)) - ->second; -} -void from_json(const json& j, Locality& e) { - static_assert(std::is_enum::value, "Locality must be an enum!"); - const auto* it = std::find_if( - std::begin(Locality_enum_table), - std::end(Locality_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(Locality_enum_table)) ? it - : std::begin(Locality_enum_table)) - ->first; + "argumentTypes", + p.argumentTypes, + "Signature", + "List", + "argumentTypes"); + from_json_key( + j, + "variableArity", + p.variableArity, + "Signature", + "bool", + "variableArity"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ProjectNode::ProjectNode() noexcept { - _type = ".ProjectNode"; +BuiltInFunctionHandle::BuiltInFunctionHandle() noexcept { + _type = "$static"; } -void to_json(json& j, const ProjectNode& p) { +void to_json(json& j, const BuiltInFunctionHandle& p) { j = json::object(); - j["@type"] = ".ProjectNode"; - to_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); + j["@type"] = "$static"; to_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); - to_json_key(j, "locality", p.locality, "ProjectNode", "Locality", "locality"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); } -void from_json(const json& j, ProjectNode& p) { +void from_json(const json& j, BuiltInFunctionHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "ProjectNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "ProjectNode", "PlanNode", "source"); from_json_key( j, - "assignments", - p.assignments, - "ProjectNode", - "Assignments", - "assignments"); - from_json_key( - j, "locality", p.locality, "ProjectNode", "Locality", "locality"); + "signature", + p.signature, + "BuiltInFunctionHandle", + "Signature", + "signature"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair LimitNodeStep_enum_table[] = - { // NOLINT: cert-err58-cpp - {LimitNodeStep::PARTIAL, "PARTIAL"}, - {LimitNodeStep::FINAL, "FINAL"}}; -void to_json(json& j, const LimitNodeStep& e) { - static_assert( - std::is_enum::value, "LimitNodeStep must be an enum!"); - const auto* it = std::find_if( - std::begin(LimitNodeStep_enum_table), - std::end(LimitNodeStep_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(LimitNodeStep_enum_table)) - ? it - : std::begin(LimitNodeStep_enum_table)) - ->second; +AllOrNoneValueSet::AllOrNoneValueSet() noexcept { + _type = "allOrNone"; } -void from_json(const json& j, LimitNodeStep& e) { - static_assert( - std::is_enum::value, "LimitNodeStep must be an enum!"); - const auto* it = std::find_if( - std::begin(LimitNodeStep_enum_table), - std::end(LimitNodeStep_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(LimitNodeStep_enum_table)) - ? it - : std::begin(LimitNodeStep_enum_table)) - ->first; + +void to_json(json& j, const AllOrNoneValueSet& p) { + j = json::object(); + j["@type"] = "allOrNone"; + to_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); + to_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); +} + +void from_json(const json& j, AllOrNoneValueSet& p) { + p._type = j["@type"]; + from_json_key(j, "type", p.type, "AllOrNoneValueSet", "Type", "type"); + from_json_key(j, "all", p.all, "AllOrNoneValueSet", "bool", "all"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -LimitNode::LimitNode() noexcept { - _type = ".LimitNode"; +DeleteHandle::DeleteHandle() noexcept { + _type = "DeleteHandle"; } -void to_json(json& j, const LimitNode& p) { +void to_json(json& j, const DeleteHandle& p) { j = json::object(); - j["@type"] = ".LimitNode"; - to_json_key(j, "id", p.id, "LimitNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "LimitNode", "PlanNode", "source"); - to_json_key(j, "count", p.count, "LimitNode", "int64_t", "count"); - to_json_key(j, "step", p.step, "LimitNode", "LimitNodeStep", "step"); + j["@type"] = "DeleteHandle"; + to_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + to_json_key( + j, + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, LimitNode& p) { +void from_json(const json& j, DeleteHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "LimitNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "LimitNode", "PlanNode", "source"); - from_json_key(j, "count", p.count, "LimitNode", "int64_t", "count"); - from_json_key(j, "step", p.step, "LimitNode", "LimitNodeStep", "step"); + from_json_key(j, "handle", p.handle, "DeleteHandle", "TableHandle", "handle"); + from_json_key( + j, + "schemaTableName", + p.schemaTableName, + "DeleteHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -UnnestNode::UnnestNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.UnnestNode"; -} -void to_json(json& j, const UnnestNode& p) { +void to_json(json& j, const RefreshMaterializedViewHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.UnnestNode"; - to_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); - to_json_key( - j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); to_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); to_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", - "VariableReferenceExpression", - "ordinalityVariable"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); } -void from_json(const json& j, UnnestNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); - from_json_key( - j, - "replicateVariables", - p.replicateVariables, - "UnnestNode", - "List", - "replicateVariables"); +void from_json(const json& j, RefreshMaterializedViewHandle& p) { from_json_key( j, - "unnestVariables", - p.unnestVariables, - "UnnestNode", - "Map>", - "unnestVariables"); + "handle", + p.handle, + "RefreshMaterializedViewHandle", + "InsertTableHandle", + "handle"); from_json_key( j, - "ordinalityVariable", - p.ordinalityVariable, - "UnnestNode", - "VariableReferenceExpression", - "ordinalityVariable"); + "schemaTableName", + p.schemaTableName, + "RefreshMaterializedViewHandle", + "SchemaTableName", + "schemaTableName"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MemoryAllocation& p) { +void to_json(json& j, const HiveBucketHandle& p) { j = json::object(); - to_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); to_json_key( j, - "allocation", - p.allocation, - "MemoryAllocation", - "int64_t", - "allocation"); + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); + to_json_key( + j, + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", + "int", + "tableBucketCount"); + to_json_key( + j, + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); } -void from_json(const json& j, MemoryAllocation& p) { - from_json_key(j, "tag", p.tag, "MemoryAllocation", "String", "tag"); +void from_json(const json& j, HiveBucketHandle& p) { from_json_key( j, - "allocation", - p.allocation, - "MemoryAllocation", - "int64_t", - "allocation"); + "columns", + p.columns, + "HiveBucketHandle", + "List", + "columns"); + from_json_key( + j, + "tableBucketCount", + p.tableBucketCount, + "HiveBucketHandle", + "int", + "tableBucketCount"); + from_json_key( + j, + "readBucketCount", + p.readBucketCount, + "HiveBucketHandle", + "int", + "readBucketCount"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +HiveTableLayoutHandle::HiveTableLayoutHandle() noexcept { + _type = "hive"; +} -void to_json(json& j, const MemoryPoolInfo& p) { +void to_json(json& j, const HiveTableLayoutHandle& p) { j = json::object(); + j["@type"] = "hive"; to_json_key( - j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); + j, + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); to_json_key( j, - "reservedBytes", - p.reservedBytes, - "MemoryPoolInfo", - "int64_t", - "reservedBytes"); + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); to_json_key( j, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); to_json_key( j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); to_json_key( j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); to_json_key( j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); -} - -void from_json(const json& j, MemoryPoolInfo& p) { - from_json_key( - j, "maxBytes", p.maxBytes, "MemoryPoolInfo", "int64_t", "maxBytes"); - from_json_key( + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + to_json_key( j, - "reservedBytes", - p.reservedBytes, - "MemoryPoolInfo", - "int64_t", - "reservedBytes"); - from_json_key( + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + to_json_key( j, - "reservedRevocableBytes", - p.reservedRevocableBytes, - "MemoryPoolInfo", - "int64_t", - "reservedRevocableBytes"); - from_json_key( + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + to_json_key( j, - "queryMemoryReservations", - p.queryMemoryReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryReservations"); - from_json_key( + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + to_json_key( j, - "queryMemoryAllocations", - p.queryMemoryAllocations, - "MemoryPoolInfo", - "Map>", - "queryMemoryAllocations"); - from_json_key( + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); + to_json_key( j, - "queryMemoryRevocableReservations", - p.queryMemoryRevocableReservations, - "MemoryPoolInfo", - "Map", - "queryMemoryRevocableReservations"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -RowNumberNode::RowNumberNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.RowNumberNode"; -} - -void to_json(json& j, const RowNumberNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.RowNumberNode"; - to_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); to_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", - "List", - "partitionBy"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); to_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); to_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); to_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); + to_json_key( + j, + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); } -void from_json(const json& j, RowNumberNode& p) { +void from_json(const json& j, HiveTableLayoutHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); from_json_key( j, - "partitionBy", - p.partitionBy, - "RowNumberNode", - "List", - "partitionBy"); + "schemaTableName", + p.schemaTableName, + "HiveTableLayoutHandle", + "SchemaTableName", + "schemaTableName"); from_json_key( j, - "rowNumberVariable", - p.rowNumberVariable, - "RowNumberNode", - "VariableReferenceExpression", - "rowNumberVariable"); + "tablePath", + p.tablePath, + "HiveTableLayoutHandle", + "String", + "tablePath"); from_json_key( j, - "maxRowCountPerPartition", - p.maxRowCountPerPartition, - "RowNumberNode", - "Integer", - "maxRowCountPerPartition"); + "partitionColumns", + p.partitionColumns, + "HiveTableLayoutHandle", + "List", + "partitionColumns"); from_json_key( j, - "hashVariable", - p.hashVariable, - "RowNumberNode", - "VariableReferenceExpression", - "hashVariable"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -WindowNode::WindowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.WindowNode"; -} - -void to_json(json& j, const WindowNode& p) { - j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.WindowNode"; - to_json_key( + "dataColumns", + p.dataColumns, + "HiveTableLayoutHandle", + "List", + "dataColumns"); + from_json_key( j, - "sourceLocation", - p.sourceLocation, - "WindowNode", - "SourceLocation", - "sourceLocation"); - to_json_key(j, "id", p.id, "WindowNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "WindowNode", "PlanNode", "source"); - to_json_key( + "tableParameters", + p.tableParameters, + "HiveTableLayoutHandle", + "Map", + "tableParameters"); + from_json_key( j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); - to_json_key( + "domainPredicate", + p.domainPredicate, + "HiveTableLayoutHandle", + "TupleDomain", + "domainPredicate"); + from_json_key( j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); - to_json_key( + "remainingPredicate", + p.remainingPredicate, + "HiveTableLayoutHandle", + "RowExpression", + "remainingPredicate"); + from_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", - "VariableReferenceExpression", - "hashVariable"); - to_json_key( + "predicateColumns", + p.predicateColumns, + "HiveTableLayoutHandle", + "Map", + "predicateColumns"); + from_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); - to_json_key( + "partitionColumnPredicate", + p.partitionColumnPredicate, + "HiveTableLayoutHandle", + "TupleDomain>", + "partitionColumnPredicate"); + from_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); -} - -void from_json(const json& j, WindowNode& p) { - p._type = j["@type"]; + "bucketHandle", + p.bucketHandle, + "HiveTableLayoutHandle", + "HiveBucketHandle", + "bucketHandle"); from_json_key( j, - "sourceLocation", - p.sourceLocation, - "WindowNode", - "SourceLocation", - "sourceLocation"); - from_json_key(j, "id", p.id, "WindowNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "WindowNode", "PlanNode", "source"); + "bucketFilter", + p.bucketFilter, + "HiveTableLayoutHandle", + "HiveBucketFilter", + "bucketFilter"); from_json_key( j, - "specification", - p.specification, - "WindowNode", - "Specification", - "specification"); + "pushdownFilterEnabled", + p.pushdownFilterEnabled, + "HiveTableLayoutHandle", + "bool", + "pushdownFilterEnabled"); from_json_key( j, - "windowFunctions", - p.windowFunctions, - "WindowNode", - "Map", - "windowFunctions"); + "layoutString", + p.layoutString, + "HiveTableLayoutHandle", + "String", + "layoutString"); from_json_key( j, - "hashVariable", - p.hashVariable, - "WindowNode", - "VariableReferenceExpression", - "hashVariable"); + "requestedColumns", + p.requestedColumns, + "HiveTableLayoutHandle", + "List", + "requestedColumns"); from_json_key( j, - "prePartitionedInputs", - p.prePartitionedInputs, - "WindowNode", - "List", - "prePartitionedInputs"); + "partialAggregationsPushedDown", + p.partialAggregationsPushedDown, + "HiveTableLayoutHandle", + "bool", + "partialAggregationsPushedDown"); from_json_key( j, - "preSortedOrderPrefix", - p.preSortedOrderPrefix, - "WindowNode", - "int", - "preSortedOrderPrefix"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const HivePartitionKey& p) { - j = json::object(); - to_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); - to_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); -} - -void from_json(const json& j, HivePartitionKey& p) { - from_json_key(j, "name", p.name, "HivePartitionKey", "String", "name"); - from_json_key(j, "value", p.value, "HivePartitionKey", "String", "value"); + "appendRowNumber", + p.appendRowNumber, + "HiveTableLayoutHandle", + "bool", + "appendRowNumber"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair BufferState_enum_table[] = +static const std::pair BufferType_enum_table[] = { // NOLINT: cert-err58-cpp - {BufferState::OPEN, "OPEN"}, - {BufferState::NO_MORE_BUFFERS, "NO_MORE_BUFFERS"}, - {BufferState::NO_MORE_PAGES, "NO_MORE_PAGES"}, - {BufferState::FLUSHING, "FLUSHING"}, - {BufferState::FINISHED, "FINISHED"}, - {BufferState::FAILED, "FAILED"}}; -void to_json(json& j, const BufferState& e) { - static_assert( - std::is_enum::value, "BufferState must be an enum!"); + {BufferType::PARTITIONED, "PARTITIONED"}, + {BufferType::BROADCAST, "BROADCAST"}, + {BufferType::ARBITRARY, "ARBITRARY"}, + {BufferType::DISCARDING, "DISCARDING"}, + {BufferType::SPOOLING, "SPOOLING"}}; +void to_json(json& j, const BufferType& e) { + static_assert(std::is_enum::value, "BufferType must be an enum!"); const auto* it = std::find_if( - std::begin(BufferState_enum_table), - std::end(BufferState_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(BufferType_enum_table), + std::end(BufferType_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(BufferState_enum_table)) + j = ((it != std::end(BufferType_enum_table)) ? it - : std::begin(BufferState_enum_table)) + : std::begin(BufferType_enum_table)) ->second; } -void from_json(const json& j, BufferState& e) { - static_assert( - std::is_enum::value, "BufferState must be an enum!"); +void from_json(const json& j, BufferType& e) { + static_assert(std::is_enum::value, "BufferType must be an enum!"); const auto* it = std::find_if( - std::begin(BufferState_enum_table), - std::end(BufferState_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(BufferType_enum_table), + std::end(BufferType_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(BufferState_enum_table)) + e = ((it != std::end(BufferType_enum_table)) ? it - : std::begin(BufferState_enum_table)) + : std::begin(BufferType_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const OutputBufferInfo& p) { +void to_json(json& j, const OutputBuffers& p) { j = json::object(); - to_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); - to_json_key(j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); - to_json_key( - j, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); + to_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); + to_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); to_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", "bool", - "canAddPages"); - to_json_key( - j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); - to_json_key( - j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); - to_json_key( - j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); - to_json_key( - j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); + "noMoreBufferIds"); to_json_key( j, "buffers", p.buffers, - "OutputBufferInfo", - "List", + "OutputBuffers", + "Map", "buffers"); } -void from_json(const json& j, OutputBufferInfo& p) { - from_json_key(j, "type", p.type, "OutputBufferInfo", "String", "type"); - from_json_key( - j, "state", p.state, "OutputBufferInfo", "BufferState", "state"); - from_json_key( - j, - "canAddBuffers", - p.canAddBuffers, - "OutputBufferInfo", - "bool", - "canAddBuffers"); +void from_json(const json& j, OutputBuffers& p) { + from_json_key(j, "type", p.type, "OutputBuffers", "BufferType", "type"); + from_json_key(j, "version", p.version, "OutputBuffers", "int64_t", "version"); from_json_key( j, - "canAddPages", - p.canAddPages, - "OutputBufferInfo", + "noMoreBufferIds", + p.noMoreBufferIds, + "OutputBuffers", "bool", - "canAddPages"); - from_json_key( - j, - "totalBufferedBytes", - p.totalBufferedBytes, - "OutputBufferInfo", - "int64_t", - "totalBufferedBytes"); - from_json_key( - j, - "totalBufferedPages", - p.totalBufferedPages, - "OutputBufferInfo", - "int64_t", - "totalBufferedPages"); - from_json_key( - j, - "totalRowsSent", - p.totalRowsSent, - "OutputBufferInfo", - "int64_t", - "totalRowsSent"); - from_json_key( - j, - "totalPagesSent", - p.totalPagesSent, - "OutputBufferInfo", - "int64_t", - "totalPagesSent"); + "noMoreBufferIds"); from_json_key( j, "buffers", p.buffers, - "OutputBufferInfo", - "List", + "OutputBuffers", + "Map", "buffers"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -ConstantExpression::ConstantExpression() noexcept { - _type = "constant"; -} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const ConstantExpression& p) { - j = json::object(); - j["@type"] = "constant"; - to_json_key( - j, - "valueBlock", - p.valueBlock, - "ConstantExpression", - "Block", - "valueBlock"); - to_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair RuntimeUnit_enum_table[] = + { // NOLINT: cert-err58-cpp + {RuntimeUnit::NONE, "NONE"}, + {RuntimeUnit::NANO, "NANO"}, + {RuntimeUnit::BYTE, "BYTE"}}; +void to_json(json& j, const RuntimeUnit& e) { + static_assert( + std::is_enum::value, "RuntimeUnit must be an enum!"); + const auto* it = std::find_if( + std::begin(RuntimeUnit_enum_table), + std::end(RuntimeUnit_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(RuntimeUnit_enum_table)) + ? it + : std::begin(RuntimeUnit_enum_table)) + ->second; +} +void from_json(const json& j, RuntimeUnit& e) { + static_assert( + std::is_enum::value, "RuntimeUnit must be an enum!"); + const auto* it = std::find_if( + std::begin(RuntimeUnit_enum_table), + std::end(RuntimeUnit_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(RuntimeUnit_enum_table)) + ? it + : std::begin(RuntimeUnit_enum_table)) + ->first; } +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { -void from_json(const json& j, ConstantExpression& p) { - p._type = j["@type"]; - from_json_key( - j, - "valueBlock", - p.valueBlock, - "ConstantExpression", - "Block", - "valueBlock"); - from_json_key(j, "type", p.type, "ConstantExpression", "Type", "type"); +void to_json(json& j, const RuntimeMetric& p) { + j = json::object(); + to_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); + to_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); + to_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); + to_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); + to_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); + to_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); +} + +void from_json(const json& j, RuntimeMetric& p) { + from_json_key(j, "name", p.name, "RuntimeMetric", "String", "name"); + from_json_key(j, "unit", p.unit, "RuntimeMetric", "RuntimeUnit", "unit"); + from_json_key(j, "sum", p.sum, "RuntimeMetric", "int64_t", "sum"); + from_json_key(j, "count", p.count, "RuntimeMetric", "int64_t", "count"); + from_json_key(j, "max", p.max, "RuntimeMetric", "int64_t", "max"); + from_json_key(j, "min", p.min, "RuntimeMetric", "int64_t", "min"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -InsertHandle::InsertHandle() noexcept { - _type = "InsertHandle"; +UnnestNode::UnnestNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.UnnestNode"; } -void to_json(json& j, const InsertHandle& p) { +void to_json(json& j, const UnnestNode& p) { j = json::object(); - j["@type"] = "InsertHandle"; + j["@type"] = "com.facebook.presto.sql.planner.plan.UnnestNode"; + to_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); to_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + j, + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); to_json_key( j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); + to_json_key( + j, + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); } -void from_json(const json& j, InsertHandle& p) { +void from_json(const json& j, UnnestNode& p) { p._type = j["@type"]; + from_json_key(j, "id", p.id, "UnnestNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "UnnestNode", "PlanNode", "source"); from_json_key( - j, "handle", p.handle, "InsertHandle", "InsertTableHandle", "handle"); + j, + "replicateVariables", + p.replicateVariables, + "UnnestNode", + "List", + "replicateVariables"); from_json_key( j, - "schemaTableName", - p.schemaTableName, - "InsertHandle", - "SchemaTableName", - "schemaTableName"); + "unnestVariables", + p.unnestVariables, + "UnnestNode", + "Map>", + "unnestVariables"); + from_json_key( + j, + "ordinalityVariable", + p.ordinalityVariable, + "UnnestNode", + "VariableReferenceExpression", + "ordinalityVariable"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const MemoryInfo& p) { +void to_json(json& j, const ResourceEstimates& p) { j = json::object(); to_json_key( j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); + to_json_key( + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + to_json_key( + j, + "peakMemory", + p.peakMemory, + "ResourceEstimates", "DataSize", - "totalNodeMemory"); + "peakMemory"); to_json_key( j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); } -void from_json(const json& j, MemoryInfo& p) { +void from_json(const json& j, ResourceEstimates& p) { from_json_key( j, - "totalNodeMemory", - p.totalNodeMemory, - "MemoryInfo", + "executionTime", + p.executionTime, + "ResourceEstimates", + "Duration", + "executionTime"); + from_json_key( + j, "cpuTime", p.cpuTime, "ResourceEstimates", "Duration", "cpuTime"); + from_json_key( + j, + "peakMemory", + p.peakMemory, + "ResourceEstimates", "DataSize", - "totalNodeMemory"); + "peakMemory"); from_json_key( j, - "pools", - p.pools, - "MemoryInfo", - "Map", - "pools"); + "peakTaskMemory", + p.peakTaskMemory, + "ResourceEstimates", + "DataSize", + "peakTaskMemory"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -void to_json(json& j, const NodeStatus& p) { +void to_json(json& j, const SqlInvokedFunction& p) { j = json::object(); - to_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); - to_json_key( - j, - "nodeVersion", - p.nodeVersion, - "NodeStatus", - "NodeVersion", - "nodeVersion"); - to_json_key( - j, "environment", p.environment, "NodeStatus", "String", "environment"); - to_json_key( - j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); - to_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); to_json_key( j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); to_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", + "description", + p.description, + "SqlInvokedFunction", "String", - "internalAddress"); - to_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - to_json_key(j, "processors", p.processors, "NodeStatus", "int", "processors"); + "description"); to_json_key( j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "routineCharacteristics", + p.routineCharacteristics, + "SqlInvokedFunction", + "RoutineCharacteristics", + "routineCharacteristics"); + to_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); to_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - to_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "signature", + p.signature, + "SqlInvokedFunction", + "Signature", + "signature"); to_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); - to_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + "functionId", + p.functionId, + "SqlInvokedFunction", + "SqlFunctionId", + "functionId"); } -void from_json(const json& j, NodeStatus& p) { - from_json_key(j, "nodeId", p.nodeId, "NodeStatus", "String", "nodeId"); - from_json_key( - j, - "nodeVersion", - p.nodeVersion, - "NodeStatus", - "NodeVersion", - "nodeVersion"); - from_json_key( - j, "environment", p.environment, "NodeStatus", "String", "environment"); - from_json_key( - j, "coordinator", p.coordinator, "NodeStatus", "bool", "coordinator"); - from_json_key(j, "uptime", p.uptime, "NodeStatus", "Duration", "uptime"); +void from_json(const json& j, SqlInvokedFunction& p) { from_json_key( j, - "externalAddress", - p.externalAddress, - "NodeStatus", - "String", - "externalAddress"); + "parameters", + p.parameters, + "SqlInvokedFunction", + "List", + "parameters"); from_json_key( j, - "internalAddress", - p.internalAddress, - "NodeStatus", + "description", + p.description, + "SqlInvokedFunction", "String", - "internalAddress"); - from_json_key( - j, "memoryInfo", p.memoryInfo, "NodeStatus", "MemoryInfo", "memoryInfo"); - from_json_key( - j, "processors", p.processors, "NodeStatus", "int", "processors"); - from_json_key( - j, - "processCpuLoad", - p.processCpuLoad, - "NodeStatus", - "double", - "processCpuLoad"); + "description"); from_json_key( j, - "systemCpuLoad", - p.systemCpuLoad, - "NodeStatus", - "double", - "systemCpuLoad"); - from_json_key(j, "heapUsed", p.heapUsed, "NodeStatus", "int64_t", "heapUsed"); + "routineCharacteristics", + p.routineCharacteristics, + "SqlInvokedFunction", + "RoutineCharacteristics", + "routineCharacteristics"); + from_json_key(j, "body", p.body, "SqlInvokedFunction", "String", "body"); from_json_key( j, - "heapAvailable", - p.heapAvailable, - "NodeStatus", - "int64_t", - "heapAvailable"); + "signature", + p.signature, + "SqlInvokedFunction", + "Signature", + "signature"); from_json_key( - j, "nonHeapUsed", p.nonHeapUsed, "NodeStatus", "int64_t", "nonHeapUsed"); + j, + "functionId", + p.functionId, + "SqlInvokedFunction", + "SqlFunctionId", + "functionId"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveTableHandle::HiveTableHandle() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveTableHandle& p) { +void to_json(json& j, const SessionRepresentation& p) { j = json::object(); - j["@type"] = "hive"; to_json_key( - j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); to_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); + j, + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); to_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); -} - -void from_json(const json& j, HiveTableHandle& p) { - p._type = j["@type"]; - from_json_key( - j, "schemaName", p.schemaName, "HiveTableHandle", "String", "schemaName"); - from_json_key( - j, "tableName", p.tableName, "HiveTableHandle", "String", "tableName"); - from_json_key( + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + to_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); + to_json_key( j, - "analyzePartitionValues", - p.analyzePartitionValues, - "HiveTableHandle", - "List>", - "analyzePartitionValues"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const TaskInfo& p) { - j = json::object(); - to_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); + "principal", + p.principal, + "SessionRepresentation", + "String", + "principal"); to_json_key( - j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); + j, "source", p.source, "SessionRepresentation", "String", "source"); to_json_key( - j, - "lastHeartbeat", - p.lastHeartbeat, - "TaskInfo", - "DateTime", - "lastHeartbeat"); + j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); + to_json_key( + j, "schema", p.schema, "SessionRepresentation", "String", "schema"); to_json_key( j, - "outputBuffers", - p.outputBuffers, - "TaskInfo", - "OutputBufferInfo", - "outputBuffers"); + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); to_json_key( j, - "noMoreSplits", - p.noMoreSplits, - "TaskInfo", - "List", - "noMoreSplits"); - to_json_key(j, "stats", p.stats, "TaskInfo", "TaskStats", "stats"); - to_json_key(j, "needsPlan", p.needsPlan, "TaskInfo", "bool", "needsPlan"); + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + to_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); to_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "TaskInfo", - "MetadataUpdates", - "metadataUpdates"); - to_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); -} - -void from_json(const json& j, TaskInfo& p) { - from_json_key(j, "taskId", p.taskId, "TaskInfo", "TaskId", "taskId"); - from_json_key( - j, "taskStatus", p.taskStatus, "TaskInfo", "TaskStatus", "taskStatus"); - from_json_key( + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); + to_json_key( j, - "lastHeartbeat", - p.lastHeartbeat, - "TaskInfo", - "DateTime", - "lastHeartbeat"); - from_json_key( + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); + to_json_key( j, - "outputBuffers", - p.outputBuffers, - "TaskInfo", - "OutputBufferInfo", - "outputBuffers"); - from_json_key( + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); + to_json_key( j, - "noMoreSplits", - p.noMoreSplits, - "TaskInfo", - "List", - "noMoreSplits"); - from_json_key(j, "stats", p.stats, "TaskInfo", "TaskStats", "stats"); - from_json_key(j, "needsPlan", p.needsPlan, "TaskInfo", "bool", "needsPlan"); - from_json_key( + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); + to_json_key( j, - "metadataUpdates", - p.metadataUpdates, - "TaskInfo", - "MetadataUpdates", - "metadataUpdates"); - from_json_key(j, "nodeId", p.nodeId, "TaskInfo", "String", "nodeId"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Range& p) { - j = json::object(); - to_json_key(j, "low", p.low, "Range", "Marker", "low"); - to_json_key(j, "high", p.high, "Range", "Marker", "high"); -} - -void from_json(const json& j, Range& p) { - from_json_key(j, "low", p.low, "Range", "Marker", "low"); - from_json_key(j, "high", p.high, "Range", "Marker", "high"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -SortedRangeSet::SortedRangeSet() noexcept { - _type = "sortable"; -} - -void to_json(json& j, const SortedRangeSet& p) { - j = json::object(); - j["@type"] = "sortable"; - to_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); - to_json_key(j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); -} - -void from_json(const json& j, SortedRangeSet& p) { - p._type = j["@type"]; - from_json_key(j, "type", p.type, "SortedRangeSet", "Type", "type"); - from_json_key( - j, "ranges", p.ranges, "SortedRangeSet", "List", "ranges"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() - -// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - AggregationNodeStep_enum_table[] = { // NOLINT: cert-err58-cpp - {AggregationNodeStep::PARTIAL, "PARTIAL"}, - {AggregationNodeStep::FINAL, "FINAL"}, - {AggregationNodeStep::INTERMEDIATE, "INTERMEDIATE"}, - {AggregationNodeStep::SINGLE, "SINGLE"}}; -void to_json(json& j, const AggregationNodeStep& e) { - static_assert( - std::is_enum::value, - "AggregationNodeStep must be an enum!"); - const auto* it = std::find_if( - std::begin(AggregationNodeStep_enum_table), - std::end(AggregationNodeStep_enum_table), - [e](const std::pair& ej_pair) -> bool { - return ej_pair.first == e; - }); - j = ((it != std::end(AggregationNodeStep_enum_table)) - ? it - : std::begin(AggregationNodeStep_enum_table)) - ->second; -} -void from_json(const json& j, AggregationNodeStep& e) { - static_assert( - std::is_enum::value, - "AggregationNodeStep must be an enum!"); - const auto* it = std::find_if( - std::begin(AggregationNodeStep_enum_table), - std::end(AggregationNodeStep_enum_table), - [&j](const std::pair& ej_pair) -> bool { - return ej_pair.second == j; - }); - e = ((it != std::end(AggregationNodeStep_enum_table)) - ? it - : std::begin(AggregationNodeStep_enum_table)) - ->first; -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const GroupingSetDescriptor& p) { - j = json::object(); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); to_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); to_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); to_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); + to_json_key( + j, + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); + to_json_key( + j, + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); + to_json_key( + j, + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); + to_json_key( + j, + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } -void from_json(const json& j, GroupingSetDescriptor& p) { +void from_json(const json& j, SessionRepresentation& p) { + from_json_key( + j, "queryId", p.queryId, "SessionRepresentation", "String", "queryId"); from_json_key( j, - "groupingKeys", - p.groupingKeys, - "GroupingSetDescriptor", - "List", - "groupingKeys"); + "transactionId", + p.transactionId, + "SessionRepresentation", + "TransactionId", + "transactionId"); from_json_key( j, - "groupingSetCount", - p.groupingSetCount, - "GroupingSetDescriptor", - "int", - "groupingSetCount"); + "clientTransactionSupport", + p.clientTransactionSupport, + "SessionRepresentation", + "bool", + "clientTransactionSupport"); + from_json_key(j, "user", p.user, "SessionRepresentation", "String", "user"); from_json_key( j, - "globalGroupingSets", - p.globalGroupingSets, - "GroupingSetDescriptor", - "List", - "globalGroupingSets"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -AggregationNode::AggregationNode() noexcept { - _type = ".AggregationNode"; -} - -void to_json(json& j, const AggregationNode& p) { - j = json::object(); - j["@type"] = ".AggregationNode"; - to_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); - to_json_key( + "principal", + p.principal, + "SessionRepresentation", + "String", + "principal"); + from_json_key( + j, "source", p.source, "SessionRepresentation", "String", "source"); + from_json_key( + j, "catalog", p.catalog, "SessionRepresentation", "String", "catalog"); + from_json_key( + j, "schema", p.schema, "SessionRepresentation", "String", "schema"); + from_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); - to_json_key( + "traceToken", + p.traceToken, + "SessionRepresentation", + "String", + "traceToken"); + from_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); - to_json_key( + "timeZoneKey", + p.timeZoneKey, + "SessionRepresentation", + "TimeZoneKey", + "timeZoneKey"); + from_json_key( + j, "locale", p.locale, "SessionRepresentation", "Locale", "locale"); + from_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); - to_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); - to_json_key( + "remoteUserAddress", + p.remoteUserAddress, + "SessionRepresentation", + "String", + "remoteUserAddress"); + from_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); - to_json_key( + "userAgent", + p.userAgent, + "SessionRepresentation", + "String", + "userAgent"); + from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); -} - -void from_json(const json& j, AggregationNode& p) { - p._type = j["@type"]; - from_json_key(j, "id", p.id, "AggregationNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "AggregationNode", "PlanNode", "source"); + "clientInfo", + p.clientInfo, + "SessionRepresentation", + "String", + "clientInfo"); from_json_key( j, - "aggregations", - p.aggregations, - "AggregationNode", - "Map", - "aggregations"); + "clientTags", + p.clientTags, + "SessionRepresentation", + "List", + "clientTags"); from_json_key( j, - "groupingSets", - p.groupingSets, - "AggregationNode", - "GroupingSetDescriptor", - "groupingSets"); + "resourceEstimates", + p.resourceEstimates, + "SessionRepresentation", + "ResourceEstimates", + "resourceEstimates"); from_json_key( j, - "preGroupedVariables", - p.preGroupedVariables, - "AggregationNode", - "List", - "preGroupedVariables"); + "startTime", + p.startTime, + "SessionRepresentation", + "int64_t", + "startTime"); from_json_key( - j, "step", p.step, "AggregationNode", "AggregationNodeStep", "step"); + j, + "systemProperties", + p.systemProperties, + "SessionRepresentation", + "Map", + "systemProperties"); from_json_key( j, - "hashVariable", - p.hashVariable, - "AggregationNode", - "VariableReferenceExpression", - "hashVariable"); + "catalogProperties", + p.catalogProperties, + "SessionRepresentation", + "Map>", + "catalogProperties"); from_json_key( j, - "groupIdVariable", - p.groupIdVariable, - "AggregationNode", - "VariableReferenceExpression", - "groupIdVariable"); + "unprocessedCatalogProperties", + p.unprocessedCatalogProperties, + "SessionRepresentation", + "Map>", + "unprocessedCatalogProperties"); + from_json_key( + j, + "roles", + p.roles, + "SessionRepresentation", + "Map", + "roles"); + from_json_key( + j, + "preparedStatements", + p.preparedStatements, + "SessionRepresentation", + "Map", + "preparedStatements"); + from_json_key( + j, + "sessionFunctions", + p.sessionFunctions, + "SessionRepresentation", + "Map", + "sessionFunctions"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HivePartitioningHandle::HivePartitioningHandle() noexcept { - _type = "hive"; +LambdaDefinitionExpression::LambdaDefinitionExpression() noexcept { + _type = "lambda"; } -void to_json(json& j, const HivePartitioningHandle& p) { +void to_json(json& j, const LambdaDefinitionExpression& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); + j["@type"] = "lambda"; to_json_key( j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); to_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); to_json_key( j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); to_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } -void from_json(const json& j, HivePartitioningHandle& p) { +void from_json(const json& j, LambdaDefinitionExpression& p) { p._type = j["@type"]; from_json_key( j, - "bucketCount", - p.bucketCount, - "HivePartitioningHandle", - "int", - "bucketCount"); - from_json_key( - j, - "maxCompatibleBucketCount", - p.maxCompatibleBucketCount, - "HivePartitioningHandle", - "int", - "maxCompatibleBucketCount"); + "sourceLocation", + p.sourceLocation, + "LambdaDefinitionExpression", + "SourceLocation", + "sourceLocation"); from_json_key( j, - "bucketFunctionType", - p.bucketFunctionType, - "HivePartitioningHandle", - "BucketFunctionType", - "bucketFunctionType"); + "argumentTypes", + p.argumentTypes, + "LambdaDefinitionExpression", + "List", + "argumentTypes"); from_json_key( j, - "hiveTypes", - p.hiveTypes, - "HivePartitioningHandle", - "List", - "hiveTypes"); + "arguments", + p.arguments, + "LambdaDefinitionExpression", + "List", + "arguments"); from_json_key( - j, "types", p.types, "HivePartitioningHandle", "List", "types"); + j, "body", p.body, "LambdaDefinitionExpression", "RowExpression", "body"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SortNode::SortNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SortNode"; +ValuesNode::ValuesNode() noexcept { + _type = ".ValuesNode"; } -void to_json(json& j, const SortNode& p) { +void to_json(json& j, const ValuesNode& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.SortNode"; - to_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + j["@type"] = ".ValuesNode"; + to_json_key( + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + to_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); to_json_key( j, - "orderingScheme", - p.orderingScheme, - "SortNode", - "OrderingScheme", - "orderingScheme"); - to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); + to_json_key( + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); + to_json_key( + j, + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } -void from_json(const json& j, SortNode& p) { +void from_json(const json& j, ValuesNode& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + from_json_key( + j, "location", p.location, "ValuesNode", "SourceLocation", "location"); + from_json_key(j, "id", p.id, "ValuesNode", "PlanNodeId", "id"); from_json_key( j, - "orderingScheme", - p.orderingScheme, - "SortNode", - "OrderingScheme", - "orderingScheme"); - from_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); + "outputVariables", + p.outputVariables, + "ValuesNode", + "List", + "outputVariables"); + from_json_key( + j, + "rows", + p.rows, + "ValuesNode", + "List>>", + "rows"); + from_json_key( + j, + "valuesNodeLabel", + p.valuesNodeLabel, + "ValuesNode", + "String", + "valuesNodeLabel"); } } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -// dependency KeyedSubclass +namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + SystemPartitionFunction_enum_table[] = { // NOLINT: cert-err58-cpp + {SystemPartitionFunction::SINGLE, "SINGLE"}, + {SystemPartitionFunction::HASH, "HASH"}, + {SystemPartitionFunction::ROUND_ROBIN, "ROUND_ROBIN"}, + {SystemPartitionFunction::BROADCAST, "BROADCAST"}, + {SystemPartitionFunction::UNKNOWN, "UNKNOWN"}}; +void to_json(json& j, const SystemPartitionFunction& e) { + static_assert( + std::is_enum::value, + "SystemPartitionFunction must be an enum!"); + const auto* it = std::find_if( + std::begin(SystemPartitionFunction_enum_table), + std::end(SystemPartitionFunction_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(SystemPartitionFunction_enum_table)) + ? it + : std::begin(SystemPartitionFunction_enum_table)) + ->second; +} +void from_json(const json& j, SystemPartitionFunction& e) { + static_assert( + std::is_enum::value, + "SystemPartitionFunction must be an enum!"); + const auto* it = std::find_if( + std::begin(SystemPartitionFunction_enum_table), + std::end(SystemPartitionFunction_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(SystemPartitionFunction_enum_table)) + ? it + : std::begin(SystemPartitionFunction_enum_table)) + ->first; +} +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -std::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) { - return j["@type"]; +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair + SystemPartitioning_enum_table[] = { // NOLINT: cert-err58-cpp + {SystemPartitioning::SINGLE, "SINGLE"}, + {SystemPartitioning::FIXED, "FIXED"}, + {SystemPartitioning::SOURCE, "SOURCE"}, + {SystemPartitioning::SCALED, "SCALED"}, + {SystemPartitioning::COORDINATOR_ONLY, "COORDINATOR_ONLY"}, + {SystemPartitioning::ARBITRARY, "ARBITRARY"}}; +void to_json(json& j, const SystemPartitioning& e) { + static_assert( + std::is_enum::value, + "SystemPartitioning must be an enum!"); + const auto* it = std::find_if( + std::begin(SystemPartitioning_enum_table), + std::end(SystemPartitioning_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(SystemPartitioning_enum_table)) + ? it + : std::begin(SystemPartitioning_enum_table)) + ->second; +} +void from_json(const json& j, SystemPartitioning& e) { + static_assert( + std::is_enum::value, + "SystemPartitioning must be an enum!"); + const auto* it = std::find_if( + std::begin(SystemPartitioning_enum_table), + std::end(SystemPartitioning_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(SystemPartitioning_enum_table)) + ? it + : std::begin(SystemPartitioning_enum_table)) + ->first; } - } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -EnforceSingleRowNode::EnforceSingleRowNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; +SystemPartitioningHandle::SystemPartitioningHandle() noexcept { + _type = "$remote"; } -void to_json(json& j, const EnforceSingleRowNode& p) { +void to_json(json& j, const SystemPartitioningHandle& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode"; - to_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); + j["@type"] = "$remote"; to_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); + j, + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); + to_json_key( + j, + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); } -void from_json(const json& j, EnforceSingleRowNode& p) { +void from_json(const json& j, SystemPartitioningHandle& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "EnforceSingleRowNode", "PlanNodeId", "id"); from_json_key( - j, "source", p.source, "EnforceSingleRowNode", "PlanNode", "source"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { - -void to_json(json& j, const Location& p) { - j = json::object(); - to_json_key(j, "location", p.location, "Location", "String", "location"); -} - -void from_json(const json& j, Location& p) { - from_json_key(j, "location", p.location, "Location", "String", "location"); + j, + "partitioning", + p.partitioning, + "SystemPartitioningHandle", + "SystemPartitioning", + "partitioning"); + from_json_key( + j, + "function", + p.function, + "SystemPartitioningHandle", + "SystemPartitionFunction", + "function"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -RemoteSplit::RemoteSplit() noexcept { - _type = "$remote"; +TpchPartitioningHandle::TpchPartitioningHandle() noexcept { + _type = "tpch"; } -void to_json(json& j, const RemoteSplit& p) { +void to_json(json& j, const TpchPartitioningHandle& p) { j = json::object(); - j["@type"] = "$remote"; - to_json_key(j, "location", p.location, "RemoteSplit", "Location", "location"); + j["@type"] = "tpch"; + to_json_key(j, "table", p.table, "TpchPartitioningHandle", "String", "table"); to_json_key( j, - "remoteSourceTaskId", - p.remoteSourceTaskId, - "RemoteSplit", - "TaskId", - "remoteSourceTaskId"); + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } -void from_json(const json& j, RemoteSplit& p) { +void from_json(const json& j, TpchPartitioningHandle& p) { p._type = j["@type"]; from_json_key( - j, "location", p.location, "RemoteSplit", "Location", "location"); + j, "table", p.table, "TpchPartitioningHandle", "String", "table"); from_json_key( j, - "remoteSourceTaskId", - p.remoteSourceTaskId, - "RemoteSplit", - "TaskId", - "remoteSourceTaskId"); + "totalRows", + p.totalRows, + "TpchPartitioningHandle", + "int64_t", + "totalRows"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -SemiJoinNode::SemiJoinNode() noexcept { - _type = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; +TpchSplit::TpchSplit() noexcept { + _type = "tpch"; } -void to_json(json& j, const SemiJoinNode& p) { +void to_json(json& j, const TpchSplit& p) { j = json::object(); - j["@type"] = "com.facebook.presto.sql.planner.plan.SemiJoinNode"; - to_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); - to_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); - to_json_key( - j, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); - to_json_key( - j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); - to_json_key( - j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); - to_json_key( - j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); - to_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + j["@type"] = "tpch"; to_json_key( j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); + to_json_key(j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); + to_json_key(j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); to_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); to_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); } -void from_json(const json& j, SemiJoinNode& p) { +void from_json(const json& j, TpchSplit& p) { p._type = j["@type"]; - from_json_key(j, "id", p.id, "SemiJoinNode", "PlanNodeId", "id"); - from_json_key(j, "source", p.source, "SemiJoinNode", "PlanNode", "source"); - from_json_key( - j, - "filteringSource", - p.filteringSource, - "SemiJoinNode", - "PlanNode", - "filteringSource"); - from_json_key( - j, - "sourceJoinVariable", - p.sourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceJoinVariable"); - from_json_key( - j, - "filteringSourceJoinVariable", - p.filteringSourceJoinVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceJoinVariable"); from_json_key( j, - "semiJoinOutput", - p.semiJoinOutput, - "SemiJoinNode", - "VariableReferenceExpression", - "semiJoinOutput"); + "tableHandle", + p.tableHandle, + "TpchSplit", + "TpchTableHandle", + "tableHandle"); from_json_key( - j, - "sourceHashVariable", - p.sourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "sourceHashVariable"); + j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); from_json_key( - j, - "filteringSourceHashVariable", - p.filteringSourceHashVariable, - "SemiJoinNode", - "VariableReferenceExpression", - "filteringSourceHashVariable"); + j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); from_json_key( j, - "distributionType", - p.distributionType, - "SemiJoinNode", - "DistributionType", - "distributionType"); + "addresses", + p.addresses, + "TpchSplit", + "List", + "addresses"); from_json_key( j, - "dynamicFilters", - p.dynamicFilters, - "SemiJoinNode", - "Map", - "dynamicFilters"); + "predicate", + p.predicate, + "TpchSplit", + "TupleDomain>", + "predicate"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays -static const std::pair - NodeSelectionStrategy_enum_table[] = { // NOLINT: cert-err58-cpp - {NodeSelectionStrategy::HARD_AFFINITY, "HARD_AFFINITY"}, - {NodeSelectionStrategy::SOFT_AFFINITY, "SOFT_AFFINITY"}, - {NodeSelectionStrategy::NO_PREFERENCE, "NO_PREFERENCE"}}; -void to_json(json& j, const NodeSelectionStrategy& e) { +static const std::pair + StageExecutionStrategy_enum_table[] = { // NOLINT: cert-err58-cpp + {StageExecutionStrategy::UNGROUPED_EXECUTION, "UNGROUPED_EXECUTION"}, + {StageExecutionStrategy::FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + "FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION"}, + {StageExecutionStrategy::DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + "DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION"}, + {StageExecutionStrategy::RECOVERABLE_GROUPED_EXECUTION, + "RECOVERABLE_GROUPED_EXECUTION"}}; +void to_json(json& j, const StageExecutionStrategy& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, + "StageExecutionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(NodeSelectionStrategy_enum_table), - std::end(NodeSelectionStrategy_enum_table), - [e](const std::pair& ej_pair) -> bool { + std::begin(StageExecutionStrategy_enum_table), + std::end(StageExecutionStrategy_enum_table), + [e](const std::pair& ej_pair) -> bool { return ej_pair.first == e; }); - j = ((it != std::end(NodeSelectionStrategy_enum_table)) + j = ((it != std::end(StageExecutionStrategy_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(StageExecutionStrategy_enum_table)) ->second; } -void from_json(const json& j, NodeSelectionStrategy& e) { +void from_json(const json& j, StageExecutionStrategy& e) { static_assert( - std::is_enum::value, - "NodeSelectionStrategy must be an enum!"); + std::is_enum::value, + "StageExecutionStrategy must be an enum!"); const auto* it = std::find_if( - std::begin(NodeSelectionStrategy_enum_table), - std::end(NodeSelectionStrategy_enum_table), - [&j](const std::pair& ej_pair) -> bool { + std::begin(StageExecutionStrategy_enum_table), + std::end(StageExecutionStrategy_enum_table), + [&j](const std::pair& ej_pair) -> bool { return ej_pair.second == j; }); - e = ((it != std::end(NodeSelectionStrategy_enum_table)) + e = ((it != std::end(StageExecutionStrategy_enum_table)) ? it - : std::begin(NodeSelectionStrategy_enum_table)) + : std::begin(StageExecutionStrategy_enum_table)) ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -HiveSplit::HiveSplit() noexcept { - _type = "hive"; -} -void to_json(json& j, const HiveSplit& p) { +void to_json(json& j, const StageExecutionDescriptor& p) { j = json::object(); - j["@type"] = "hive"; - to_json_key( - j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); - to_json_key(j, "database", p.database, "HiveSplit", "String", "database"); - to_json_key(j, "table", p.table, "HiveSplit", "String", "table"); - to_json_key( - j, - "partitionName", - p.partitionName, - "HiveSplit", - "String", - "partitionName"); - to_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); to_json_key( j, - "partitionKeys", - p.partitionKeys, - "HiveSplit", - "List", - "partitionKeys"); + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); to_json_key( j, - "addresses", - p.addresses, - "HiveSplit", - "List", - "addresses"); + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); to_json_key( j, - "readBucketNumber", - p.readBucketNumber, - "HiveSplit", + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", "int", - "readBucketNumber"); - to_json_key( + "totalLifespans"); +} + +void from_json(const json& j, StageExecutionDescriptor& p) { + from_json_key( j, - "tableBucketNumber", - p.tableBucketNumber, - "HiveSplit", - "int", - "tableBucketNumber"); - to_json_key( + "stageExecutionStrategy", + p.stageExecutionStrategy, + "StageExecutionDescriptor", + "StageExecutionStrategy", + "stageExecutionStrategy"); + from_json_key( j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "HiveSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); - to_json_key( + "groupedExecutionScanNodes", + p.groupedExecutionScanNodes, + "StageExecutionDescriptor", + "List", + "groupedExecutionScanNodes"); + from_json_key( j, - "partitionDataColumnCount", - p.partitionDataColumnCount, - "HiveSplit", + "totalLifespans", + p.totalLifespans, + "StageExecutionDescriptor", "int", - "partitionDataColumnCount"); - to_json_key( - j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); + "totalLifespans"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { + +void to_json(json& j, const TaskUpdateRequest& p) { + j = json::object(); to_json_key( j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); to_json_key( j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); to_json_key( - j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); to_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); to_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); to_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } -void from_json(const json& j, HiveSplit& p) { - p._type = j["@type"]; - from_json_key( - j, "fileSplit", p.fileSplit, "HiveSplit", "HiveFileSplit", "fileSplit"); - from_json_key(j, "database", p.database, "HiveSplit", "String", "database"); - from_json_key(j, "table", p.table, "HiveSplit", "String", "table"); - from_json_key( - j, - "partitionName", - p.partitionName, - "HiveSplit", - "String", - "partitionName"); - from_json_key(j, "storage", p.storage, "HiveSplit", "Storage", "storage"); - from_json_key( - j, - "partitionKeys", - p.partitionKeys, - "HiveSplit", - "List", - "partitionKeys"); - from_json_key( - j, - "addresses", - p.addresses, - "HiveSplit", - "List", - "addresses"); - from_json_key( - j, - "readBucketNumber", - p.readBucketNumber, - "HiveSplit", - "int", - "readBucketNumber"); - from_json_key( - j, - "tableBucketNumber", - p.tableBucketNumber, - "HiveSplit", - "int", - "tableBucketNumber"); - from_json_key( - j, - "nodeSelectionStrategy", - p.nodeSelectionStrategy, - "HiveSplit", - "NodeSelectionStrategy", - "nodeSelectionStrategy"); - from_json_key( - j, - "partitionDataColumnCount", - p.partitionDataColumnCount, - "HiveSplit", - "int", - "partitionDataColumnCount"); - from_json_key( - j, - "tableToPartitionMapping", - p.tableToPartitionMapping, - "HiveSplit", - "TableToPartitionMapping", - "tableToPartitionMapping"); - from_json_key( - j, - "bucketConversion", - p.bucketConversion, - "HiveSplit", - "BucketConversion", - "bucketConversion"); - from_json_key( - j, - "s3SelectPushdownEnabled", - p.s3SelectPushdownEnabled, - "HiveSplit", - "bool", - "s3SelectPushdownEnabled"); - from_json_key( - j, - "cacheQuota", - p.cacheQuota, - "HiveSplit", - "CacheQuotaRequirement", - "cacheQuota"); +void from_json(const json& j, TaskUpdateRequest& p) { from_json_key( j, - "encryptionMetadata", - p.encryptionMetadata, - "HiveSplit", - "EncryptionInformation", - "encryptionMetadata"); + "session", + p.session, + "TaskUpdateRequest", + "SessionRepresentation", + "session"); from_json_key( j, - "redundantColumnDomains", - p.redundantColumnDomains, - "HiveSplit", - "List>", - "redundantColumnDomains"); + "extraCredentials", + p.extraCredentials, + "TaskUpdateRequest", + "Map", + "extraCredentials"); + from_json_key( + j, "fragment", p.fragment, "TaskUpdateRequest", "String", "fragment"); from_json_key( j, - "splitWeight", - p.splitWeight, - "HiveSplit", - "SplitWeight", - "splitWeight"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchPartitioningHandle::TpchPartitioningHandle() noexcept { - _type = "tpch"; -} -void to_json(json& j, const TpchPartitioningHandle& p) { - j = json::object(); - to_json_key( - j, - "table", - p.table, - "TpchPartitioningHandle", - "String", - "table"); - to_json_key( - j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); -} -void from_json(const json& j, TpchPartitioningHandle& p) { - p._type = j["@type"]; + "sources", + p.sources, + "TaskUpdateRequest", + "List", + "sources"); from_json_key( j, - "table", - p.table, - "TpchPartitioningHandle", - "String", - "table"); + "outputIds", + p.outputIds, + "TaskUpdateRequest", + "OutputBuffers", + "outputIds"); from_json_key( j, - "totalRows", - p.totalRows, - "TpchPartitioningHandle", - "int64_t", - "totalRows"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchTransactionHandle::TpchTransactionHandle() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchTransactionHandle& p) { - j = json::array(); - j.push_back(p._type); - j.push_back(p.instance); -} - -void from_json(const json& j, TpchTransactionHandle& p) { - j[0].get_to(p._type); - j[1].get_to(p.instance); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchColumnHandle::TpchColumnHandle() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchColumnHandle& p) { - j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "columnName", p.columnName, "TpchColumnHandle", "string", "columnName"); - to_json_key(j, "type", p.type, "TpchColumnHandle", "string", "type"); -} - -void from_json(const json& j, TpchColumnHandle& p) { - p._type = j["@type"]; - from_json_key(j, "columnName", p.columnName, "TpchColumnHandle", "string", "columnName"); - from_json_key(j, "type", p.type, "TpchColumnHandle", "string", "type"); + "tableWriteInfo", + p.tableWriteInfo, + "TaskUpdateRequest", + "TableWriteInfo", + "tableWriteInfo"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchPredicate::TpchPredicate() noexcept {} +// Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() -void to_json(json& j, const TpchPredicate& p) { - j = json::object(); - to_json_key(j, "columnDomains", p.columnDomains, "TpchPredicate", "List>>", "columnDomains"); +// NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays +static const std::pair LimitNodeStep_enum_table[] = + { // NOLINT: cert-err58-cpp + {LimitNodeStep::PARTIAL, "PARTIAL"}, + {LimitNodeStep::FINAL, "FINAL"}}; +void to_json(json& j, const LimitNodeStep& e) { + static_assert( + std::is_enum::value, "LimitNodeStep must be an enum!"); + const auto* it = std::find_if( + std::begin(LimitNodeStep_enum_table), + std::end(LimitNodeStep_enum_table), + [e](const std::pair& ej_pair) -> bool { + return ej_pair.first == e; + }); + j = ((it != std::end(LimitNodeStep_enum_table)) + ? it + : std::begin(LimitNodeStep_enum_table)) + ->second; } - -void from_json(const json& j, TpchPredicate& p) { - from_json_key(j, "columnDomains", p.columnDomains, "TpchPredicate", "List>>", "columnDomains"); +void from_json(const json& j, LimitNodeStep& e) { + static_assert( + std::is_enum::value, "LimitNodeStep must be an enum!"); + const auto* it = std::find_if( + std::begin(LimitNodeStep_enum_table), + std::end(LimitNodeStep_enum_table), + [&j](const std::pair& ej_pair) -> bool { + return ej_pair.second == j; + }); + e = ((it != std::end(LimitNodeStep_enum_table)) + ? it + : std::begin(LimitNodeStep_enum_table)) + ->first; } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchTableLayoutHandle::TpchTableLayoutHandle() noexcept { - _type = "tpch"; +LimitNode::LimitNode() noexcept { + _type = ".LimitNode"; } -void to_json(json& j, const TpchTableLayoutHandle& p) { +void to_json(json& j, const LimitNode& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); - to_json_key(j, "predicate", p.predicate, "TpchTableLayoutHandle", "TpchPredicate", "predicate"); + j["@type"] = ".LimitNode"; + to_json_key(j, "id", p.id, "LimitNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "LimitNode", "PlanNode", "source"); + to_json_key(j, "count", p.count, "LimitNode", "int64_t", "count"); + to_json_key(j, "step", p.step, "LimitNode", "LimitNodeStep", "step"); } -void from_json(const json& j, TpchTableLayoutHandle& p) { +void from_json(const json& j, LimitNode& p) { p._type = j["@type"]; - from_json_key(j, "table", p.table, "TpchTableLayoutHandle", "TpchTableHandle", "table"); - from_json_key(j, "predicate", p.predicate, "TpchTableLayoutHandle", "TpchPredicate", "predicate"); + from_json_key(j, "id", p.id, "LimitNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "LimitNode", "PlanNode", "source"); + from_json_key(j, "count", p.count, "LimitNode", "int64_t", "count"); + from_json_key(j, "step", p.step, "LimitNode", "LimitNodeStep", "step"); } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -TpchTableHandle::TpchTableHandle() noexcept { - _type = "tpch"; +HivePartitioningHandle::HivePartitioningHandle() noexcept { + _type = "hive"; } -void to_json(json& j, const TpchTableHandle& p) { +void to_json(json& j, const HivePartitioningHandle& p) { j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "tableName", p.tableName, "TpchTableHandle", "string", "tableName"); - to_json_key(j, "scaleFactor", p.scaleFactor, "TpchTableHandle", "double", "scaleFactor"); + j["@type"] = "hive"; + to_json_key( + j, + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); + to_json_key( + j, + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); + to_json_key( + j, + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); + to_json_key( + j, + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); + to_json_key( + j, "types", p.types, "HivePartitioningHandle", "List", "types"); } -void from_json(const json& j, TpchTableHandle& p) { +void from_json(const json& j, HivePartitioningHandle& p) { p._type = j["@type"]; - from_json_key(j, "tableName", p.tableName, "TpchTableHandle", "string", "tableName"); - from_json_key(j, "scaleFactor", p.scaleFactor, "TpchTableHandle", "double", "scaleFactor"); -} -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -TpchSplit::TpchSplit() noexcept { - _type = "tpch"; -} - -void to_json(json& j, const TpchSplit& p) { - j = json::object(); - j["@type"] = "tpch"; - to_json_key(j, "tableHandle", p.tableHandle, "TpchSplit", "TpchTableHandle", "tableHandle"); - to_json_key(j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); - to_json_key(j, "totalParts", p.partNumber, "TpchSplit", "int", "totalParts"); - to_json_key( + from_json_key( j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); - to_json_key( + "bucketCount", + p.bucketCount, + "HivePartitioningHandle", + "int", + "bucketCount"); + from_json_key( j, - "predicate", - p.predicate, - "TpchSplit", - "List>", - "predicate"); + "maxCompatibleBucketCount", + p.maxCompatibleBucketCount, + "HivePartitioningHandle", + "int", + "maxCompatibleBucketCount"); + from_json_key( + j, + "bucketFunctionType", + p.bucketFunctionType, + "HivePartitioningHandle", + "BucketFunctionType", + "bucketFunctionType"); + from_json_key( + j, + "hiveTypes", + p.hiveTypes, + "HivePartitioningHandle", + "List", + "hiveTypes"); + from_json_key( + j, "types", p.types, "HivePartitioningHandle", "List", "types"); } +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ -void from_json(const json& j, TpchSplit& p) { - p._type = j["@type"]; - from_json_key(j, "tableHandle", p.tableHandle, "TpchSplit", "TpchTableHandle", "tableHandle"); - from_json_key(j, "partNumber", p.partNumber, "TpchSplit", "int", "partNumber"); - from_json_key(j, "totalParts", p.totalParts, "TpchSplit", "int", "totalParts"); - from_json_key( - j, - "addresses", - p.addresses, - "TpchSplit", - "List", - "addresses"); - from_json_key( - j, - "predicate", - p.predicate, - "TpchSplit", - "TpchPredicate", - "predicate"); +// dependency KeyedSubclass + +namespace facebook::presto::protocol { + +std::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) { + return j["@type"]; } + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { OutputNode::OutputNode() noexcept { @@ -11819,6 +11643,239 @@ void from_json(const json& j, OutputNode& p) { } } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +SortNode::SortNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.SortNode"; +} + +void to_json(json& j, const SortNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.SortNode"; + to_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + to_json_key( + j, + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + to_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); +} + +void from_json(const json& j, SortNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "SortNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "SortNode", "PlanNode", "source"); + from_json_key( + j, + "orderingScheme", + p.orderingScheme, + "SortNode", + "OrderingScheme", + "orderingScheme"); + from_json_key(j, "isPartial", p.isPartial, "SortNode", "bool", "isPartial"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +RowNumberNode::RowNumberNode() noexcept { + _type = "com.facebook.presto.sql.planner.plan.RowNumberNode"; +} + +void to_json(json& j, const RowNumberNode& p) { + j = json::object(); + j["@type"] = "com.facebook.presto.sql.planner.plan.RowNumberNode"; + to_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); + to_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); + to_json_key( + j, + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); + to_json_key( + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + to_json_key( + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + to_json_key( + j, + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); +} + +void from_json(const json& j, RowNumberNode& p) { + p._type = j["@type"]; + from_json_key(j, "id", p.id, "RowNumberNode", "PlanNodeId", "id"); + from_json_key(j, "source", p.source, "RowNumberNode", "PlanNode", "source"); + from_json_key( + j, + "partitionBy", + p.partitionBy, + "RowNumberNode", + "List", + "partitionBy"); + from_json_key( + j, + "rowNumberVariable", + p.rowNumberVariable, + "RowNumberNode", + "VariableReferenceExpression", + "rowNumberVariable"); + from_json_key( + j, + "maxRowCountPerPartition", + p.maxRowCountPerPartition, + "RowNumberNode", + "Integer", + "maxRowCountPerPartition"); + from_json_key( + j, + "hashVariable", + p.hashVariable, + "RowNumberNode", + "VariableReferenceExpression", + "hashVariable"); +} +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace facebook::presto::protocol { + +void to_json(json& j, const PlanFragment& p) { + j = json::object(); + to_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); + to_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); + to_json_key( + j, + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); + to_json_key( + j, + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); + to_json_key( + j, + "tableScanSchedulingOrder", + p.tableScanSchedulingOrder, + "PlanFragment", + "List", + "tableScanSchedulingOrder"); + to_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "PlanFragment", + "PartitioningScheme", + "partitioningScheme"); + to_json_key( + j, + "stageExecutionDescriptor", + p.stageExecutionDescriptor, + "PlanFragment", + "StageExecutionDescriptor", + "stageExecutionDescriptor"); + to_json_key( + j, + "outputTableWriterFragment", + p.outputTableWriterFragment, + "PlanFragment", + "bool", + "outputTableWriterFragment"); + to_json_key( + j, + "jsonRepresentation", + p.jsonRepresentation, + "PlanFragment", + "String", + "jsonRepresentation"); +} + +void from_json(const json& j, PlanFragment& p) { + from_json_key(j, "id", p.id, "PlanFragment", "PlanFragmentId", "id"); + from_json_key(j, "root", p.root, "PlanFragment", "PlanNode", "root"); + from_json_key( + j, + "variables", + p.variables, + "PlanFragment", + "List", + "variables"); + from_json_key( + j, + "partitioning", + p.partitioning, + "PlanFragment", + "PartitioningHandle", + "partitioning"); + from_json_key( + j, + "tableScanSchedulingOrder", + p.tableScanSchedulingOrder, + "PlanFragment", + "List", + "tableScanSchedulingOrder"); + from_json_key( + j, + "partitioningScheme", + p.partitioningScheme, + "PlanFragment", + "PartitioningScheme", + "partitioningScheme"); + from_json_key( + j, + "stageExecutionDescriptor", + p.stageExecutionDescriptor, + "PlanFragment", + "StageExecutionDescriptor", + "stageExecutionDescriptor"); + from_json_key( + j, + "outputTableWriterFragment", + p.outputTableWriterFragment, + "PlanFragment", + "bool", + "outputTableWriterFragment"); + from_json_key( + j, + "jsonRepresentation", + p.jsonRepresentation, + "PlanFragment", + "String", + "jsonRepresentation"); +} +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { // Loosly copied this here from NLOHMANN_JSON_SERIALIZE_ENUM() // NOLINTNEXTLINE: cppcoreguidelines-avoid-c-arrays diff --git a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h index c5f5ed6d7368d..a977f00df5340 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.h @@ -252,6 +252,13 @@ struct adl_serializer> { // Forward declaration of all abstract types // namespace facebook::presto::protocol { +struct PlanNode : public JsonEncodedSubclass { + PlanNodeId id = {}; +}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { struct ConnectorTransactionHandle : public JsonEncodedSubclass {}; void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); @@ -262,16 +269,9 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorMetadataUpdateHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json( - const json& j, - std::shared_ptr& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ValueSet : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +struct ConnectorPartitioningHandle : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct RowExpression : public JsonEncodedSubclass { @@ -286,11 +286,11 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PlanNode : public JsonEncodedSubclass { - PlanNodeId id = {}; -}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +struct ConnectorMetadataUpdateHandle : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json( + const json& j, + std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct ConnectorTableLayoutHandle : public JsonEncodedSubclass {}; @@ -303,14 +303,9 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorPartitioningHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct ConnectorOutputTableHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); +struct ConnectorInsertTableHandle : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct ColumnHandle : public JsonEncodedSubclass { @@ -322,16 +317,29 @@ void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConnectorInsertTableHandle : public JsonEncodedSubclass {}; -void to_json(json& j, const std::shared_ptr& p); -void from_json(const json& j, std::shared_ptr& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { struct ExecutionWriterTarget : public JsonEncodedSubclass {}; void to_json(json& j, const std::shared_ptr& p); void from_json(const json& j, std::shared_ptr& p); } // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ConnectorOutputTableHandle : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ValueSet : public JsonEncodedSubclass {}; +void to_json(json& j, const std::shared_ptr& p); +void from_json(const json& j, std::shared_ptr& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SourceLocation { + int line = {}; + int column = {}; +}; +void to_json(json& j, const SourceLocation& p); +void from_json(const json& j, SourceLocation& p); +} // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -345,63 +353,69 @@ void from_json(const json& j, std::shared_ptr& p); * See the License for the specific language governing permissions and * limitations under the License. */ - namespace facebook::presto::protocol { -using HostAddress = std::string; +struct VariableReferenceExpression : RowExpression { + String name; + Type type; // dependency -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class ErrorCause { - UNKNOWN, - LOW_PARTITION_COUNT, - EXCEEDS_BROADCAST_MEMORY_LIMIT -}; -extern void to_json(json& j, const ErrorCause& e); -extern void from_json(const json& j, ErrorCause& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class ErrorType { - USER_ERROR, - INTERNAL_ERROR, - INSUFFICIENT_RESOURCES, - EXTERNAL + VariableReferenceExpression() noexcept; + explicit VariableReferenceExpression(const String& str) { + _type = "variable"; + + std::vector> parts; + + folly::split("<", str, parts); + name = parts[0]; + type = parts[1].substr(0, parts[1].length() - 1); + } + + bool operator<(const VariableReferenceExpression& o) const { + if (name == o.name) { + return type < o.type; + } + + return name < o.name; + } }; -extern void to_json(json& j, const ErrorType& e); -extern void from_json(const json& j, ErrorType& e); + +void to_json(json& j, const VariableReferenceExpression& p); +void from_json(const json& j, VariableReferenceExpression& p); + +std::string json_map_key( + const facebook::presto::protocol::VariableReferenceExpression& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ErrorCode { - int code = {}; - String name = {}; - ErrorType type = {}; - bool retriable = {}; +struct GroupIdNode : public PlanNode { + std::shared_ptr source = {}; + List> groupingSets = {}; + Map + groupingColumns = {}; + List aggregationArguments = {}; + VariableReferenceExpression groupIdVariable = {}; + + GroupIdNode() noexcept; }; -void to_json(json& j, const ErrorCode& p); -void from_json(const json& j, ErrorCode& p); +void to_json(json& j, const GroupIdNode& p); +void from_json(const json& j, GroupIdNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ErrorLocation { - int lineNumber = {}; - int columnNumber = {}; +struct DwrfEncryptionMetadata { + Map fieldToKeyData = {}; + Map extraMetadata = {}; + String encryptionAlgorithm = {}; + String encryptionProvider = {}; }; -void to_json(json& j, const ErrorLocation& p); -void from_json(const json& j, ErrorLocation& p); +void to_json(json& j, const DwrfEncryptionMetadata& p); +void from_json(const json& j, DwrfEncryptionMetadata& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ExecutionFailureInfo { - String type = {}; - String message = {}; - std::shared_ptr cause = {}; - List suppressed = {}; - List stack = {}; - ErrorLocation errorLocation = {}; - ErrorCode errorCode = {}; - HostAddress remoteHost = {}; - ErrorCause errorCause = {}; +struct EncryptionInformation { + std::shared_ptr dwrfEncryptionMetadata = {}; }; -void to_json(json& j, const ExecutionFailureInfo& p); -void from_json(const json& j, ExecutionFailureInfo& p); +void to_json(json& j, const EncryptionInformation& p); +void from_json(const json& j, EncryptionInformation& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -418,11 +432,25 @@ void from_json(const json& j, ExecutionFailureInfo& p); */ namespace facebook::presto::protocol { -std::ostream& operator<<(std::ostream& os, const DataSize& d); +struct Lifespan { + bool isgroup = false; + long groupid = 0; -void to_json(nlohmann::json& j, const DataSize& p); -void from_json(const nlohmann::json& j, DataSize& p); + bool operator<(const Lifespan& o) const { + return groupid < o.groupid; + } +}; +void to_json(json& j, const Lifespan& p); +void from_json(const json& j, Lifespan& p); + +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SplitContext { + bool cacheable = {}; +}; +void to_json(json& j, const SplitContext& p); +void from_json(const json& j, SplitContext& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -438,23 +466,30 @@ void from_json(const nlohmann::json& j, DataSize& p); * limitations under the License. */ +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. + namespace facebook::presto::protocol { -std::ostream& operator<<(std::ostream& os, const Duration& d); +struct TpchTransactionHandle : public ConnectorTransactionHandle { + String instance = {}; +}; -void to_json(json& j, const Duration& p); -void from_json(const json& j, Duration& p); +void to_json(json& j, const TpchTransactionHandle& p); + +void from_json(const json& j, TpchTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ResourceEstimates { - std::shared_ptr executionTime = {}; - std::shared_ptr cpuTime = {}; - std::shared_ptr peakMemory = {}; - std::shared_ptr peakTaskMemory = {}; +struct Split { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorSplit = {}; + Lifespan lifespan = {}; + SplitContext splitContext = {}; }; -void to_json(json& j, const ResourceEstimates& p); -void from_json(const json& j, ResourceEstimates& p); +void to_json(json& j, const Split& p); +void from_json(const json& j, Split& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -471,63 +506,29 @@ void from_json(const json& j, ResourceEstimates& p); */ namespace facebook::presto::protocol { -struct Lifespan { - bool isgroup = false; - long groupid = 0; +struct ScheduledSplit { + long sequenceId = {}; + PlanNodeId planNodeId = {}; // dependency + Split split = {}; - bool operator<(const Lifespan& o) const { - return groupid < o.groupid; + bool operator<(const ScheduledSplit& o) const { + return sequenceId < o.sequenceId; } }; -void to_json(json& j, const Lifespan& p); -void from_json(const json& j, Lifespan& p); +void to_json(json& j, const ScheduledSplit& p); +void from_json(const json& j, ScheduledSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class TaskState { PLANNED, RUNNING, FINISHED, CANCELED, ABORTED, FAILED }; -extern void to_json(json& j, const TaskState& e); -extern void from_json(const json& j, TaskState& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct TaskStatus { - int64_t taskInstanceIdLeastSignificantBits = {}; - int64_t taskInstanceIdMostSignificantBits = {}; - int64_t version = {}; - TaskState state = {}; - URI self = {}; - List completedDriverGroups = {}; - List failures = {}; - int queuedPartitionedDrivers = {}; - int runningPartitionedDrivers = {}; - double outputBufferUtilization = {}; - bool outputBufferOverutilized = {}; - int64_t physicalWrittenDataSizeInBytes = {}; - int64_t memoryReservationInBytes = {}; - int64_t systemMemoryReservationInBytes = {}; - int64_t peakNodeTotalMemoryReservationInBytes = {}; - int64_t fullGcCount = {}; - int64_t fullGcTimeInMillis = {}; - int64_t totalCpuTimeInNanos = {}; - int64_t taskAgeInMillis = {}; - int64_t queuedPartitionedSplitsWeight = {}; - int64_t runningPartitionedSplitsWeight = {}; +struct TaskSource { + PlanNodeId planNodeId = {}; + List splits = {}; + List noMoreSplitsForLifespan = {}; + bool noMoreSplits = {}; }; -void to_json(json& j, const TaskStatus& p); -void from_json(const json& j, TaskStatus& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class CacheQuotaScope { GLOBAL, SCHEMA, TABLE, PARTITION }; -extern void to_json(json& j, const CacheQuotaScope& e); -extern void from_json(const json& j, CacheQuotaScope& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct CacheQuotaRequirement { - CacheQuotaScope cacheQuotaScope = {}; - std::shared_ptr quota = {}; -}; -void to_json(json& j, const CacheQuotaRequirement& p); -void from_json(const json& j, CacheQuotaRequirement& p); +void to_json(json& j, const TaskSource& p); +void from_json(const json& j, TaskSource& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -543,138 +544,139 @@ void from_json(const json& j, CacheQuotaRequirement& p); * limitations under the License. */ namespace facebook::presto::protocol { -struct OperatorInfo {}; -void to_json(json& j, const OperatorInfo& p); -void from_json(const json& j, OperatorInfo& p); + +struct TypeVariableConstraint { + String name = {}; + bool comparableRequired = {}; + bool orderableRequired = {}; + String variadicBound = {}; + bool nonDecimalNumericRequired = {}; + String boundedBy = {}; +}; +void to_json(json& j, const TypeVariableConstraint& p); +void from_json(const json& j, TypeVariableConstraint& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BlockedReason { WAITING_FOR_MEMORY }; -extern void to_json(json& j, const BlockedReason& e); -extern void from_json(const json& j, BlockedReason& e); +enum class SelectedRoleType { ROLE, ALL, NONE }; +extern void to_json(json& j, const SelectedRoleType& e); +extern void from_json(const json& j, SelectedRoleType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OperatorStats { - int stageId = {}; - int stageExecutionId = {}; - int pipelineId = {}; - int operatorId = {}; - PlanNodeId planNodeId = {}; - String operatorType = {}; - int64_t totalDrivers = {}; - int64_t addInputCalls = {}; - Duration addInputWall = {}; - Duration addInputCpu = {}; - DataSize addInputAllocation = {}; - DataSize rawInputDataSize = {}; - int64_t rawInputPositions = {}; - DataSize inputDataSize = {}; - int64_t inputPositions = {}; - double sumSquaredInputPositions = {}; - int64_t getOutputCalls = {}; - Duration getOutputWall = {}; - Duration getOutputCpu = {}; - DataSize getOutputAllocation = {}; - DataSize outputDataSize = {}; - int64_t outputPositions = {}; - DataSize physicalWrittenDataSize = {}; - Duration additionalCpu = {}; - Duration blockedWall = {}; - int64_t finishCalls = {}; - Duration finishWall = {}; - Duration finishCpu = {}; - DataSize finishAllocation = {}; - DataSize userMemoryReservation = {}; - DataSize revocableMemoryReservation = {}; - DataSize systemMemoryReservation = {}; - DataSize peakUserMemoryReservation = {}; - DataSize peakSystemMemoryReservation = {}; - DataSize peakTotalMemoryReservation = {}; - DataSize spilledDataSize = {}; - std::shared_ptr blockedReason = {}; - OperatorInfo info = {}; - RuntimeStats runtimeStats = {}; +struct SelectedRole { + SelectedRoleType type = {}; + std::shared_ptr role = {}; }; -void to_json(json& j, const OperatorStats& p); -void from_json(const json& j, OperatorStats& p); +void to_json(json& j, const SelectedRole& p); +void from_json(const json& j, SelectedRole& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DriverStats { - Lifespan lifespan = {}; - DateTime createTime = {}; - DateTime startTime = {}; - DateTime endTime = {}; - Duration queuedTime = {}; - Duration elapsedTime = {}; - DataSize userMemoryReservation = {}; - DataSize revocableMemoryReservation = {}; - DataSize systemMemoryReservation = {}; - Duration totalScheduledTime = {}; - Duration totalCpuTime = {}; - Duration totalBlockedTime = {}; - bool fullyBlocked = {}; - List blockedReasons = {}; - DataSize totalAllocation = {}; - DataSize rawInputDataSize = {}; - int64_t rawInputPositions = {}; - Duration rawInputReadTime = {}; - DataSize processedInputDataSize = {}; - int64_t processedInputPositions = {}; - DataSize outputDataSize = {}; - int64_t outputPositions = {}; - DataSize physicalWrittenDataSize = {}; - List operatorStats = {}; +enum class ExchangeNodeScope { LOCAL, REMOTE_STREAMING, REMOTE_MATERIALIZED }; +extern void to_json(json& j, const ExchangeNodeScope& e); +extern void from_json(const json& j, ExchangeNodeScope& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class SortOrder { + ASC_NULLS_FIRST, + ASC_NULLS_LAST, + DESC_NULLS_FIRST, + DESC_NULLS_LAST }; -void to_json(json& j, const DriverStats& p); -void from_json(const json& j, DriverStats& p); +extern void to_json(json& j, const SortOrder& e); +extern void from_json(const json& j, SortOrder& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SplitContext { - bool cacheable = {}; +struct Ordering { + VariableReferenceExpression variable = {}; + SortOrder sortOrder = {}; }; -void to_json(json& j, const SplitContext& p); -void from_json(const json& j, SplitContext& p); +void to_json(json& j, const Ordering& p); +void from_json(const json& j, Ordering& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Split { - ConnectorId connectorId = {}; +struct OrderingScheme { + List orderBy = {}; +}; +void to_json(json& j, const OrderingScheme& p); +void from_json(const json& j, OrderingScheme& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class ExchangeNodeType { + GATHER, + REPARTITION, + REPLICATE, +}; +extern void to_json(json& j, const ExchangeNodeType& e); +extern void from_json(const json& j, ExchangeNodeType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct PartitioningHandle { + std::shared_ptr connectorId = {}; std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorSplit = {}; - Lifespan lifespan = {}; - SplitContext splitContext = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const Split& p); -void from_json(const json& j, Split& p); +void to_json(json& j, const PartitioningHandle& p); +void from_json(const json& j, PartitioningHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MetadataUpdates { - ConnectorId connectorId = {}; - List> metadataUpdates = {}; +struct Partitioning { + PartitioningHandle handle = {}; + List> arguments = {}; }; -void to_json(json& j, const MetadataUpdates& p); -void from_json(const json& j, MetadataUpdates& p); +void to_json(json& j, const Partitioning& p); +void from_json(const json& j, Partitioning& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SourceLocation { - int line = {}; - int column = {}; +struct PartitioningScheme { + Partitioning partitioning = {}; + List outputLayout = {}; + std::shared_ptr hashColumn = {}; + bool replicateNullsAndAny = {}; + std::shared_ptr> bucketToPartition = {}; }; -void to_json(json& j, const SourceLocation& p); -void from_json(const json& j, SourceLocation& p); +void to_json(json& j, const PartitioningScheme& p); +void from_json(const json& j, PartitioningScheme& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveBucketFilter { - List bucketsToKeep = {}; +struct ExchangeNode : public PlanNode { + ExchangeNodeType type = {}; + ExchangeNodeScope scope = {}; + PartitioningScheme partitioningScheme = {}; + List> sources = {}; + List> inputs = {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; + + ExchangeNode() noexcept; }; -void to_json(json& j, const HiveBucketFilter& p); -void from_json(const json& j, HiveBucketFilter& p); +void to_json(json& j, const ExchangeNode& p); +void from_json(const json& j, ExchangeNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Domain { - std::shared_ptr values = {}; - bool nullAllowed = {}; +struct RemoteSourceNode : public PlanNode { + List sourceFragmentIds = {}; + List outputVariables = {}; + bool ensureSourceOrdering = {}; + std::shared_ptr orderingScheme = {}; + ExchangeNodeType exchangeType = {}; + + RemoteSourceNode() noexcept; }; -void to_json(json& j, const Domain& p); -void from_json(const json& j, Domain& p); +void to_json(json& j, const RemoteSourceNode& p); +void from_json(const json& j, RemoteSourceNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Parameter { + String name = {}; + TypeSignature type = {}; +}; +void to_json(json& j, const Parameter& p); +void from_json(const json& j, Parameter& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class Bound { BELOW, EXACTLY, ABOVE }; +extern void to_json(json& j, const Bound& e); +extern void from_json(const json& j, Bound& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -691,168 +693,63 @@ void from_json(const json& j, Domain& p); */ namespace facebook::presto::protocol { -template -struct pointerDerefCompare { - bool operator()(const std::shared_ptr& a, const std::shared_ptr& b) - const { - return *a < *b; - } +struct Block { + std::string data; }; -template -struct TupleDomain { - std::shared_ptr> domains; -}; +void to_json(json& j, const Block& p); -template -struct TupleDomain> { - std::shared_ptr, Domain, pointerDerefCompare>> - domains; -}; +void from_json(const json& j, Block& p); -template -struct ColumnDomain { - T column; - Domain domain; // dependency +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Marker { + Type type = {}; + std::shared_ptr valueBlock = {}; + Bound bound = {}; }; - +void to_json(json& j, const Marker& p); +void from_json(const json& j, Marker& p); } // namespace facebook::presto::protocol - -namespace nlohmann { - -template -struct adl_serializer> { - static void to_json( - json& j, - const facebook::presto::protocol::ColumnDomain& p) { - facebook::presto::protocol::to_json_key( - j, "column", p.column, "ColumnDomain", "T", "column"); - facebook::presto::protocol::to_json_key( - j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); - } - - static void from_json( - const json& j, - facebook::presto::protocol::ColumnDomain& p) { - facebook::presto::protocol::from_json_key( - j, "column", p.column, "ColumnDomain", "T", "column"); - facebook::presto::protocol::from_json_key( - j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); - } -}; - -template -struct adl_serializer> { - static void to_json( - json& j, - const facebook::presto::protocol::TupleDomain& tup) { - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain> - list; - if (tup.domains != nullptr) { - for (auto& el : *tup.domains) { - facebook::presto::protocol::ColumnDomain domain; - domain.column = el.first; - domain.domain = el.second; - list.push_back(domain); - } - } - - j["columnDomains"] = list; - } - - static void from_json( - const json& j, - facebook::presto::protocol::TupleDomain& tup) { - if (j.count("columnDomains") != 0U) { - std::shared_ptr> - map = std::make_shared< - std::map>(); - - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain> - list = j.at("columnDomains"); - for (const facebook::presto::protocol::ColumnDomain& value : list) { - map->insert(std::make_pair(T(value.column), value.domain)); - } - tup.domains = map; - } - } +namespace facebook::presto::protocol { +struct Range { + Marker low = {}; + Marker high = {}; }; +void to_json(json& j, const Range& p); +void from_json(const json& j, Range& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct SortedRangeSet : public ValueSet { + Type type = {}; + List ranges = {}; -template -struct adl_serializer< - facebook::presto::protocol::TupleDomain>> { - static void to_json( - json& j, - const facebook::presto::protocol::TupleDomain>& tup) { - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain>> - list; - if (tup.domains != nullptr) { - for (auto& el : *tup.domains) { - facebook::presto::protocol::ColumnDomain> domain; - domain.column = el.first; - domain.domain = el.second; - list.push_back(domain); - } - } - - j["columnDomains"] = list; - } - - static void from_json( - const json& j, - facebook::presto::protocol::TupleDomain>& tup) { - if (j.count("columnDomains") != 0U) { - auto map = std::make_shared, - facebook::presto::protocol::Domain, - facebook::presto::protocol::pointerDerefCompare>>(); - - facebook::presto::protocol::List< - facebook::presto::protocol::ColumnDomain>> - list = j.at("columnDomains"); - for (const facebook::presto::protocol::ColumnDomain>& - value : list) { - map->insert( - std::make_pair(std::shared_ptr(value.column), value.domain)); - } - tup.domains = map; - } - } + SortedRangeSet() noexcept; }; - -} // namespace nlohmann +void to_json(json& j, const SortedRangeSet& p); +void from_json(const json& j, SortedRangeSet& p); +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class Form { - IF, - NULL_IF, - SWITCH, - WHEN, - IS_NULL, - COALESCE, - IN, - AND, - OR, - DEREFERENCE, - ROW_CONSTRUCTOR, - BIND +enum class TaskState { PLANNED, RUNNING, FINISHED, CANCELED, ABORTED, FAILED }; +extern void to_json(json& j, const TaskState& e); +extern void from_json(const json& j, TaskState& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ErrorLocation { + int lineNumber = {}; + int columnNumber = {}; }; -extern void to_json(json& j, const Form& e); -extern void from_json(const json& j, Form& e); +void to_json(json& j, const ErrorLocation& p); +void from_json(const json& j, ErrorLocation& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SpecialFormExpression : public RowExpression { - Form form = {}; - Type returnType = {}; - List> arguments = {}; - - SpecialFormExpression() noexcept; +enum class ErrorCause { + UNKNOWN, + LOW_PARTITION_COUNT, + EXCEEDS_BROADCAST_MEMORY_LIMIT }; -void to_json(json& j, const SpecialFormExpression& p); -void from_json(const json& j, SpecialFormExpression& p); +extern void to_json(json& j, const ErrorCause& e); +extern void from_json(const json& j, ErrorCause& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -867,71 +764,160 @@ void from_json(const json& j, SpecialFormExpression& p); * See the License for the specific language governing permissions and * limitations under the License. */ -namespace facebook::presto::protocol { - -struct VariableReferenceExpression : RowExpression { - String name; - Type type; // dependency - VariableReferenceExpression() noexcept; - explicit VariableReferenceExpression(const String& str) { - _type = "variable"; - - std::vector> parts; - - folly::split("<", str, parts); - name = parts[0]; - type = parts[1].substr(0, parts[1].length() - 1); - } +namespace facebook::presto::protocol { - bool operator<(const VariableReferenceExpression& o) const { - if (name == o.name) { - return type < o.type; - } +using HostAddress = std::string; - return name < o.name; - } +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class ErrorType { + USER_ERROR, + INTERNAL_ERROR, + INSUFFICIENT_RESOURCES, + EXTERNAL }; +extern void to_json(json& j, const ErrorType& e); +extern void from_json(const json& j, ErrorType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ErrorCode { + int code = {}; + String name = {}; + ErrorType type = {}; + bool retriable = {}; +}; +void to_json(json& j, const ErrorCode& p); +void from_json(const json& j, ErrorCode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct ExecutionFailureInfo { + String type = {}; + String message = {}; + std::shared_ptr cause = {}; + List suppressed = {}; + List stack = {}; + ErrorLocation errorLocation = {}; + ErrorCode errorCode = {}; + HostAddress remoteHost = {}; + ErrorCause errorCause = {}; +}; +void to_json(json& j, const ExecutionFailureInfo& p); +void from_json(const json& j, ExecutionFailureInfo& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct TaskStatus { + int64_t taskInstanceIdLeastSignificantBits = {}; + int64_t taskInstanceIdMostSignificantBits = {}; + int64_t version = {}; + TaskState state = {}; + URI self = {}; + List completedDriverGroups = {}; + List failures = {}; + int queuedPartitionedDrivers = {}; + int runningPartitionedDrivers = {}; + double outputBufferUtilization = {}; + bool outputBufferOverutilized = {}; + int64_t physicalWrittenDataSizeInBytes = {}; + int64_t memoryReservationInBytes = {}; + int64_t systemMemoryReservationInBytes = {}; + int64_t peakNodeTotalMemoryReservationInBytes = {}; + int64_t fullGcCount = {}; + int64_t fullGcTimeInMillis = {}; + int64_t totalCpuTimeInNanos = {}; + int64_t taskAgeInMillis = {}; + int64_t queuedPartitionedSplitsWeight = {}; + int64_t runningPartitionedSplitsWeight = {}; +}; +void to_json(json& j, const TaskStatus& p); +void from_json(const json& j, TaskStatus& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct HiveBucketFilter { + List bucketsToKeep = {}; +}; +void to_json(json& j, const HiveBucketFilter& p); +void from_json(const json& j, HiveBucketFilter& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct EmptySplit : public ConnectorSplit { + ConnectorId connectorId = {}; -void to_json(json& j, const VariableReferenceExpression& p); -void from_json(const json& j, VariableReferenceExpression& p); - -std::string json_map_key( - const facebook::presto::protocol::VariableReferenceExpression& p); + EmptySplit() noexcept; +}; +void to_json(json& j, const EmptySplit& p); +void from_json(const json& j, EmptySplit& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Location { + String location = {}; +}; +void to_json(json& j, const Location& p); +void from_json(const json& j, Location& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct RemoteSplit : public ConnectorSplit { + Location location = {}; + TaskId remoteSourceTaskId = {}; + RemoteSplit() noexcept; +}; +void to_json(json& j, const RemoteSplit& p); +void from_json(const json& j, RemoteSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct EquiJoinClause { - VariableReferenceExpression left = {}; - VariableReferenceExpression right = {}; +struct VariableStatsEstimate { + double lowValue = {}; + double highValue = {}; + double nullsFraction = {}; + double averageRowSize = {}; + double distinctValuesCount = {}; }; -void to_json(json& j, const EquiJoinClause& p); -void from_json(const json& j, EquiJoinClause& p); +void to_json(json& j, const VariableStatsEstimate& p); +void from_json(const json& j, VariableStatsEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SortOrder { - ASC_NULLS_FIRST, - ASC_NULLS_LAST, - DESC_NULLS_FIRST, - DESC_NULLS_LAST +struct PlanNodeStatsEstimate { + double outputRowCount = {}; + double totalSize = {}; + bool confident = {}; + Map variableStatistics = + {}; }; -extern void to_json(json& j, const SortOrder& e); -extern void from_json(const json& j, SortOrder& e); +void to_json(json& j, const PlanNodeStatsEstimate& p); +void from_json(const json& j, PlanNodeStatsEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Ordering { - VariableReferenceExpression variable = {}; - SortOrder sortOrder = {}; +enum class BoundType { + UNBOUNDED_PRECEDING, + PRECEDING, + CURRENT_ROW, + FOLLOWING, + UNBOUNDED_FOLLOWING }; -void to_json(json& j, const Ordering& p); -void from_json(const json& j, Ordering& p); +extern void to_json(json& j, const BoundType& e); +extern void from_json(const json& j, BoundType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OrderingScheme { - List orderBy = {}; +enum class WindowType { + RANGE, + ROWS, }; -void to_json(json& j, const OrderingScheme& p); -void from_json(const json& j, OrderingScheme& p); +extern void to_json(json& j, const WindowType& e); +extern void from_json(const json& j, WindowType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct Frame { + WindowType type = {}; + BoundType startType = {}; + std::shared_ptr startValue = {}; + BoundType endType = {}; + std::shared_ptr endValue = {}; + std::shared_ptr originalStartValue = {}; + std::shared_ptr originalEndValue = {}; +}; +void to_json(json& j, const Frame& p); +void from_json(const json& j, Frame& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct Specification { @@ -942,11 +928,6 @@ void to_json(json& j, const Specification& p); void from_json(const json& j, Specification& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ColumnType { PARTITION_KEY, REGULAR, SYNTHESIZED, AGGREGATED }; -extern void to_json(json& j, const ColumnType& e); -extern void from_json(const json& j, ColumnType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { struct CallExpression : public RowExpression { String displayName = {}; std::shared_ptr functionHandle = {}; @@ -959,6 +940,36 @@ void to_json(json& j, const CallExpression& p); void from_json(const json& j, CallExpression& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +struct Function { + CallExpression functionCall = {}; + Frame frame = {}; + bool ignoreNulls = {}; +}; +void to_json(json& j, const Function& p); +void from_json(const json& j, Function& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct WindowNode : public PlanNode { + std::shared_ptr sourceLocation = {}; + + std::shared_ptr source = {}; + Specification specification = {}; + Map windowFunctions = {}; + std::shared_ptr hashVariable = {}; + List prePartitionedInputs = {}; + int preSortedOrderPrefix = {}; + + WindowNode() noexcept; +}; +void to_json(json& j, const WindowNode& p); +void from_json(const json& j, WindowNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +enum class ColumnType { PARTITION_KEY, REGULAR, SYNTHESIZED, AGGREGATED }; +extern void to_json(json& j, const ColumnType& e); +extern void from_json(const json& j, ColumnType& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { struct Aggregation { CallExpression call = {}; std::shared_ptr> filter = {}; @@ -1007,116 +1018,29 @@ struct HiveColumnHandle : public ColumnHandle { void to_json(json& j, const HiveColumnHandle& p); void from_json(const json& j, HiveColumnHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class BucketFunctionType { HIVE_COMPATIBLE, PRESTO_NATIVE }; -extern void to_json(json& j, const BucketFunctionType& e); -extern void from_json(const json& j, BucketFunctionType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class Order { ASCENDING, DESCENDING }; -extern void to_json(json& j, const Order& e); -extern void from_json(const json& j, Order& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct SortingColumn { - String columnName = {}; - Order order = {}; -}; -void to_json(json& j, const SortingColumn& p); -void from_json(const json& j, SortingColumn& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HiveBucketProperty { - List bucketedBy = {}; - int bucketCount = {}; - List sortedBy = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> types = {}; -}; -void to_json(json& j, const HiveBucketProperty& p); -void from_json(const json& j, HiveBucketProperty& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct DwrfEncryptionMetadata { - Map fieldToKeyData = {}; - Map extraMetadata = {}; - String encryptionAlgorithm = {}; - String encryptionProvider = {}; -}; -void to_json(json& j, const DwrfEncryptionMetadata& p); -void from_json(const json& j, DwrfEncryptionMetadata& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct EncryptionInformation { - std::shared_ptr dwrfEncryptionMetadata = {}; -}; -void to_json(json& j, const EncryptionInformation& p); -void from_json(const json& j, EncryptionInformation& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class HiveCompressionCodec { NONE, SNAPPY, GZIP, LZ4, ZSTD }; -extern void to_json(json& j, const HiveCompressionCodec& e); -extern void from_json(const json& j, HiveCompressionCodec& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -namespace facebook::presto::protocol { - -enum class HiveStorageFormat { - ORC, - DWRF, - PARQUET, - AVRO, - RCBINARY, - RCTEXT, - SEQUENCEFILE, - JSON, - TEXTFILE, - CSV, - PAGEFILE -}; - -void to_json(json& j, const HiveStorageFormat& p); -void from_json(const json& j, HiveStorageFormat& p); - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class WriteMode { - STAGE_AND_MOVE_TO_TARGET_DIRECTORY, - DIRECT_TO_TARGET_NEW_DIRECTORY, - DIRECT_TO_TARGET_EXISTING_DIRECTORY -}; -extern void to_json(json& j, const WriteMode& e); -extern void from_json(const json& j, WriteMode& e); -} // namespace facebook::presto::protocol + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -enum class TableType { NEW, EXISTING, TEMPORARY }; -extern void to_json(json& j, const TableType& e); -extern void from_json(const json& j, TableType& e); +struct OperatorInfo {}; +void to_json(json& j, const OperatorInfo& p); +void from_json(const json& j, OperatorInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct LocationHandle { - String targetPath = {}; - String writePath = {}; - std::shared_ptr tempPath = {}; - TableType tableType = {}; - WriteMode writeMode = {}; -}; -void to_json(json& j, const LocationHandle& p); -void from_json(const json& j, LocationHandle& p); +enum class BlockedReason { WAITING_FOR_MEMORY }; +extern void to_json(json& j, const BlockedReason& e); +extern void from_json(const json& j, BlockedReason& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1134,76 +1058,11 @@ void from_json(const json& j, LocationHandle& p); namespace facebook::presto::protocol { -struct Column { - String name; - String type; - - Column() = default; - explicit Column(const String& str) { - name = str; - } -}; +std::ostream& operator<<(std::ostream& os, const Duration& d); -void to_json(json& j, const Column& p); -void from_json(const json& j, Column& p); +void to_json(json& j, const Duration& p); +void from_json(const json& j, Duration& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct StorageFormat { - String serDe = {}; - String inputFormat = {}; - String outputFormat = {}; -}; -void to_json(json& j, const StorageFormat& p); -void from_json(const json& j, StorageFormat& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Storage { - StorageFormat storageFormat = {}; - String location = {}; - std::shared_ptr bucketProperty = {}; - bool skewed = {}; - Map serdeParameters = {}; - Map parameters = {}; -}; -void to_json(json& j, const Storage& p); -void from_json(const json& j, Storage& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class PrestoTableType { - MANAGED_TABLE, - EXTERNAL_TABLE, - VIRTUAL_VIEW, - MATERIALIZED_VIEW, - TEMPORARY_TABLE, - OTHER -}; -extern void to_json(json& j, const PrestoTableType& e); -extern void from_json(const json& j, PrestoTableType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Table { - String databaseName = {}; - String tableName = {}; - String owner = {}; - PrestoTableType tableType = {}; - Storage storage = {}; - List dataColumns = {}; - List partitionColumns = {}; - Map parameters = {}; - std::shared_ptr viewOriginalText = {}; - std::shared_ptr viewExpandedText = {}; -}; -void to_json(json& j, const Table& p); -void from_json(const json& j, Table& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct SchemaTableName { - String schema = {}; - String table = {}; -}; -void to_json(json& j, const SchemaTableName& p); -void from_json(const json& j, SchemaTableName& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1218,87 +1077,137 @@ void from_json(const json& j, SchemaTableName& p); * See the License for the specific language governing permissions and * limitations under the License. */ - -// dependency Table -// dependency SchemaTableName - namespace facebook::presto::protocol { -struct HivePageSinkMetadata { - SchemaTableName schemaTableName = {}; - std::shared_ptr table = {}; - // TODO Add modifiedPartitions -}; -void to_json(json& j, const HivePageSinkMetadata& p); -void from_json(const json& j, HivePageSinkMetadata& p); +std::ostream& operator<<(std::ostream& os, const DataSize& d); + +void to_json(nlohmann::json& j, const DataSize& p); +void from_json(const nlohmann::json& j, DataSize& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveInsertTableHandle : public ConnectorInsertTableHandle { - String schemaName = {}; - String tableName = {}; - List inputColumns = {}; - HivePageSinkMetadata pageSinkMetadata = {}; - LocationHandle locationHandle = {}; - std::shared_ptr bucketProperty = {}; - List preferredOrderingColumns = {}; - HiveStorageFormat tableStorageFormat = {}; - HiveStorageFormat partitionStorageFormat = {}; - HiveStorageFormat actualStorageFormat = {}; - HiveCompressionCodec compressionCodec = {}; - std::shared_ptr encryptionInformation = {}; - - HiveInsertTableHandle() noexcept; +struct OperatorStats { + int stageId = {}; + int stageExecutionId = {}; + int pipelineId = {}; + int operatorId = {}; + PlanNodeId planNodeId = {}; + String operatorType = {}; + int64_t totalDrivers = {}; + int64_t addInputCalls = {}; + Duration addInputWall = {}; + Duration addInputCpu = {}; + DataSize addInputAllocation = {}; + DataSize rawInputDataSize = {}; + int64_t rawInputPositions = {}; + DataSize inputDataSize = {}; + int64_t inputPositions = {}; + double sumSquaredInputPositions = {}; + int64_t getOutputCalls = {}; + Duration getOutputWall = {}; + Duration getOutputCpu = {}; + DataSize getOutputAllocation = {}; + DataSize outputDataSize = {}; + int64_t outputPositions = {}; + DataSize physicalWrittenDataSize = {}; + Duration additionalCpu = {}; + Duration blockedWall = {}; + int64_t finishCalls = {}; + Duration finishWall = {}; + Duration finishCpu = {}; + DataSize finishAllocation = {}; + DataSize userMemoryReservation = {}; + DataSize revocableMemoryReservation = {}; + DataSize systemMemoryReservation = {}; + DataSize peakUserMemoryReservation = {}; + DataSize peakSystemMemoryReservation = {}; + DataSize peakTotalMemoryReservation = {}; + DataSize spilledDataSize = {}; + std::shared_ptr blockedReason = {}; + OperatorInfo info = {}; + RuntimeStats runtimeStats = {}; }; -void to_json(json& j, const HiveInsertTableHandle& p); -void from_json(const json& j, HiveInsertTableHandle& p); +void to_json(json& j, const OperatorStats& p); +void from_json(const json& j, OperatorStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct NodeVersion { - String version = {}; +enum class BufferState { + OPEN, + NO_MORE_BUFFERS, + NO_MORE_PAGES, + FLUSHING, + FINISHED, + FAILED }; -void to_json(json& j, const NodeVersion& p); -void from_json(const json& j, NodeVersion& p); +extern void to_json(json& j, const BufferState& e); +extern void from_json(const json& j, BufferState& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ServerInfo { - NodeVersion nodeVersion = {}; - String environment = {}; - bool coordinator = {}; - bool starting = {}; - std::shared_ptr uptime = {}; +struct PageBufferInfo { + int partition = {}; + int64_t bufferedPages = {}; + int64_t bufferedBytes = {}; + int64_t rowsAdded = {}; + int64_t pagesAdded = {}; }; -void to_json(json& j, const ServerInfo& p); -void from_json(const json& j, ServerInfo& p); +void to_json(json& j, const PageBufferInfo& p); +void from_json(const json& j, PageBufferInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class Step { SINGLE, PARTIAL, FINAL }; -extern void to_json(json& j, const Step& e); -extern void from_json(const json& j, Step& e); +struct BufferInfo { + OutputBufferId bufferId = {}; + bool finished = {}; + int bufferedPages = {}; + int64_t pagesSent = {}; + PageBufferInfo pageBufferInfo = {}; +}; +void to_json(json& j, const BufferInfo& p); +void from_json(const json& j, BufferInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TopNNode : public PlanNode { - std::shared_ptr source = {}; - int64_t count = {}; - OrderingScheme orderingScheme = {}; - Step step = {}; - - TopNNode() noexcept; +struct OutputBufferInfo { + String type = {}; + BufferState state = {}; + bool canAddBuffers = {}; + bool canAddPages = {}; + int64_t totalBufferedBytes = {}; + int64_t totalBufferedPages = {}; + int64_t totalRowsSent = {}; + int64_t totalPagesSent = {}; + List buffers = {}; }; -void to_json(json& j, const TopNNode& p); -void from_json(const json& j, TopNNode& p); +void to_json(json& j, const OutputBufferInfo& p); +void from_json(const json& j, OutputBufferInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { - UUID requestId = {}; - SchemaTableName schemaTableName = {}; - std::shared_ptr partitionName = {}; - std::shared_ptr fileName = {}; - - HiveMetadataUpdateHandle() noexcept; +struct DriverStats { + Lifespan lifespan = {}; + DateTime createTime = {}; + DateTime startTime = {}; + DateTime endTime = {}; + Duration queuedTime = {}; + Duration elapsedTime = {}; + DataSize userMemoryReservation = {}; + DataSize revocableMemoryReservation = {}; + DataSize systemMemoryReservation = {}; + Duration totalScheduledTime = {}; + Duration totalCpuTime = {}; + Duration totalBlockedTime = {}; + bool fullyBlocked = {}; + List blockedReasons = {}; + DataSize totalAllocation = {}; + DataSize rawInputDataSize = {}; + int64_t rawInputPositions = {}; + Duration rawInputReadTime = {}; + DataSize processedInputDataSize = {}; + int64_t processedInputPositions = {}; + DataSize outputDataSize = {}; + int64_t outputPositions = {}; + DataSize physicalWrittenDataSize = {}; + List operatorStats = {}; }; -void to_json(json& j, const HiveMetadataUpdateHandle& p); -void from_json(const json& j, HiveMetadataUpdateHandle& p); +void to_json(json& j, const DriverStats& p); +void from_json(const json& j, DriverStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct DistributionSnapshot { @@ -1410,399 +1319,254 @@ void to_json(json& j, const TaskStats& p); void from_json(const json& j, TaskStats& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableHandle { +struct MetadataUpdates { ConnectorId connectorId = {}; - std::shared_ptr connectorHandle = {}; - std::shared_ptr transaction = {}; - std::shared_ptr connectorTableLayout = {}; + List> metadataUpdates = {}; }; -void to_json(json& j, const TableHandle& p); -void from_json(const json& j, TableHandle& p); +void to_json(json& j, const MetadataUpdates& p); +void from_json(const json& j, MetadataUpdates& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DeleteHandle : public ExecutionWriterTarget { - TableHandle handle = {}; - SchemaTableName schemaTableName = {}; - - DeleteHandle() noexcept; +struct TaskInfo { + TaskId taskId = {}; + TaskStatus taskStatus = {}; + DateTime lastHeartbeat = {}; + OutputBufferInfo outputBuffers = {}; + List noMoreSplits = {}; + TaskStats stats = {}; + bool needsPlan = {}; + MetadataUpdates metadataUpdates = {}; + String nodeId = {}; }; -void to_json(json& j, const DeleteHandle& p); -void from_json(const json& j, DeleteHandle& p); +void to_json(json& j, const TaskInfo& p); +void from_json(const json& j, TaskInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class ExchangeNodeType { - GATHER, - REPARTITION, - REPLICATE, +struct RemoteTransactionHandle : public ConnectorTransactionHandle { + std::shared_ptr dummy = {}; + + RemoteTransactionHandle() noexcept; }; -extern void to_json(json& j, const ExchangeNodeType& e); -extern void from_json(const json& j, ExchangeNodeType& e); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class ExchangeNodeScope { LOCAL, REMOTE_STREAMING, REMOTE_MATERIALIZED }; -extern void to_json(json& j, const ExchangeNodeScope& e); -extern void from_json(const json& j, ExchangeNodeScope& e); +void to_json(json& j, const RemoteTransactionHandle& p); +void from_json(const json& j, RemoteTransactionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PartitioningHandle { - std::shared_ptr connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; +struct TableHandle { + ConnectorId connectorId = {}; + std::shared_ptr connectorHandle = {}; + std::shared_ptr transaction = {}; + std::shared_ptr connectorTableLayout = {}; }; -void to_json(json& j, const PartitioningHandle& p); -void from_json(const json& j, PartitioningHandle& p); +void to_json(json& j, const TableHandle& p); +void from_json(const json& j, TableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Partitioning { - PartitioningHandle handle = {}; - List> arguments = {}; +struct DeleteScanInfo { + PlanNodeId id = {}; + TableHandle tableHandle = {}; }; -void to_json(json& j, const Partitioning& p); -void from_json(const json& j, Partitioning& p); +void to_json(json& j, const DeleteScanInfo& p); +void from_json(const json& j, DeleteScanInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PartitioningScheme { - Partitioning partitioning = {}; - List outputLayout = {}; - std::shared_ptr hashColumn = {}; - bool replicateNullsAndAny = {}; - std::shared_ptr> bucketToPartition = {}; +struct Language { + String language = {}; }; -void to_json(json& j, const PartitioningScheme& p); -void from_json(const json& j, PartitioningScheme& p); +void to_json(json& j, const Language& p); +void from_json(const json& j, Language& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ExchangeNode : public PlanNode { - ExchangeNodeType type = {}; - ExchangeNodeScope scope = {}; - PartitioningScheme partitioningScheme = {}; - List> sources = {}; - List> inputs = {}; - bool ensureSourceOrdering = {}; - std::shared_ptr orderingScheme = {}; - - ExchangeNode() noexcept; -}; -void to_json(json& j, const ExchangeNode& p); -void from_json(const json& j, ExchangeNode& p); +enum class NullCallClause { RETURNS_NULL_ON_NULL_INPUT, CALLED_ON_NULL_INPUT }; +extern void to_json(json& j, const NullCallClause& e); +extern void from_json(const json& j, NullCallClause& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RemoteSourceNode : public PlanNode { - List sourceFragmentIds = {}; - List outputVariables = {}; - bool ensureSourceOrdering = {}; - std::shared_ptr orderingScheme = {}; - ExchangeNodeType exchangeType = {}; - - RemoteSourceNode() noexcept; +enum class Determinism { + DETERMINISTIC, + NOT_DETERMINISTIC, }; -void to_json(json& j, const RemoteSourceNode& p); -void from_json(const json& j, RemoteSourceNode& p); +extern void to_json(json& j, const Determinism& e); +extern void from_json(const json& j, Determinism& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct FilterNode : public PlanNode { - std::shared_ptr source = {}; - std::shared_ptr predicate = {}; - - FilterNode() noexcept; +struct RoutineCharacteristics { + std::shared_ptr language = {}; + std::shared_ptr determinism = {}; + std::shared_ptr nullCallClause = {}; }; -void to_json(json& j, const FilterNode& p); -void from_json(const json& j, FilterNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class DistributionType { PARTITIONED, REPLICATED }; -extern void to_json(json& j, const DistributionType& e); -extern void from_json(const json& j, DistributionType& e); +void to_json(json& j, const RoutineCharacteristics& p); +void from_json(const json& j, RoutineCharacteristics& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class JoinNodeType { INNER, LEFT, RIGHT, FULL }; -extern void to_json(json& j, const JoinNodeType& e); -extern void from_json(const json& j, JoinNodeType& e); +struct MemoryAllocation { + String tag = {}; + int64_t allocation = {}; +}; +void to_json(json& j, const MemoryAllocation& p); +void from_json(const json& j, MemoryAllocation& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct JoinNode : public PlanNode { - JoinNodeType type = {}; - std::shared_ptr left = {}; - std::shared_ptr right = {}; - List criteria = {}; - List outputVariables = {}; - std::shared_ptr> filter = {}; - std::shared_ptr leftHashVariable = {}; - std::shared_ptr rightHashVariable = {}; - std::shared_ptr distributionType = {}; - Map dynamicFilters = {}; - - JoinNode() noexcept; +struct MemoryPoolInfo { + int64_t maxBytes = {}; + int64_t reservedBytes = {}; + int64_t reservedRevocableBytes = {}; + Map queryMemoryReservations = {}; + Map> queryMemoryAllocations = {}; + Map queryMemoryRevocableReservations = {}; }; -void to_json(json& j, const JoinNode& p); -void from_json(const json& j, JoinNode& p); +void to_json(json& j, const MemoryPoolInfo& p); +void from_json(const json& j, MemoryPoolInfo& p); } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -struct MergeJoinNode : public PlanNode { - MergeJoinNode() noexcept; - PlanNodeId id = {}; - // JoinNodeType is referenced as JoinNode.Type in Presto - // Since presto_cpp codegen can't nicely handle inner class references - // So a special hard-coded template is required here - JoinNodeType type = {}; - std::shared_ptr left = {}; - std::shared_ptr right = {}; - // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto - List criteria = {}; - List outputVariables = {}; - std::shared_ptr> filter = {}; - std::shared_ptr leftHashVariable = {}; - std::shared_ptr rightHashVariable = {}; +struct MemoryInfo { + DataSize totalNodeMemory = {}; + Map pools = {}; }; -void to_json(json& j, const MergeJoinNode& p); -void from_json(const json& j, MergeJoinNode& p); +void to_json(json& j, const MemoryInfo& p); +void from_json(const json& j, MemoryInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SystemPartitioning { - SINGLE, - FIXED, - SOURCE, - SCALED, - COORDINATOR_ONLY, - ARBITRARY +struct PlanCostEstimate { + double cpuCost = {}; + double maxMemory = {}; + double maxMemoryWhenOutputting = {}; + double networkCost = {}; }; -extern void to_json(json& j, const SystemPartitioning& e); -extern void from_json(const json& j, SystemPartitioning& e); +void to_json(json& j, const PlanCostEstimate& p); +void from_json(const json& j, PlanCostEstimate& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SystemPartitionFunction { - SINGLE, - HASH, - ROUND_ROBIN, - BROADCAST, - UNKNOWN +struct StatsAndCosts { + Map stats = {}; + Map costs = {}; }; -extern void to_json(json& j, const SystemPartitionFunction& e); -extern void from_json(const json& j, SystemPartitionFunction& e); +void to_json(json& j, const StatsAndCosts& p); +void from_json(const json& j, StatsAndCosts& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SystemPartitioningHandle : public ConnectorPartitioningHandle { - SystemPartitioning partitioning = {}; - SystemPartitionFunction function = {}; - - SystemPartitioningHandle() noexcept; +struct SchemaTableName { + String schema = {}; + String table = {}; }; -void to_json(json& j, const SystemPartitioningHandle& p); -void from_json(const json& j, SystemPartitioningHandle& p); +void to_json(json& j, const SchemaTableName& p); +void from_json(const json& j, SchemaTableName& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OutputTableHandle { +struct InsertTableHandle { ConnectorId connectorId = {}; std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const OutputTableHandle& p); -void from_json(const json& j, OutputTableHandle& p); +void to_json(json& j, const InsertTableHandle& p); +void from_json(const json& j, InsertTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct CreateHandle : public ExecutionWriterTarget { - OutputTableHandle handle = {}; +struct InsertHandle : public ExecutionWriterTarget { + InsertTableHandle handle = {}; SchemaTableName schemaTableName = {}; - CreateHandle() noexcept; + InsertHandle() noexcept; }; -void to_json(json& j, const CreateHandle& p); -void from_json(const json& j, CreateHandle& p); +void to_json(json& j, const InsertHandle& p); +void from_json(const json& j, InsertHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BucketConversion { - int tableBucketCount = {}; - int partitionBucketCount = {}; - List bucketColumnHandles = {}; -}; -void to_json(json& j, const BucketConversion& p); -void from_json(const json& j, BucketConversion& p); +enum class Order { ASCENDING, DESCENDING }; +extern void to_json(json& j, const Order& e); +extern void from_json(const json& j, Order& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveOutputTableHandle : public ConnectorOutputTableHandle { - String schemaName = {}; - String tableName = {}; - List inputColumns = {}; - HivePageSinkMetadata pageSinkMetadata = {}; - LocationHandle locationHandle = {}; - HiveStorageFormat tableStorageFormat = {}; - HiveStorageFormat partitionStorageFormat = {}; - HiveStorageFormat actualStorageFormat = {}; - HiveCompressionCodec compressionCodec = {}; - List partitionedBy = {}; - std::shared_ptr bucketProperty = {}; - List preferredOrderingColumns = {}; - String tableOwner = {}; - Map additionalTableParameters = {}; - std::shared_ptr encryptionInformation = {}; - - HiveOutputTableHandle() noexcept; +struct SortingColumn { + String columnName = {}; + Order order = {}; }; -void to_json(json& j, const HiveOutputTableHandle& p); -void from_json(const json& j, HiveOutputTableHandle& p); +void to_json(json& j, const SortingColumn& p); +void from_json(const json& j, SortingColumn& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AllOrNoneValueSet : public ValueSet { - Type type = {}; - bool all = {}; +struct TableScanNode : public PlanNode { + TableHandle table = {}; + List outputVariables = {}; + Map> assignments = + {}; - AllOrNoneValueSet() noexcept; + TableScanNode() noexcept; }; -void to_json(json& j, const AllOrNoneValueSet& p); -void from_json(const json& j, AllOrNoneValueSet& p); +void to_json(json& j, const TableScanNode& p); +void from_json(const json& j, TableScanNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DistinctLimitNode : public PlanNode { +struct FilterNode : public PlanNode { std::shared_ptr source = {}; - int64_t limit = {}; - bool partial = {}; - List distinctVariables = {}; - std::shared_ptr hashVariable = {}; + std::shared_ptr predicate = {}; - DistinctLimitNode() noexcept; -}; -void to_json(json& j, const DistinctLimitNode& p); -void from_json(const json& j, DistinctLimitNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct HiveBucketHandle { - List columns = {}; - int tableBucketCount = {}; - int readBucketCount = {}; + FilterNode() noexcept; }; -void to_json(json& j, const HiveBucketHandle& p); -void from_json(const json& j, HiveBucketHandle& p); +void to_json(json& j, const FilterNode& p); +void from_json(const json& j, FilterNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveTableLayoutHandle : public ConnectorTableLayoutHandle { - SchemaTableName schemaTableName = {}; - String tablePath = {}; - List partitionColumns = {}; - List dataColumns = {}; - Map tableParameters = {}; - TupleDomain domainPredicate = {}; - std::shared_ptr remainingPredicate = {}; - Map predicateColumns = {}; - TupleDomain> partitionColumnPredicate = {}; - std::shared_ptr bucketHandle = {}; - std::shared_ptr bucketFilter = {}; - bool pushdownFilterEnabled = {}; - String layoutString = {}; - std::shared_ptr> requestedColumns = {}; - bool partialAggregationsPushedDown = {}; - bool appendRowNumber = {}; - - HiveTableLayoutHandle() noexcept; +struct AnalyzeTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const HiveTableLayoutHandle& p); -void from_json(const json& j, HiveTableLayoutHandle& p); +void to_json(json& j, const AnalyzeTableHandle& p); +void from_json(const json& j, AnalyzeTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BufferType { - PARTITIONED, - BROADCAST, - ARBITRARY, - DISCARDING, - SPOOLING +struct TableWriteInfo { + std::shared_ptr writerTarget = {}; + std::shared_ptr analyzeTableHandle = {}; + std::shared_ptr deleteScanInfo = {}; }; -extern void to_json(json& j, const BufferType& e); -extern void from_json(const json& j, BufferType& e); +void to_json(json& j, const TableWriteInfo& p); +void from_json(const json& j, TableWriteInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OutputBuffers { - BufferType type = {}; - int64_t version = {}; - bool noMoreBufferIds = {}; - Map buffers = {}; +struct EnforceSingleRowNode : public PlanNode { + std::shared_ptr source = {}; + + EnforceSingleRowNode() noexcept; }; -void to_json(json& j, const OutputBuffers& p); -void from_json(const json& j, OutputBuffers& p); +void to_json(json& j, const EnforceSingleRowNode& p); +void from_json(const json& j, EnforceSingleRowNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Assignments { - Map> assignments = - {}; +struct EquiJoinClause { + VariableReferenceExpression left = {}; + VariableReferenceExpression right = {}; }; -void to_json(json& j, const Assignments& p); -void from_json(const json& j, Assignments& p); +void to_json(json& j, const EquiJoinClause& p); +void from_json(const json& j, EquiJoinClause& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveTransactionHandle : public ConnectorTransactionHandle { - UUID uuid = {}; - - HiveTransactionHandle() noexcept; -}; -void to_json(json& j, const HiveTransactionHandle& p); -void from_json(const json& j, HiveTransactionHandle& p); +enum class DistributionType { PARTITIONED, REPLICATED }; +extern void to_json(json& j, const DistributionType& e); +extern void from_json(const json& j, DistributionType& e); } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { -struct TableWriterNode : public PlanNode { - std::shared_ptr source = {}; - // TODO Add target - VariableReferenceExpression rowCountVariable = {}; - VariableReferenceExpression fragmentVariable = {}; - VariableReferenceExpression tableCommitContextVariable = {}; - List columns = {}; - List columnNames = {}; - List notNullColumnVariables = {}; - std::shared_ptr partitioningScheme = {}; - std::shared_ptr preferredShufflePartitioningScheme = {}; - // TODO Add statisticsAggregation - - TableWriterNode() noexcept; -}; -void to_json(json& j, const TableWriterNode& p); -void from_json(const json& j, TableWriterNode& p); +enum class JoinNodeType { INNER, LEFT, RIGHT, FULL }; +extern void to_json(json& j, const JoinNodeType& e); +extern void from_json(const json& j, JoinNodeType& e); } // namespace facebook::presto::protocol -/* - * Licensed under the Apache License, Version 2.0 (the "License"); - * you may not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ namespace facebook::presto::protocol { +struct JoinNode : public PlanNode { + JoinNodeType type = {}; + std::shared_ptr left = {}; + std::shared_ptr right = {}; + List criteria = {}; + List outputVariables = {}; + std::shared_ptr> filter = {}; + std::shared_ptr leftHashVariable = {}; + std::shared_ptr rightHashVariable = {}; + std::shared_ptr distributionType = {}; + Map dynamicFilters = {}; -struct Block { - std::string data; + JoinNode() noexcept; }; - -void to_json(json& j, const Block& p); - -void from_json(const json& j, Block& p); - +void to_json(json& j, const JoinNode& p); +void from_json(const json& j, JoinNode& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1818,62 +1582,55 @@ void from_json(const json& j, Block& p); * limitations under the License. */ namespace facebook::presto::protocol { - -class ValueEntry { - public: - Type type; - std::shared_ptr block; -}; - -void to_json(json& j, const ValueEntry& p); -void from_json(const json& j, ValueEntry& p); - -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct EquatableValueSet : public ValueSet { - Type type = {}; - bool whiteList = {}; - List entries = {}; - - EquatableValueSet() noexcept; +struct MergeJoinNode : public PlanNode { + MergeJoinNode() noexcept; + PlanNodeId id = {}; + // JoinNodeType is referenced as JoinNode.Type in Presto + // Since presto_cpp codegen can't nicely handle inner class references + // So a special hard-coded template is required here + JoinNodeType type = {}; + std::shared_ptr left = {}; + std::shared_ptr right = {}; + // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto + List criteria = {}; + List outputVariables = {}; + std::shared_ptr> filter = {}; + std::shared_ptr leftHashVariable = {}; + std::shared_ptr rightHashVariable = {}; }; -void to_json(json& j, const EquatableValueSet& p); -void from_json(const json& j, EquatableValueSet& p); +void to_json(json& j, const MergeJoinNode& p); +void from_json(const json& j, MergeJoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PageBufferInfo { - int partition = {}; - int64_t bufferedPages = {}; - int64_t bufferedBytes = {}; - int64_t rowsAdded = {}; - int64_t pagesAdded = {}; +struct BucketConversion { + int tableBucketCount = {}; + int partitionBucketCount = {}; + List bucketColumnHandles = {}; }; -void to_json(json& j, const PageBufferInfo& p); -void from_json(const json& j, PageBufferInfo& p); +void to_json(json& j, const BucketConversion& p); +void from_json(const json& j, BucketConversion& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BufferInfo { - OutputBufferId bufferId = {}; - bool finished = {}; - int bufferedPages = {}; - int64_t pagesSent = {}; - PageBufferInfo pageBufferInfo = {}; -}; -void to_json(json& j, const BufferInfo& p); -void from_json(const json& j, BufferInfo& p); +enum class CacheQuotaScope { GLOBAL, SCHEMA, TABLE, PARTITION }; +extern void to_json(json& j, const CacheQuotaScope& e); +extern void from_json(const json& j, CacheQuotaScope& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class FunctionKind { SCALAR, AGGREGATE, WINDOW }; -extern void to_json(json& j, const FunctionKind& e); -extern void from_json(const json& j, FunctionKind& e); +struct CacheQuotaRequirement { + CacheQuotaScope cacheQuotaScope = {}; + std::shared_ptr quota = {}; +}; +void to_json(json& j, const CacheQuotaRequirement& p); +void from_json(const json& j, CacheQuotaRequirement& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct LongVariableConstraint { - String name = {}; - String expression = {}; +enum class NodeSelectionStrategy { + HARD_AFFINITY, + SOFT_AFFINITY, + NO_PREFERENCE }; -void to_json(json& j, const LongVariableConstraint& p); -void from_json(const json& j, LongVariableConstraint& p); +extern void to_json(json& j, const NodeSelectionStrategy& e); +extern void from_json(const json& j, NodeSelectionStrategy& e); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -1888,274 +1645,374 @@ void from_json(const json& j, LongVariableConstraint& p); * See the License for the specific language governing permissions and * limitations under the License. */ + namespace facebook::presto::protocol { -struct TypeVariableConstraint { - String name = {}; - bool comparableRequired = {}; - bool orderableRequired = {}; - String variadicBound = {}; - bool nonDecimalNumericRequired = {}; - String boundedBy = {}; -}; -void to_json(json& j, const TypeVariableConstraint& p); -void from_json(const json& j, TypeVariableConstraint& p); +struct Column { + String name; + String type; -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Signature { - QualifiedObjectName name = {}; - FunctionKind kind = {}; - List typeVariableConstraints = {}; - List longVariableConstraints = {}; - TypeSignature returnType = {}; - List argumentTypes = {}; - bool variableArity = {}; + Column() = default; + explicit Column(const String& str) { + name = str; + } }; -void to_json(json& j, const Signature& p); -void from_json(const json& j, Signature& p); + +void to_json(json& j, const Column& p); +void from_json(const json& j, Column& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct BuiltInFunctionHandle : public FunctionHandle { - Signature signature = {}; - - BuiltInFunctionHandle() noexcept; +struct TableToPartitionMapping { + std::shared_ptr> tableToPartitionColumns = {}; + Map partitionSchemaDifference = {}; }; -void to_json(json& j, const BuiltInFunctionHandle& p); -void from_json(const json& j, BuiltInFunctionHandle& p); +void to_json(json& j, const TableToPartitionMapping& p); +void from_json(const json& j, TableToPartitionMapping& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct LambdaDefinitionExpression : public RowExpression { - List argumentTypes = {}; - List arguments = {}; - std::shared_ptr body = {}; - - LambdaDefinitionExpression() noexcept; +struct HiveFileSplit { + String path = {}; + int64_t start = {}; + int64_t length = {}; + int64_t fileSize = {}; + int64_t fileModifiedTime = {}; + std::shared_ptr extraFileInfo = {}; + Map customSplitInfo = {}; }; -void to_json(json& j, const LambdaDefinitionExpression& p); -void from_json(const json& j, LambdaDefinitionExpression& p); +void to_json(json& j, const HiveFileSplit& p); +void from_json(const json& j, HiveFileSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BoundType { - UNBOUNDED_PRECEDING, - PRECEDING, - CURRENT_ROW, - FOLLOWING, - UNBOUNDED_FOLLOWING +struct HivePartitionKey { + String name = {}; + std::shared_ptr value = {}; }; -extern void to_json(json& j, const BoundType& e); -extern void from_json(const json& j, BoundType& e); +void to_json(json& j, const HivePartitionKey& p); +void from_json(const json& j, HivePartitionKey& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class WindowType { - RANGE, - ROWS, +struct StorageFormat { + String serDe = {}; + String inputFormat = {}; + String outputFormat = {}; }; -extern void to_json(json& j, const WindowType& e); -extern void from_json(const json& j, WindowType& e); +void to_json(json& j, const StorageFormat& p); +void from_json(const json& j, StorageFormat& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Frame { - WindowType type = {}; - BoundType startType = {}; - std::shared_ptr startValue = {}; - BoundType endType = {}; - std::shared_ptr endValue = {}; - std::shared_ptr originalStartValue = {}; - std::shared_ptr originalEndValue = {}; -}; -void to_json(json& j, const Frame& p); -void from_json(const json& j, Frame& p); +enum class BucketFunctionType { HIVE_COMPATIBLE, PRESTO_NATIVE }; +extern void to_json(json& j, const BucketFunctionType& e); +extern void from_json(const json& j, BucketFunctionType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Function { - CallExpression functionCall = {}; - Frame frame = {}; - bool ignoreNulls = {}; +struct HiveBucketProperty { + List bucketedBy = {}; + int bucketCount = {}; + List sortedBy = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> types = {}; }; -void to_json(json& j, const Function& p); -void from_json(const json& j, Function& p); +void to_json(json& j, const HiveBucketProperty& p); +void from_json(const json& j, HiveBucketProperty& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct VariableStatsEstimate { - double lowValue = {}; - double highValue = {}; - double nullsFraction = {}; - double averageRowSize = {}; - double distinctValuesCount = {}; +struct Storage { + StorageFormat storageFormat = {}; + String location = {}; + std::shared_ptr bucketProperty = {}; + bool skewed = {}; + Map serdeParameters = {}; + Map parameters = {}; }; -void to_json(json& j, const VariableStatsEstimate& p); -void from_json(const json& j, VariableStatsEstimate& p); +void to_json(json& j, const Storage& p); +void from_json(const json& j, Storage& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PlanNodeStatsEstimate { - double outputRowCount = {}; - double totalSize = {}; - bool confident = {}; - Map variableStatistics = - {}; +struct HiveSplit : public ConnectorSplit { + HiveFileSplit fileSplit = {}; + String database = {}; + String table = {}; + String partitionName = {}; + Storage storage = {}; + List partitionKeys = {}; + List addresses = {}; + std::shared_ptr readBucketNumber = {}; + std::shared_ptr tableBucketNumber = {}; + NodeSelectionStrategy nodeSelectionStrategy = {}; + int partitionDataColumnCount = {}; + TableToPartitionMapping tableToPartitionMapping = {}; + std::shared_ptr bucketConversion = {}; + bool s3SelectPushdownEnabled = {}; + CacheQuotaRequirement cacheQuota = {}; + std::shared_ptr encryptionMetadata = {}; + List> redundantColumnDomains = {}; + SplitWeight splitWeight = {}; + + HiveSplit() noexcept; }; -void to_json(json& j, const PlanNodeStatsEstimate& p); -void from_json(const json& j, PlanNodeStatsEstimate& p); +void to_json(json& j, const HiveSplit& p); +void from_json(const json& j, HiveSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PlanCostEstimate { - double cpuCost = {}; - double maxMemory = {}; - double maxMemoryWhenOutputting = {}; - double networkCost = {}; +struct AssignUniqueId : public PlanNode { + std::shared_ptr source = {}; + VariableReferenceExpression idVariable = {}; + + AssignUniqueId() noexcept; }; -void to_json(json& j, const PlanCostEstimate& p); -void from_json(const json& j, PlanCostEstimate& p); +void to_json(json& j, const AssignUniqueId& p); +void from_json(const json& j, AssignUniqueId& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchColumnHandle is special since we require an implementation of +// operator<(). + namespace facebook::presto::protocol { -struct StatsAndCosts { - Map stats = {}; - Map costs = {}; +struct TpchColumnHandle : public ColumnHandle { + String columnName = {}; + Type type = {}; + + TpchColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return columnName < dynamic_cast(o).columnName; + } }; -void to_json(json& j, const StatsAndCosts& p); -void from_json(const json& j, StatsAndCosts& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -enum class Bound { BELOW, EXACTLY, ABOVE }; -extern void to_json(json& j, const Bound& e); -extern void from_json(const json& j, Bound& e); +void to_json(json& j, const TpchColumnHandle& p); +void from_json(const json& j, TpchColumnHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Marker { - Type type = {}; - std::shared_ptr valueBlock = {}; - Bound bound = {}; +enum class PrestoTableType { + MANAGED_TABLE, + EXTERNAL_TABLE, + VIRTUAL_VIEW, + MATERIALIZED_VIEW, + TEMPORARY_TABLE, + OTHER }; -void to_json(json& j, const Marker& p); -void from_json(const json& j, Marker& p); +extern void to_json(json& j, const PrestoTableType& e); +extern void from_json(const json& j, PrestoTableType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableToPartitionMapping { - std::shared_ptr> tableToPartitionColumns = {}; - Map partitionSchemaDifference = {}; +struct Table { + String databaseName = {}; + String tableName = {}; + String owner = {}; + PrestoTableType tableType = {}; + Storage storage = {}; + List dataColumns = {}; + List partitionColumns = {}; + Map parameters = {}; + std::shared_ptr viewOriginalText = {}; + std::shared_ptr viewExpandedText = {}; }; -void to_json(json& j, const TableToPartitionMapping& p); -void from_json(const json& j, TableToPartitionMapping& p); +void to_json(json& j, const Table& p); +void from_json(const json& j, Table& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// dependency Table +// dependency SchemaTableName + namespace facebook::presto::protocol { -struct TableScanNode : public PlanNode { - TableHandle table = {}; - List outputVariables = {}; - Map> assignments = - {}; - TableScanNode() noexcept; +struct HivePageSinkMetadata { + SchemaTableName schemaTableName = {}; + std::shared_ptr
table = {}; + // TODO Add modifiedPartitions }; -void to_json(json& j, const TableScanNode& p); -void from_json(const json& j, TableScanNode& p); +void to_json(json& j, const HivePageSinkMetadata& p); +void from_json(const json& j, HivePageSinkMetadata& p); + } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct EmptySplit : public ConnectorSplit { - ConnectorId connectorId = {}; +struct HiveMetadataUpdateHandle : public ConnectorMetadataUpdateHandle { + UUID requestId = {}; + SchemaTableName schemaTableName = {}; + std::shared_ptr partitionName = {}; + std::shared_ptr fileName = {}; - EmptySplit() noexcept; + HiveMetadataUpdateHandle() noexcept; }; -void to_json(json& j, const EmptySplit& p); -void from_json(const json& j, EmptySplit& p); +void to_json(json& j, const HiveMetadataUpdateHandle& p); +void from_json(const json& j, HiveMetadataUpdateHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ValuesNode : public PlanNode { - std::shared_ptr location = {}; - - List outputVariables = {}; - List>> rows = {}; - std::shared_ptr valuesNodeLabel = {}; - - ValuesNode() noexcept; +struct NodeVersion { + String version = {}; }; -void to_json(json& j, const ValuesNode& p); -void from_json(const json& j, ValuesNode& p); +void to_json(json& j, const NodeVersion& p); +void from_json(const json& j, NodeVersion& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class RuntimeUnit { NONE, NANO, BYTE }; -extern void to_json(json& j, const RuntimeUnit& e); -extern void from_json(const json& j, RuntimeUnit& e); +struct ServerInfo { + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + bool starting = {}; + std::shared_ptr uptime = {}; +}; +void to_json(json& j, const ServerInfo& p); +void from_json(const json& j, ServerInfo& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RuntimeMetric { - String name = {}; - RuntimeUnit unit = {}; - int64_t sum = {}; - int64_t count = {}; - int64_t max = {}; - int64_t min = {}; +struct OutputTableHandle { + ConnectorId connectorId = {}; + std::shared_ptr transactionHandle = {}; + std::shared_ptr connectorHandle = {}; }; -void to_json(json& j, const RuntimeMetric& p); -void from_json(const json& j, RuntimeMetric& p); +void to_json(json& j, const OutputTableHandle& p); +void from_json(const json& j, OutputTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AssignUniqueId : public PlanNode { - std::shared_ptr source = {}; - VariableReferenceExpression idVariable = {}; +struct CreateHandle : public ExecutionWriterTarget { + OutputTableHandle handle = {}; + SchemaTableName schemaTableName = {}; - AssignUniqueId() noexcept; + CreateHandle() noexcept; }; -void to_json(json& j, const AssignUniqueId& p); -void from_json(const json& j, AssignUniqueId& p); +void to_json(json& j, const CreateHandle& p); +void from_json(const json& j, CreateHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RemoteTransactionHandle : public ConnectorTransactionHandle { - std::shared_ptr dummy = {}; +struct NodeStatus { + String nodeId = {}; + NodeVersion nodeVersion = {}; + String environment = {}; + bool coordinator = {}; + Duration uptime = {}; + String externalAddress = {}; + String internalAddress = {}; + MemoryInfo memoryInfo = {}; + int processors = {}; + double processCpuLoad = {}; + double systemCpuLoad = {}; + int64_t heapUsed = {}; + int64_t heapAvailable = {}; + int64_t nonHeapUsed = {}; +}; +void to_json(json& j, const NodeStatus& p); +void from_json(const json& j, NodeStatus& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct HiveTableHandle : public ConnectorTableHandle { + String schemaName = {}; + String tableName = {}; + std::shared_ptr>> analyzePartitionValues = {}; - RemoteTransactionHandle() noexcept; + HiveTableHandle() noexcept; }; -void to_json(json& j, const RemoteTransactionHandle& p); -void from_json(const json& j, RemoteTransactionHandle& p); +void to_json(json& j, const HiveTableHandle& p); +void from_json(const json& j, HiveTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct InsertTableHandle { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; +struct HiveTransactionHandle : public ConnectorTransactionHandle { + UUID uuid = {}; + + HiveTransactionHandle() noexcept; }; -void to_json(json& j, const InsertTableHandle& p); -void from_json(const json& j, InsertTableHandle& p); +void to_json(json& j, const HiveTransactionHandle& p); +void from_json(const json& j, HiveTransactionHandle& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -struct RefreshMaterializedViewHandle { - InsertTableHandle handle = {}; - SchemaTableName schemaTableName = {}; +struct TableWriterNode : public PlanNode { + std::shared_ptr source = {}; + // TODO Add target + VariableReferenceExpression rowCountVariable = {}; + VariableReferenceExpression fragmentVariable = {}; + VariableReferenceExpression tableCommitContextVariable = {}; + List columns = {}; + List columnNames = {}; + List notNullColumnVariables = {}; + std::shared_ptr partitioningScheme = {}; + std::shared_ptr preferredShufflePartitioningScheme = {}; + // TODO Add statisticsAggregation + + TableWriterNode() noexcept; }; -void to_json(json& j, const RefreshMaterializedViewHandle& p); -void from_json(const json& j, RefreshMaterializedViewHandle& p); +void to_json(json& j, const TableWriterNode& p); +void from_json(const json& j, TableWriterNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveFileSplit { - String path = {}; - int64_t start = {}; - int64_t length = {}; - int64_t fileSize = {}; - int64_t fileModifiedTime = {}; - std::shared_ptr extraFileInfo = {}; - Map customSplitInfo = {}; +struct DistinctLimitNode : public PlanNode { + std::shared_ptr source = {}; + int64_t limit = {}; + bool partial = {}; + List distinctVariables = {}; + std::shared_ptr hashVariable = {}; + + DistinctLimitNode() noexcept; }; -void to_json(json& j, const HiveFileSplit& p); -void from_json(const json& j, HiveFileSplit& p); +void to_json(json& j, const DistinctLimitNode& p); +void from_json(const json& j, DistinctLimitNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class StageExecutionStrategy { - UNGROUPED_EXECUTION, - FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, - RECOVERABLE_GROUPED_EXECUTION +enum class Form { + IF, + NULL_IF, + SWITCH, + WHEN, + IS_NULL, + COALESCE, + IN, + AND, + OR, + DEREFERENCE, + ROW_CONSTRUCTOR, + BIND }; -extern void to_json(json& j, const StageExecutionStrategy& e); -extern void from_json(const json& j, StageExecutionStrategy& e); +extern void to_json(json& j, const Form& e); +extern void from_json(const json& j, Form& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct StageExecutionDescriptor { - StageExecutionStrategy stageExecutionStrategy = {}; - List groupedExecutionScanNodes = {}; - int totalLifespans = {}; +struct SpecialFormExpression : public RowExpression { + Form form = {}; + Type returnType = {}; + List> arguments = {}; + + SpecialFormExpression() noexcept; }; -void to_json(json& j, const StageExecutionDescriptor& p); -void from_json(const json& j, StageExecutionDescriptor& p); +void to_json(json& j, const SpecialFormExpression& p); +void from_json(const json& j, SpecialFormExpression& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2170,112 +2027,244 @@ void from_json(const json& j, StageExecutionDescriptor& p); * See the License for the specific language governing permissions and * limitations under the License. */ +namespace facebook::presto::protocol { + +class ValueEntry { + public: + Type type; + std::shared_ptr block; +}; +void to_json(json& j, const ValueEntry& p); +void from_json(const json& j, ValueEntry& p); + +} // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct PlanFragment { - PlanFragmentId id = {}; - std::shared_ptr root = {}; - List variables = {}; - PartitioningHandle partitioning = {}; - List tableScanSchedulingOrder = {}; - PartitioningScheme partitioningScheme = {}; - StageExecutionDescriptor stageExecutionDescriptor = {}; - bool outputTableWriterFragment = {}; - std::shared_ptr jsonRepresentation = {}; +struct EquatableValueSet : public ValueSet { + Type type = {}; + bool whiteList = {}; + List entries = {}; + + EquatableValueSet() noexcept; }; -void to_json(json& j, const PlanFragment& p); -void from_json(const json& j, PlanFragment& p); +void to_json(json& j, const EquatableValueSet& p); +void from_json(const json& j, EquatableValueSet& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class SelectedRoleType { ROLE, ALL, NONE }; -extern void to_json(json& j, const SelectedRoleType& e); -extern void from_json(const json& j, SelectedRoleType& e); +enum class AggregationNodeStep { PARTIAL, FINAL, INTERMEDIATE, SINGLE }; +extern void to_json(json& j, const AggregationNodeStep& e); +extern void from_json(const json& j, AggregationNodeStep& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SelectedRole { - SelectedRoleType type = {}; - std::shared_ptr role = {}; +struct GroupingSetDescriptor { + List groupingKeys = {}; + int groupingSetCount = {}; + List globalGroupingSets = {}; }; -void to_json(json& j, const SelectedRole& p); -void from_json(const json& j, SelectedRole& p); +void to_json(json& j, const GroupingSetDescriptor& p); +void from_json(const json& j, GroupingSetDescriptor& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Language { - String language = {}; +struct AggregationNode : public PlanNode { + std::shared_ptr source = {}; + Map aggregations = {}; + GroupingSetDescriptor groupingSets = {}; + List preGroupedVariables = {}; + AggregationNodeStep step = {}; + std::shared_ptr hashVariable = {}; + std::shared_ptr groupIdVariable = {}; + + AggregationNode() noexcept; }; -void to_json(json& j, const Language& p); -void from_json(const json& j, Language& p); +void to_json(json& j, const AggregationNode& p); +void from_json(const json& j, AggregationNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class Determinism { - DETERMINISTIC, - NOT_DETERMINISTIC, +struct ConstantExpression : public RowExpression { + Block valueBlock = {}; + Type type = {}; + + ConstantExpression() noexcept; }; -extern void to_json(json& j, const Determinism& e); -extern void from_json(const json& j, Determinism& e); +void to_json(json& j, const ConstantExpression& p); +void from_json(const json& j, ConstantExpression& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class NullCallClause { RETURNS_NULL_ON_NULL_INPUT, CALLED_ON_NULL_INPUT }; -extern void to_json(json& j, const NullCallClause& e); -extern void from_json(const json& j, NullCallClause& e); +struct Domain { + std::shared_ptr values = {}; + bool nullAllowed = {}; +}; +void to_json(json& j, const Domain& p); +void from_json(const json& j, Domain& p); } // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ namespace facebook::presto::protocol { -struct RoutineCharacteristics { - std::shared_ptr language = {}; - std::shared_ptr determinism = {}; - std::shared_ptr nullCallClause = {}; + +template +struct pointerDerefCompare { + bool operator()(const std::shared_ptr& a, const std::shared_ptr& b) + const { + return *a < *b; + } }; -void to_json(json& j, const RoutineCharacteristics& p); -void from_json(const json& j, RoutineCharacteristics& p); + +template +struct TupleDomain { + std::shared_ptr> domains; +}; + +template +struct TupleDomain> { + std::shared_ptr, Domain, pointerDerefCompare>> + domains; +}; + +template +struct ColumnDomain { + T column; + Domain domain; // dependency +}; + } // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct Parameter { - String name = {}; - TypeSignature type = {}; + +namespace nlohmann { + +template +struct adl_serializer> { + static void to_json( + json& j, + const facebook::presto::protocol::ColumnDomain& p) { + facebook::presto::protocol::to_json_key( + j, "column", p.column, "ColumnDomain", "T", "column"); + facebook::presto::protocol::to_json_key( + j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); + } + + static void from_json( + const json& j, + facebook::presto::protocol::ColumnDomain& p) { + facebook::presto::protocol::from_json_key( + j, "column", p.column, "ColumnDomain", "T", "column"); + facebook::presto::protocol::from_json_key( + j, "domain", p.domain, "ColumnDomain", "Domain", "domain"); + } +}; + +template +struct adl_serializer> { + static void to_json( + json& j, + const facebook::presto::protocol::TupleDomain& tup) { + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain> + list; + if (tup.domains != nullptr) { + for (auto& el : *tup.domains) { + facebook::presto::protocol::ColumnDomain domain; + domain.column = el.first; + domain.domain = el.second; + list.push_back(domain); + } + } + + j["columnDomains"] = list; + } + + static void from_json( + const json& j, + facebook::presto::protocol::TupleDomain& tup) { + if (j.count("columnDomains") != 0U) { + std::shared_ptr> + map = std::make_shared< + std::map>(); + + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain> + list = j.at("columnDomains"); + for (const facebook::presto::protocol::ColumnDomain& value : list) { + map->insert(std::make_pair(T(value.column), value.domain)); + } + tup.domains = map; + } + } +}; + +template +struct adl_serializer< + facebook::presto::protocol::TupleDomain>> { + static void to_json( + json& j, + const facebook::presto::protocol::TupleDomain>& tup) { + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain>> + list; + if (tup.domains != nullptr) { + for (auto& el : *tup.domains) { + facebook::presto::protocol::ColumnDomain> domain; + domain.column = el.first; + domain.domain = el.second; + list.push_back(domain); + } + } + + j["columnDomains"] = list; + } + + static void from_json( + const json& j, + facebook::presto::protocol::TupleDomain>& tup) { + if (j.count("columnDomains") != 0U) { + auto map = std::make_shared, + facebook::presto::protocol::Domain, + facebook::presto::protocol::pointerDerefCompare>>(); + + facebook::presto::protocol::List< + facebook::presto::protocol::ColumnDomain>> + list = j.at("columnDomains"); + for (const facebook::presto::protocol::ColumnDomain>& + value : list) { + map->insert( + std::make_pair(std::shared_ptr(value.column), value.domain)); + } + tup.domains = map; + } + } }; -void to_json(json& j, const Parameter& p); -void from_json(const json& j, Parameter& p); -} // namespace facebook::presto::protocol + +} // namespace nlohmann namespace facebook::presto::protocol { -struct SqlInvokedFunction { - List parameters = {}; - String description = {}; - RoutineCharacteristics routineCharacteristics = {}; - String body = {}; - Signature signature = {}; - SqlFunctionId functionId = {}; +struct TpchTableHandle : public ConnectorTableHandle { + String tableName = {}; + double scaleFactor = {}; + + TpchTableHandle() noexcept; }; -void to_json(json& j, const SqlInvokedFunction& p); -void from_json(const json& j, SqlInvokedFunction& p); +void to_json(json& j, const TpchTableHandle& p); +void from_json(const json& j, TpchTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SessionRepresentation { - String queryId = {}; - std::shared_ptr transactionId = {}; - bool clientTransactionSupport = {}; - String user = {}; - std::shared_ptr principal = {}; - std::shared_ptr source = {}; - std::shared_ptr catalog = {}; - std::shared_ptr schema = {}; - std::shared_ptr traceToken = {}; - TimeZoneKey timeZoneKey = {}; - Locale locale = {}; - std::shared_ptr remoteUserAddress = {}; - std::shared_ptr userAgent = {}; - std::shared_ptr clientInfo = {}; - List clientTags = {}; - ResourceEstimates resourceEstimates = {}; - int64_t startTime = {}; - Map systemProperties = {}; - Map> catalogProperties = {}; - Map> unprocessedCatalogProperties = {}; - Map roles = {}; - Map preparedStatements = {}; - Map sessionFunctions = {}; +struct TpchTableLayoutHandle : public ConnectorTableLayoutHandle { + TpchTableHandle table = {}; + TupleDomain> predicate = {}; + + TpchTableLayoutHandle() noexcept; }; -void to_json(json& j, const SessionRepresentation& p); -void from_json(const json& j, SessionRepresentation& p); +void to_json(json& j, const TpchTableLayoutHandle& p); +void from_json(const json& j, TpchTableLayoutHandle& p); } // namespace facebook::presto::protocol /* * Licensed under the Apache License, Version 2.0 (the "License"); @@ -2290,83 +2279,84 @@ void from_json(const json& j, SessionRepresentation& p); * See the License for the specific language governing permissions and * limitations under the License. */ -namespace facebook::presto::protocol { -struct ScheduledSplit { - long sequenceId = {}; - PlanNodeId planNodeId = {}; // dependency - Split split = {}; +namespace facebook::presto::protocol { - bool operator<(const ScheduledSplit& o) const { - return sequenceId < o.sequenceId; - } +enum class HiveStorageFormat { + ORC, + DWRF, + PARQUET, + AVRO, + RCBINARY, + RCTEXT, + SEQUENCEFILE, + JSON, + TEXTFILE, + CSV, + PAGEFILE }; -void to_json(json& j, const ScheduledSplit& p); -void from_json(const json& j, ScheduledSplit& p); +void to_json(json& j, const HiveStorageFormat& p); +void from_json(const json& j, HiveStorageFormat& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TaskSource { - PlanNodeId planNodeId = {}; - List splits = {}; - List noMoreSplitsForLifespan = {}; - bool noMoreSplits = {}; -}; -void to_json(json& j, const TaskSource& p); -void from_json(const json& j, TaskSource& p); +enum class HiveCompressionCodec { NONE, SNAPPY, GZIP, LZ4, ZSTD }; +extern void to_json(json& j, const HiveCompressionCodec& e); +extern void from_json(const json& j, HiveCompressionCodec& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct DeleteScanInfo { - PlanNodeId id = {}; - TableHandle tableHandle = {}; +enum class WriteMode { + STAGE_AND_MOVE_TO_TARGET_DIRECTORY, + DIRECT_TO_TARGET_NEW_DIRECTORY, + DIRECT_TO_TARGET_EXISTING_DIRECTORY }; -void to_json(json& j, const DeleteScanInfo& p); -void from_json(const json& j, DeleteScanInfo& p); +extern void to_json(json& j, const WriteMode& e); +extern void from_json(const json& j, WriteMode& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AnalyzeTableHandle { - ConnectorId connectorId = {}; - std::shared_ptr transactionHandle = {}; - std::shared_ptr connectorHandle = {}; -}; -void to_json(json& j, const AnalyzeTableHandle& p); -void from_json(const json& j, AnalyzeTableHandle& p); +enum class TableType { NEW, EXISTING, TEMPORARY }; +extern void to_json(json& j, const TableType& e); +extern void from_json(const json& j, TableType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TableWriteInfo { - std::shared_ptr writerTarget = {}; - std::shared_ptr analyzeTableHandle = {}; - std::shared_ptr deleteScanInfo = {}; +struct LocationHandle { + String targetPath = {}; + String writePath = {}; + std::shared_ptr tempPath = {}; + TableType tableType = {}; + WriteMode writeMode = {}; }; -void to_json(json& j, const TableWriteInfo& p); -void from_json(const json& j, TableWriteInfo& p); +void to_json(json& j, const LocationHandle& p); +void from_json(const json& j, LocationHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TaskUpdateRequest { - SessionRepresentation session = {}; - Map extraCredentials = {}; - std::shared_ptr fragment = {}; - List sources = {}; - OutputBuffers outputIds = {}; - std::shared_ptr tableWriteInfo = {}; +struct HiveInsertTableHandle : public ConnectorInsertTableHandle { + String schemaName = {}; + String tableName = {}; + List inputColumns = {}; + HivePageSinkMetadata pageSinkMetadata = {}; + LocationHandle locationHandle = {}; + std::shared_ptr bucketProperty = {}; + List preferredOrderingColumns = {}; + HiveStorageFormat tableStorageFormat = {}; + HiveStorageFormat partitionStorageFormat = {}; + HiveStorageFormat actualStorageFormat = {}; + HiveCompressionCodec compressionCodec = {}; + std::shared_ptr encryptionInformation = {}; + + HiveInsertTableHandle() noexcept; }; -void to_json(json& j, const TaskUpdateRequest& p); -void from_json(const json& j, TaskUpdateRequest& p); +void to_json(json& j, const HiveInsertTableHandle& p); +void from_json(const json& j, HiveInsertTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct GroupIdNode : public PlanNode { - std::shared_ptr source = {}; - List> groupingSets = {}; - Map - groupingColumns = {}; - List aggregationArguments = {}; - VariableReferenceExpression groupIdVariable = {}; - - GroupIdNode() noexcept; +struct Assignments { + Map> assignments = + {}; }; -void to_json(json& j, const GroupIdNode& p); -void from_json(const json& j, GroupIdNode& p); +void to_json(json& j, const Assignments& p); +void from_json(const json& j, Assignments& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { enum class Locality { UNKNOWN, LOCAL, REMOTE }; @@ -2385,409 +2375,399 @@ void to_json(json& j, const ProjectNode& p); void from_json(const json& j, ProjectNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class LimitNodeStep { PARTIAL, FINAL }; -extern void to_json(json& j, const LimitNodeStep& e); -extern void from_json(const json& j, LimitNodeStep& e); +enum class Step { SINGLE, PARTIAL, FINAL }; +extern void to_json(json& j, const Step& e); +extern void from_json(const json& j, Step& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct LimitNode : public PlanNode { +struct TopNNode : public PlanNode { std::shared_ptr source = {}; int64_t count = {}; - LimitNodeStep step = {}; - - LimitNode() noexcept; -}; -void to_json(json& j, const LimitNode& p); -void from_json(const json& j, LimitNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct UnnestNode : public PlanNode { - std::shared_ptr source = {}; - List replicateVariables = {}; - Map> - unnestVariables = {}; - std::shared_ptr ordinalityVariable = {}; + OrderingScheme orderingScheme = {}; + Step step = {}; - UnnestNode() noexcept; -}; -void to_json(json& j, const UnnestNode& p); -void from_json(const json& j, UnnestNode& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct MemoryAllocation { - String tag = {}; - int64_t allocation = {}; -}; -void to_json(json& j, const MemoryAllocation& p); -void from_json(const json& j, MemoryAllocation& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct MemoryPoolInfo { - int64_t maxBytes = {}; - int64_t reservedBytes = {}; - int64_t reservedRevocableBytes = {}; - Map queryMemoryReservations = {}; - Map> queryMemoryAllocations = {}; - Map queryMemoryRevocableReservations = {}; + TopNNode() noexcept; }; -void to_json(json& j, const MemoryPoolInfo& p); -void from_json(const json& j, MemoryPoolInfo& p); +void to_json(json& j, const TopNNode& p); +void from_json(const json& j, TopNNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RowNumberNode : public PlanNode { +struct SemiJoinNode : public PlanNode { std::shared_ptr source = {}; - List partitionBy = {}; - VariableReferenceExpression rowNumberVariable = {}; - std::shared_ptr maxRowCountPerPartition = {}; - std::shared_ptr hashVariable = {}; + std::shared_ptr filteringSource = {}; + VariableReferenceExpression sourceJoinVariable = {}; + VariableReferenceExpression filteringSourceJoinVariable = {}; + VariableReferenceExpression semiJoinOutput = {}; + std::shared_ptr sourceHashVariable = {}; + std::shared_ptr filteringSourceHashVariable = {}; + std::shared_ptr distributionType = {}; + Map dynamicFilters = {}; - RowNumberNode() noexcept; + SemiJoinNode() noexcept; }; -void to_json(json& j, const RowNumberNode& p); -void from_json(const json& j, RowNumberNode& p); +void to_json(json& j, const SemiJoinNode& p); +void from_json(const json& j, SemiJoinNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct WindowNode : public PlanNode { - std::shared_ptr sourceLocation = {}; - - std::shared_ptr source = {}; - Specification specification = {}; - Map windowFunctions = {}; - std::shared_ptr hashVariable = {}; - List prePartitionedInputs = {}; - int preSortedOrderPrefix = {}; +struct HiveOutputTableHandle : public ConnectorOutputTableHandle { + String schemaName = {}; + String tableName = {}; + List inputColumns = {}; + HivePageSinkMetadata pageSinkMetadata = {}; + LocationHandle locationHandle = {}; + HiveStorageFormat tableStorageFormat = {}; + HiveStorageFormat partitionStorageFormat = {}; + HiveStorageFormat actualStorageFormat = {}; + HiveCompressionCodec compressionCodec = {}; + List partitionedBy = {}; + std::shared_ptr bucketProperty = {}; + List preferredOrderingColumns = {}; + String tableOwner = {}; + Map additionalTableParameters = {}; + std::shared_ptr encryptionInformation = {}; - WindowNode() noexcept; + HiveOutputTableHandle() noexcept; }; -void to_json(json& j, const WindowNode& p); -void from_json(const json& j, WindowNode& p); +void to_json(json& j, const HiveOutputTableHandle& p); +void from_json(const json& j, HiveOutputTableHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HivePartitionKey { +enum class FunctionKind { SCALAR, AGGREGATE, WINDOW }; +extern void to_json(json& j, const FunctionKind& e); +extern void from_json(const json& j, FunctionKind& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct LongVariableConstraint { String name = {}; - std::shared_ptr value = {}; + String expression = {}; }; -void to_json(json& j, const HivePartitionKey& p); -void from_json(const json& j, HivePartitionKey& p); +void to_json(json& j, const LongVariableConstraint& p); +void from_json(const json& j, LongVariableConstraint& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class BufferState { - OPEN, - NO_MORE_BUFFERS, - NO_MORE_PAGES, - FLUSHING, - FINISHED, - FAILED +struct Signature { + QualifiedObjectName name = {}; + FunctionKind kind = {}; + List typeVariableConstraints = {}; + List longVariableConstraints = {}; + TypeSignature returnType = {}; + List argumentTypes = {}; + bool variableArity = {}; }; -extern void to_json(json& j, const BufferState& e); -extern void from_json(const json& j, BufferState& e); +void to_json(json& j, const Signature& p); +void from_json(const json& j, Signature& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct OutputBufferInfo { - String type = {}; - BufferState state = {}; - bool canAddBuffers = {}; - bool canAddPages = {}; - int64_t totalBufferedBytes = {}; - int64_t totalBufferedPages = {}; - int64_t totalRowsSent = {}; - int64_t totalPagesSent = {}; - List buffers = {}; +struct BuiltInFunctionHandle : public FunctionHandle { + Signature signature = {}; + + BuiltInFunctionHandle() noexcept; }; -void to_json(json& j, const OutputBufferInfo& p); -void from_json(const json& j, OutputBufferInfo& p); +void to_json(json& j, const BuiltInFunctionHandle& p); +void from_json(const json& j, BuiltInFunctionHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct ConstantExpression : public RowExpression { - Block valueBlock = {}; +struct AllOrNoneValueSet : public ValueSet { Type type = {}; + bool all = {}; - ConstantExpression() noexcept; + AllOrNoneValueSet() noexcept; }; -void to_json(json& j, const ConstantExpression& p); -void from_json(const json& j, ConstantExpression& p); +void to_json(json& j, const AllOrNoneValueSet& p); +void from_json(const json& j, AllOrNoneValueSet& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct InsertHandle : public ExecutionWriterTarget { - InsertTableHandle handle = {}; +struct DeleteHandle : public ExecutionWriterTarget { + TableHandle handle = {}; SchemaTableName schemaTableName = {}; - InsertHandle() noexcept; + DeleteHandle() noexcept; }; -void to_json(json& j, const InsertHandle& p); -void from_json(const json& j, InsertHandle& p); +void to_json(json& j, const DeleteHandle& p); +void from_json(const json& j, DeleteHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct MemoryInfo { - DataSize totalNodeMemory = {}; - Map pools = {}; +struct RefreshMaterializedViewHandle { + InsertTableHandle handle = {}; + SchemaTableName schemaTableName = {}; }; -void to_json(json& j, const MemoryInfo& p); -void from_json(const json& j, MemoryInfo& p); +void to_json(json& j, const RefreshMaterializedViewHandle& p); +void from_json(const json& j, RefreshMaterializedViewHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct NodeStatus { - String nodeId = {}; - NodeVersion nodeVersion = {}; - String environment = {}; - bool coordinator = {}; - Duration uptime = {}; - String externalAddress = {}; - String internalAddress = {}; - MemoryInfo memoryInfo = {}; - int processors = {}; - double processCpuLoad = {}; - double systemCpuLoad = {}; - int64_t heapUsed = {}; - int64_t heapAvailable = {}; - int64_t nonHeapUsed = {}; +struct HiveBucketHandle { + List columns = {}; + int tableBucketCount = {}; + int readBucketCount = {}; }; -void to_json(json& j, const NodeStatus& p); -void from_json(const json& j, NodeStatus& p); +void to_json(json& j, const HiveBucketHandle& p); +void from_json(const json& j, HiveBucketHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveTableHandle : public ConnectorTableHandle { - String schemaName = {}; - String tableName = {}; - std::shared_ptr>> analyzePartitionValues = {}; +struct HiveTableLayoutHandle : public ConnectorTableLayoutHandle { + SchemaTableName schemaTableName = {}; + String tablePath = {}; + List partitionColumns = {}; + List dataColumns = {}; + Map tableParameters = {}; + TupleDomain domainPredicate = {}; + std::shared_ptr remainingPredicate = {}; + Map predicateColumns = {}; + TupleDomain> partitionColumnPredicate = {}; + std::shared_ptr bucketHandle = {}; + std::shared_ptr bucketFilter = {}; + bool pushdownFilterEnabled = {}; + String layoutString = {}; + std::shared_ptr> requestedColumns = {}; + bool partialAggregationsPushedDown = {}; + bool appendRowNumber = {}; - HiveTableHandle() noexcept; -}; -void to_json(json& j, const HiveTableHandle& p); -void from_json(const json& j, HiveTableHandle& p); -} // namespace facebook::presto::protocol -namespace facebook::presto::protocol { -struct TaskInfo { - TaskId taskId = {}; - TaskStatus taskStatus = {}; - DateTime lastHeartbeat = {}; - OutputBufferInfo outputBuffers = {}; - List noMoreSplits = {}; - TaskStats stats = {}; - bool needsPlan = {}; - MetadataUpdates metadataUpdates = {}; - String nodeId = {}; + HiveTableLayoutHandle() noexcept; }; -void to_json(json& j, const TaskInfo& p); -void from_json(const json& j, TaskInfo& p); +void to_json(json& j, const HiveTableLayoutHandle& p); +void from_json(const json& j, HiveTableLayoutHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Range { - Marker low = {}; - Marker high = {}; +enum class BufferType { + PARTITIONED, + BROADCAST, + ARBITRARY, + DISCARDING, + SPOOLING }; -void to_json(json& j, const Range& p); -void from_json(const json& j, Range& p); +extern void to_json(json& j, const BufferType& e); +extern void from_json(const json& j, BufferType& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SortedRangeSet : public ValueSet { - Type type = {}; - List ranges = {}; - - SortedRangeSet() noexcept; +struct OutputBuffers { + BufferType type = {}; + int64_t version = {}; + bool noMoreBufferIds = {}; + Map buffers = {}; }; -void to_json(json& j, const SortedRangeSet& p); -void from_json(const json& j, SortedRangeSet& p); +void to_json(json& j, const OutputBuffers& p); +void from_json(const json& j, OutputBuffers& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class AggregationNodeStep { PARTIAL, FINAL, INTERMEDIATE, SINGLE }; -extern void to_json(json& j, const AggregationNodeStep& e); -extern void from_json(const json& j, AggregationNodeStep& e); +enum class RuntimeUnit { NONE, NANO, BYTE }; +extern void to_json(json& j, const RuntimeUnit& e); +extern void from_json(const json& j, RuntimeUnit& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct GroupingSetDescriptor { - List groupingKeys = {}; - int groupingSetCount = {}; - List globalGroupingSets = {}; +struct RuntimeMetric { + String name = {}; + RuntimeUnit unit = {}; + int64_t sum = {}; + int64_t count = {}; + int64_t max = {}; + int64_t min = {}; }; -void to_json(json& j, const GroupingSetDescriptor& p); -void from_json(const json& j, GroupingSetDescriptor& p); +void to_json(json& j, const RuntimeMetric& p); +void from_json(const json& j, RuntimeMetric& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct AggregationNode : public PlanNode { +struct UnnestNode : public PlanNode { std::shared_ptr source = {}; - Map aggregations = {}; - GroupingSetDescriptor groupingSets = {}; - List preGroupedVariables = {}; - AggregationNodeStep step = {}; - std::shared_ptr hashVariable = {}; - std::shared_ptr groupIdVariable = {}; + List replicateVariables = {}; + Map> + unnestVariables = {}; + std::shared_ptr ordinalityVariable = {}; - AggregationNode() noexcept; + UnnestNode() noexcept; }; -void to_json(json& j, const AggregationNode& p); -void from_json(const json& j, AggregationNode& p); +void to_json(json& j, const UnnestNode& p); +void from_json(const json& j, UnnestNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HivePartitioningHandle : public ConnectorPartitioningHandle { - int bucketCount = {}; - std::shared_ptr maxCompatibleBucketCount = {}; - BucketFunctionType bucketFunctionType = {}; - std::shared_ptr> hiveTypes = {}; - std::shared_ptr> types = {}; - - HivePartitioningHandle() noexcept; +struct ResourceEstimates { + std::shared_ptr executionTime = {}; + std::shared_ptr cpuTime = {}; + std::shared_ptr peakMemory = {}; + std::shared_ptr peakTaskMemory = {}; }; -void to_json(json& j, const HivePartitioningHandle& p); -void from_json(const json& j, HivePartitioningHandle& p); +void to_json(json& j, const ResourceEstimates& p); +void from_json(const json& j, ResourceEstimates& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SortNode : public PlanNode { - std::shared_ptr source = {}; - OrderingScheme orderingScheme = {}; - bool isPartial = {}; - - SortNode() noexcept; -}; -void to_json(json& j, const SortNode& p); -void from_json(const json& j, SortNode& p); +struct SqlInvokedFunction { + List parameters = {}; + String description = {}; + RoutineCharacteristics routineCharacteristics = {}; + String body = {}; + Signature signature = {}; + SqlFunctionId functionId = {}; +}; +void to_json(json& j, const SqlInvokedFunction& p); +void from_json(const json& j, SqlInvokedFunction& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct EnforceSingleRowNode : public PlanNode { - std::shared_ptr source = {}; - - EnforceSingleRowNode() noexcept; +struct SessionRepresentation { + String queryId = {}; + std::shared_ptr transactionId = {}; + bool clientTransactionSupport = {}; + String user = {}; + std::shared_ptr principal = {}; + std::shared_ptr source = {}; + std::shared_ptr catalog = {}; + std::shared_ptr schema = {}; + std::shared_ptr traceToken = {}; + TimeZoneKey timeZoneKey = {}; + Locale locale = {}; + std::shared_ptr remoteUserAddress = {}; + std::shared_ptr userAgent = {}; + std::shared_ptr clientInfo = {}; + List clientTags = {}; + ResourceEstimates resourceEstimates = {}; + int64_t startTime = {}; + Map systemProperties = {}; + Map> catalogProperties = {}; + Map> unprocessedCatalogProperties = {}; + Map roles = {}; + Map preparedStatements = {}; + Map sessionFunctions = {}; }; -void to_json(json& j, const EnforceSingleRowNode& p); -void from_json(const json& j, EnforceSingleRowNode& p); +void to_json(json& j, const SessionRepresentation& p); +void from_json(const json& j, SessionRepresentation& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct Location { - String location = {}; +struct LambdaDefinitionExpression : public RowExpression { + List argumentTypes = {}; + List arguments = {}; + std::shared_ptr body = {}; + + LambdaDefinitionExpression() noexcept; }; -void to_json(json& j, const Location& p); -void from_json(const json& j, Location& p); +void to_json(json& j, const LambdaDefinitionExpression& p); +void from_json(const json& j, LambdaDefinitionExpression& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct RemoteSplit : public ConnectorSplit { - Location location = {}; - TaskId remoteSourceTaskId = {}; +struct ValuesNode : public PlanNode { + std::shared_ptr location = {}; - RemoteSplit() noexcept; + List outputVariables = {}; + List>> rows = {}; + std::shared_ptr valuesNodeLabel = {}; + + ValuesNode() noexcept; }; -void to_json(json& j, const RemoteSplit& p); -void from_json(const json& j, RemoteSplit& p); +void to_json(json& j, const ValuesNode& p); +void from_json(const json& j, ValuesNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct SemiJoinNode : public PlanNode { - std::shared_ptr source = {}; - std::shared_ptr filteringSource = {}; - VariableReferenceExpression sourceJoinVariable = {}; - VariableReferenceExpression filteringSourceJoinVariable = {}; - VariableReferenceExpression semiJoinOutput = {}; - std::shared_ptr sourceHashVariable = {}; - std::shared_ptr filteringSourceHashVariable = {}; - std::shared_ptr distributionType = {}; - Map dynamicFilters = {}; - - SemiJoinNode() noexcept; +enum class SystemPartitionFunction { + SINGLE, + HASH, + ROUND_ROBIN, + BROADCAST, + UNKNOWN }; -void to_json(json& j, const SemiJoinNode& p); -void from_json(const json& j, SemiJoinNode& p); +extern void to_json(json& j, const SystemPartitionFunction& e); +extern void from_json(const json& j, SystemPartitionFunction& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -enum class NodeSelectionStrategy { - HARD_AFFINITY, - SOFT_AFFINITY, - NO_PREFERENCE +enum class SystemPartitioning { + SINGLE, + FIXED, + SOURCE, + SCALED, + COORDINATOR_ONLY, + ARBITRARY }; -extern void to_json(json& j, const NodeSelectionStrategy& e); -extern void from_json(const json& j, NodeSelectionStrategy& e); +extern void to_json(json& j, const SystemPartitioning& e); +extern void from_json(const json& j, SystemPartitioning& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct HiveSplit : public ConnectorSplit { - HiveFileSplit fileSplit = {}; - String database = {}; - String table = {}; - String partitionName = {}; - Storage storage = {}; - List partitionKeys = {}; - List addresses = {}; - std::shared_ptr readBucketNumber = {}; - std::shared_ptr tableBucketNumber = {}; - NodeSelectionStrategy nodeSelectionStrategy = {}; - int partitionDataColumnCount = {}; - TableToPartitionMapping tableToPartitionMapping = {}; - std::shared_ptr bucketConversion = {}; - bool s3SelectPushdownEnabled = {}; - CacheQuotaRequirement cacheQuota = {}; - std::shared_ptr encryptionMetadata = {}; - List> redundantColumnDomains = {}; - SplitWeight splitWeight = {}; +struct SystemPartitioningHandle : public ConnectorPartitioningHandle { + SystemPartitioning partitioning = {}; + SystemPartitionFunction function = {}; - HiveSplit() noexcept; + SystemPartitioningHandle() noexcept; }; -void to_json(json& j, const HiveSplit& p); -void from_json(const json& j, HiveSplit& p); +void to_json(json& j, const SystemPartitioningHandle& p); +void from_json(const json& j, SystemPartitioningHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct TpchPartitioningHandle : public ConnectorPartitioningHandle { String table = {}; - int64_t totalRows; + int64_t totalRows = {}; + TpchPartitioningHandle() noexcept; }; void to_json(json& j, const TpchPartitioningHandle& p); void from_json(const json& j, TpchPartitioningHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchTransactionHandle : public ConnectorTransactionHandle { - String instance = {}; +struct TpchSplit : public ConnectorSplit { + TpchTableHandle tableHandle = {}; + int partNumber = {}; + int totalParts = {}; + List addresses = {}; + TupleDomain> predicate = {}; - TpchTransactionHandle() noexcept; + TpchSplit() noexcept; }; -void to_json(json& j, const TpchTransactionHandle& p); -void from_json(const json& j, TpchTransactionHandle& p); +void to_json(json& j, const TpchSplit& p); +void from_json(const json& j, TpchSplit& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchColumnHandle : public ColumnHandle { - String columnName = {}; - Type type = {}; - TpchColumnHandle() noexcept; +enum class StageExecutionStrategy { + UNGROUPED_EXECUTION, + FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION, + RECOVERABLE_GROUPED_EXECUTION }; -void to_json(json& j, const TpchColumnHandle& p); -void from_json(const json& j, TpchColumnHandle& p); +extern void to_json(json& j, const StageExecutionStrategy& e); +extern void from_json(const json& j, StageExecutionStrategy& e); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchTableHandle : public ConnectorTableHandle { - String tableName = {}; - double scaleFactor = 0; - - TpchTableHandle() noexcept; +struct StageExecutionDescriptor { + StageExecutionStrategy stageExecutionStrategy = {}; + List groupedExecutionScanNodes = {}; + int totalLifespans = {}; }; -void to_json(json& j, const TpchTableHandle& p); -void from_json(const json& j, TpchTableHandle& p); +void to_json(json& j, const StageExecutionDescriptor& p); +void from_json(const json& j, StageExecutionDescriptor& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchPredicate { - List>> columnDomains; - - TpchPredicate() noexcept; +struct TaskUpdateRequest { + SessionRepresentation session = {}; + Map extraCredentials = {}; + std::shared_ptr fragment = {}; + List sources = {}; + OutputBuffers outputIds = {}; + std::shared_ptr tableWriteInfo = {}; }; -void to_json(json& j, const TpchPredicate& p); -void from_json(const json& j, TpchPredicate& p); +void to_json(json& j, const TaskUpdateRequest& p); +void from_json(const json& j, TaskUpdateRequest& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchTableLayoutHandle : public ConnectorTableLayoutHandle { - TpchTableHandle table; - TpchPredicate predicate; +enum class LimitNodeStep { PARTIAL, FINAL }; +extern void to_json(json& j, const LimitNodeStep& e); +extern void from_json(const json& j, LimitNodeStep& e); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct LimitNode : public PlanNode { + std::shared_ptr source = {}; + int64_t count = {}; + LimitNodeStep step = {}; - TpchTableLayoutHandle() noexcept; + LimitNode() noexcept; }; -void to_json(json& j, const TpchTableLayoutHandle& p); -void from_json(const json& j, TpchTableLayoutHandle& p); +void to_json(json& j, const LimitNode& p); +void from_json(const json& j, LimitNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { -struct TpchSplit : public ConnectorSplit { - TpchTableHandle tableHandle = {}; - int partNumber; - int totalParts; - List addresses = {}; - TpchPredicate predicate; +struct HivePartitioningHandle : public ConnectorPartitioningHandle { + int bucketCount = {}; + std::shared_ptr maxCompatibleBucketCount = {}; + BucketFunctionType bucketFunctionType = {}; + std::shared_ptr> hiveTypes = {}; + std::shared_ptr> types = {}; - TpchSplit() noexcept; + HivePartitioningHandle() noexcept; }; -void to_json(json& j, const TpchSplit& p); -void from_json(const json& j, TpchSplit& p); +void to_json(json& j, const HivePartitioningHandle& p); +void from_json(const json& j, HivePartitioningHandle& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { struct OutputNode : public PlanNode { @@ -2801,6 +2781,59 @@ void to_json(json& j, const OutputNode& p); void from_json(const json& j, OutputNode& p); } // namespace facebook::presto::protocol namespace facebook::presto::protocol { +struct SortNode : public PlanNode { + std::shared_ptr source = {}; + OrderingScheme orderingScheme = {}; + bool isPartial = {}; + + SortNode() noexcept; +}; +void to_json(json& j, const SortNode& p); +void from_json(const json& j, SortNode& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { +struct RowNumberNode : public PlanNode { + std::shared_ptr source = {}; + List partitionBy = {}; + VariableReferenceExpression rowNumberVariable = {}; + std::shared_ptr maxRowCountPerPartition = {}; + std::shared_ptr hashVariable = {}; + + RowNumberNode() noexcept; +}; +void to_json(json& j, const RowNumberNode& p); +void from_json(const json& j, RowNumberNode& p); +} // namespace facebook::presto::protocol +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +namespace facebook::presto::protocol { +struct PlanFragment { + PlanFragmentId id = {}; + std::shared_ptr root = {}; + List variables = {}; + PartitioningHandle partitioning = {}; + List tableScanSchedulingOrder = {}; + PartitioningScheme partitioningScheme = {}; + StageExecutionDescriptor stageExecutionDescriptor = {}; + bool outputTableWriterFragment = {}; + std::shared_ptr jsonRepresentation = {}; +}; +void to_json(json& j, const PlanFragment& p); +void from_json(const json& j, PlanFragment& p); +} // namespace facebook::presto::protocol +namespace facebook::presto::protocol { enum class NodeState { ACTIVE, INACTIVE, SHUTTING_DOWN }; extern void to_json(json& j, const NodeState& e); extern void from_json(const json& j, NodeState& e); diff --git a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.json b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.json index 37f4528bcc336..91da931aff4b2 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.json +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.json @@ -3,99 +3,19 @@ "comment": "// This file is generated DO NOT EDIT @generated" }, { - "class_name": "HostAddress", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nusing HostAddress = std::string;\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "ErrorCause", - "enum": true, - "elements": [ - { - "element": "UNKNOWN", - "_N": 1 - }, - { - "element": "LOW_PARTITION_COUNT", - "_N": 2 - }, - { - "element": "EXCEEDS_BROADCAST_MEMORY_LIMIT", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "ErrorType", - "enum": true, - "elements": [ - { - "element": "USER_ERROR", - "_N": 1 - }, - { - "element": "INTERNAL_ERROR", - "_N": 2 - }, - { - "element": "INSUFFICIENT_RESOURCES", - "_N": 3 - }, - { - "element": "EXTERNAL", - "_N": 4, - "_last": true - } - ] - }, - { - "class_name": "ErrorCode", - "struct": true, - "fields": [ - { - "field_type": "int", - "field_name": "code", - "field_text": "int", - "_N": 1, - "field_local": true - }, - { - "field_type": "String", - "field_name": "name", - "field_text": "String", - "_N": 2, - "field_local": true - }, - { - "field_type": "ErrorType", - "field_name": "type", - "field_text": "ErrorType", - "_N": 3, - "field_local": true - }, - { - "field_type": "boolean", - "field_name": "retriable", - "field_text": "bool", - "_N": 4, - "field_local": true - } - ] - }, - { - "class_name": "ErrorLocation", + "class_name": "SourceLocation", "struct": true, "fields": [ { "field_type": "int", - "field_name": "lineNumber", + "field_name": "line", "field_text": "int", "_N": 1, "field_local": true }, { "field_type": "int", - "field_name": "columnNumber", + "field_name": "column", "field_text": "int", "_N": 2, "field_local": true @@ -103,1304 +23,1644 @@ ] }, { - "class_name": "ExecutionFailureInfo", + "class_name": "VariableReferenceExpression", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct VariableReferenceExpression : RowExpression {\n String name;\n Type type; // dependency\n\n VariableReferenceExpression() noexcept;\n explicit VariableReferenceExpression(const String& str) {\n _type = \"variable\";\n\n std::vector> parts;\n\n folly::split(\"<\", str, parts);\n name = parts[0];\n type = parts[1].substr(0, parts[1].length() - 1);\n }\n\n bool operator<(const VariableReferenceExpression& o) const {\n if (name == o.name) {\n return type < o.type;\n }\n\n return name < o.name;\n }\n};\n\nvoid to_json(json& j, const VariableReferenceExpression& p);\nvoid from_json(const json& j, VariableReferenceExpression& p);\n\nstd::string json_map_key(\n const facebook::presto::protocol::VariableReferenceExpression& p);\n\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "type", - "field_text": "String", + "field_type": "Optional", + "field_name": "sourceLocation", + "field_text": "SourceLocation", + "optional": true, "_N": 1, - "field_local": true + "field_local": false }, { "field_type": "String", - "field_name": "message", + "field_name": "name", "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "ExecutionFailureInfo", - "field_name": "cause", - "field_text": "ExecutionFailureInfo", + "field_type": "Type", + "field_name": "type", + "field_text": "Type", "_N": 3, - "optional": true, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "RowExpression", + "json_key": "variable" + }, + { + "class_name": "PlanNode", + "field_name": "planNode", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "field_type": "List", - "field_name": "suppressed", - "field_text": "List", - "_N": 4, - "field_local": true + "type": "AggregationNode", + "name": "aggregationNode", + "key": ".AggregationNode", + "_N": 1 }, { - "field_type": "List", - "field_name": "stack", - "field_text": "List", - "_N": 5, - "field_local": true + "type": "GroupIdNode", + "name": "groupIdNode", + "key": "com.facebook.presto.sql.planner.plan.GroupIdNode", + "_N": 2 }, { - "field_type": "ErrorLocation", - "field_name": "errorLocation", - "field_text": "ErrorLocation", - "_N": 6, - "field_local": true + "type": "DistinctLimitNode", + "name": "distinctLimitNode", + "key": ".DistinctLimitNode", + "_N": 3 }, { - "field_type": "ErrorCode", - "field_name": "errorCode", - "field_text": "ErrorCode", - "_N": 7, - "field_local": true + "type": "EnforceSingleRowNode", + "name": "enforceSingleRowNode", + "key": "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode", + "_N": 4 }, { - "field_type": "HostAddress", - "field_name": "remoteHost", - "field_text": "HostAddress", - "_N": 8, - "field_local": true + "type": "ExchangeNode", + "name": "exchangeNode", + "key": "com.facebook.presto.sql.planner.plan.ExchangeNode", + "_N": 5 }, { - "field_type": "ErrorCause", - "field_name": "errorCause", - "field_text": "ErrorCause", - "_N": 9, - "field_local": true - } - ] - }, - { - "class_name": "DataSize", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(nlohmann::json& j, const DataSize& p) {\n j = p.toString();\n}\n\nvoid from_json(const nlohmann::json& j, DataSize& p) {\n p = DataSize(std::string(j));\n}\n\nstd::ostream& operator<<(std::ostream& os, const DataSize& d) {\n return os << d.toString();\n}\n\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstd::ostream& operator<<(std::ostream& os, const DataSize& d);\n\nvoid to_json(nlohmann::json& j, const DataSize& p);\nvoid from_json(const nlohmann::json& j, DataSize& p);\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "Duration", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Duration& p) {\n j = p.toString();\n}\n\nvoid from_json(const json& j, Duration& p) {\n p = Duration(std::string(j));\n}\n\nstd::ostream& operator<<(std::ostream& os, const Duration& d) {\n return os << d.toString();\n}\n\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nnamespace facebook::presto::protocol {\n\nstd::ostream& operator<<(std::ostream& os, const Duration& d);\n\nvoid to_json(json& j, const Duration& p);\nvoid from_json(const json& j, Duration& p);\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "ResourceEstimates", - "struct": true, - "fields": [ - { - "field_type": "Optional", - "field_name": "executionTime", - "field_text": "Duration", - "optional": true, - "_N": 1, - "field_local": true + "type": "FilterNode", + "name": "filterNode", + "key": ".FilterNode", + "_N": 6 }, { - "field_type": "Optional", - "field_name": "cpuTime", - "field_text": "Duration", - "optional": true, - "_N": 2, - "field_local": true + "type": "JoinNode", + "name": "joinNode", + "key": "com.facebook.presto.sql.planner.plan.JoinNode", + "_N": 7 }, { - "field_type": "Optional", - "field_name": "peakMemory", - "field_text": "DataSize", - "optional": true, - "_N": 3, - "field_local": true + "type": "LimitNode", + "name": "limitNode", + "key": ".LimitNode", + "_N": 8 }, { - "field_type": "Optional", - "field_name": "peakTaskMemory", - "field_text": "DataSize", - "optional": true, - "_N": 4, - "field_local": true - } - ] - }, - { - "class_name": "Lifespan", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Lifespan& p) {\n if (p.isgroup) {\n j = \"Group\" + std::to_string(p.groupid);\n } else {\n j = \"TaskWide\";\n }\n}\n\nvoid from_json(const json& j, Lifespan& p) {\n String lifespan = j;\n\n if (lifespan == \"TaskWide\") {\n p.isgroup = false;\n p.groupid = 0;\n } else {\n if (lifespan != \"Group\") {\n // fail...\n }\n p.isgroup = true;\n p.groupid = std::stoi(lifespan.substr(strlen(\"Group\")));\n }\n}\n\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct Lifespan {\n bool isgroup = false;\n long groupid = 0;\n\n bool operator<(const Lifespan& o) const {\n return groupid < o.groupid;\n }\n};\n\nvoid to_json(json& j, const Lifespan& p);\nvoid from_json(const json& j, Lifespan& p);\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "TaskState", - "enum": true, - "elements": [ - { - "element": "PLANNED", - "_N": 1 + "type": "SortNode", + "name": "sortNode", + "key": "com.facebook.presto.sql.planner.plan.SortNode", + "_N": 9 }, { - "element": "RUNNING", - "_N": 2 + "type": "OutputNode", + "name": "outputNode", + "key": "com.facebook.presto.sql.planner.plan.OutputNode", + "_N": 10 }, { - "element": "FINISHED", - "_N": 3 + "type": "ProjectNode", + "name": "projectNode", + "key": ".ProjectNode", + "_N": 11 }, { - "element": "CANCELED", - "_N": 4 + "type": "RowNumberNode", + "name": "rowNumberNode", + "key": "com.facebook.presto.sql.planner.plan.RowNumberNode", + "_N": 12 }, { - "element": "ABORTED", - "_N": 5 + "type": "RemoteSourceNode", + "name": "remoteSourceNode", + "key": "com.facebook.presto.sql.planner.plan.RemoteSourceNode", + "_N": 13 }, { - "element": "FAILED", - "_N": 6, + "type": "SemiJoinNode", + "name": "semiJoinNode", + "key": "com.facebook.presto.sql.planner.plan.SemiJoinNode", + "_N": 14 + }, + { + "type": "TableScanNode", + "name": "tableScanNode", + "key": ".TableScanNode", + "_N": 15 + }, + { + "type": "TableWriterNode", + "name": "tableWriterNode", + "key": "com.facebook.presto.sql.planner.plan.TableWriterNode", + "_N": 16 + }, + { + "type": "TopNNode", + "name": "topNNode", + "key": ".TopNNode", + "_N": 17 + }, + { + "type": "UnnestNode", + "name": "unnestNode", + "key": "com.facebook.presto.sql.planner.plan.UnnestNode", + "_N": 18 + }, + { + "type": "ValuesNode", + "name": "valuesNode", + "key": ".ValuesNode", + "_N": 19 + }, + { + "type": "AssignUniqueId", + "name": "assignUniqueId", + "key": "com.facebook.presto.sql.planner.plan.AssignUniqueId", + "_N": 20 + }, + { + "type": "MergeJoinNode", + "name": "mergeJoinNode", + "key": "com.facebook.presto.sql.planner.plan.MergeJoinNode", + "_N": 21 + }, + { + "type": "WindowNode", + "name": "windowNode", + "key": "com.facebook.presto.sql.planner.plan.WindowNode", + "_N": 22, "_last": true } + ], + "fields": [ + { + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": true, + "last": true + } ] }, { - "class_name": "TaskStatus", + "class_name": "GroupIdNode", "struct": true, "fields": [ { - "field_type": "long", - "field_name": "taskInstanceIdLeastSignificantBits", - "field_text": "int64_t", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 1, - "field_local": true + "field_local": false }, { - "field_type": "long", - "field_name": "taskInstanceIdMostSignificantBits", - "field_text": "int64_t", + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", "_N": 2, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "long", - "field_name": "version", - "field_text": "int64_t", + "field_type": "List>", + "field_name": "groupingSets", + "field_text": "List>", "_N": 3, "field_local": true }, { - "field_type": "TaskState", - "field_name": "state", - "field_text": "TaskState", + "field_type": "Map", + "field_name": "groupingColumns", + "field_text": "Map", "_N": 4, "field_local": true }, { - "field_type": "URI", - "field_name": "self", - "field_text": "URI", + "field_type": "List", + "field_name": "aggregationArguments", + "field_text": "List", "_N": 5, "field_local": true }, { - "field_type": "Set", - "field_name": "completedDriverGroups", - "field_text": "List", + "field_type": "VariableReferenceExpression", + "field_name": "groupIdVariable", + "field_text": "VariableReferenceExpression", "_N": 6, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.GroupIdNode" + }, + { + "class_name": "DwrfEncryptionMetadata", + "struct": true, + "fields": [ { - "field_type": "List", - "field_name": "failures", - "field_text": "List", - "_N": 7, + "field_type": "Map", + "field_name": "fieldToKeyData", + "field_text": "Map", + "_N": 1, "field_local": true }, { - "field_type": "int", - "field_name": "queuedPartitionedDrivers", - "field_text": "int", - "_N": 8, + "field_type": "Map", + "field_name": "extraMetadata", + "field_text": "Map", + "_N": 2, "field_local": true }, { - "field_type": "int", - "field_name": "runningPartitionedDrivers", - "field_text": "int", - "_N": 9, + "field_type": "String", + "field_name": "encryptionAlgorithm", + "field_text": "String", + "_N": 3, "field_local": true }, { - "field_type": "double", - "field_name": "outputBufferUtilization", - "field_text": "double", - "_N": 10, + "field_type": "String", + "field_name": "encryptionProvider", + "field_text": "String", + "_N": 4, "field_local": true - }, + } + ] + }, + { + "class_name": "EncryptionInformation", + "struct": true, + "fields": [ + { + "field_type": "Optional", + "field_name": "dwrfEncryptionMetadata", + "field_text": "DwrfEncryptionMetadata", + "optional": true, + "_N": 1, + "field_local": true + } + ] + }, + { + "class_name": "Lifespan", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Lifespan& p) {\n if (p.isgroup) {\n j = \"Group\" + std::to_string(p.groupid);\n } else {\n j = \"TaskWide\";\n }\n}\n\nvoid from_json(const json& j, Lifespan& p) {\n String lifespan = j;\n\n if (lifespan == \"TaskWide\") {\n p.isgroup = false;\n p.groupid = 0;\n } else {\n if (lifespan != \"Group\") {\n // fail...\n }\n p.isgroup = true;\n p.groupid = std::stoi(lifespan.substr(strlen(\"Group\")));\n }\n}\n\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct Lifespan {\n bool isgroup = false;\n long groupid = 0;\n\n bool operator<(const Lifespan& o) const {\n return groupid < o.groupid;\n }\n};\n\nvoid to_json(json& j, const Lifespan& p);\nvoid from_json(const json& j, Lifespan& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "SplitContext", + "struct": true, + "fields": [ { "field_type": "boolean", - "field_name": "outputBufferOverutilized", + "field_name": "cacheable", "field_text": "bool", - "_N": 11, + "_N": 1, "field_local": true - }, + } + ] + }, + { + "class_name": "TpchTransactionHandle", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// TpchTransactionHandle is special since\n// the corresponding class in Java is an enum.\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const TpchTransactionHandle& p) {\n j = json::array();\n j.push_back(p._type);\n j.push_back(p.instance);\n}\n\nvoid from_json(const json& j, TpchTransactionHandle& p) {\n j[0].get_to(p._type);\n j[1].get_to(p.instance);\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// TpchTransactionHandle is special since\n// the corresponding class in Java is an enum.\n\nnamespace facebook::presto::protocol {\n\nstruct TpchTransactionHandle : public ConnectorTransactionHandle {\n String instance = {};\n};\n\nvoid to_json(json& j, const TpchTransactionHandle& p);\n\nvoid from_json(const json& j, TpchTransactionHandle& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "ConnectorTransactionHandle", + "field_name": "connectorTransactionHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "field_type": "long", - "field_name": "physicalWrittenDataSizeInBytes", - "field_text": "int64_t", - "_N": 12, - "field_local": true + "type": "HiveTransactionHandle", + "name": "hiveTransactionHandle", + "key": "hive", + "_N": 1 }, { - "field_type": "long", - "field_name": "memoryReservationInBytes", - "field_text": "int64_t", - "_N": 13, - "field_local": true + "type": "RemoteTransactionHandle", + "name": "remoteTransactionHandle", + "key": "$remote", + "_N": 2, + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// dependency TpchTransactionHandle\n\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTransactionHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n // TPC-H transactionHandle is an array [\"tpch\",\"INSTANCE\"].\n if (j.is_array()) {\n type = j[0];\n } else {\n type = p->getSubclassKey(j);\n }\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorTransactionHandle ConnectorTransactionHandle\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"tpch\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTransactionHandle\");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "ConnectorSplit", + "field_name": "connectorSplit", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ + { + "type": "HiveSplit", + "name": "hiveSplit", + "key": "hive", + "_N": 1 }, { - "field_type": "long", - "field_name": "systemMemoryReservationInBytes", - "field_text": "int64_t", - "_N": 14, - "field_local": true + "type": "TpchSplit", + "name": "tpchSplit", + "key": "tpch", + "_N": 2 }, { - "field_type": "long", - "field_name": "peakNodeTotalMemoryReservationInBytes", - "field_text": "int64_t", - "_N": 15, - "field_local": true + "type": "RemoteSplit", + "name": "remoteSplit", + "key": "$remote", + "_N": 3 }, { - "field_type": "long", - "field_name": "fullGcCount", - "field_text": "int64_t", - "_N": 16, + "type": "EmptySplit", + "name": "emptySplit", + "key": "$empty", + "_N": 4, + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (type == \"$empty\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorSplit\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorSplit\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (type == \"$empty\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorSplit\");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "Split", + "struct": true, + "fields": [ + { + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "fullGcTimeInMillis", - "field_text": "int64_t", - "_N": 17, - "field_local": true + "field_type": "ConnectorTransactionHandle", + "field_name": "transactionHandle", + "field_text": "ConnectorTransactionHandle", + "_N": 2, + "field_local": true, + "optional": true }, { - "field_type": "long", - "field_name": "totalCpuTimeInNanos", - "field_text": "int64_t", - "_N": 18, + "field_type": "ConnectorSplit", + "field_name": "connectorSplit", + "field_text": "ConnectorSplit", + "_N": 3, + "field_local": true, + "optional": true + }, + { + "field_type": "Lifespan", + "field_name": "lifespan", + "field_text": "Lifespan", + "_N": 4, "field_local": true }, + { + "field_type": "SplitContext", + "field_name": "splitContext", + "field_text": "SplitContext", + "_N": 5, + "field_local": true + } + ] + }, + { + "class_name": "ScheduledSplit", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct ScheduledSplit {\n long sequenceId = {};\n PlanNodeId planNodeId = {}; // dependency\n Split split = {};\n\n bool operator<(const ScheduledSplit& o) const {\n return sequenceId < o.sequenceId;\n }\n};\n\nvoid to_json(json& j, const ScheduledSplit& p);\nvoid from_json(const json& j, ScheduledSplit& p);\n\n} // namespace facebook::presto::protocol", + "struct": true, + "fields": [ { "field_type": "long", - "field_name": "taskAgeInMillis", + "field_name": "sequenceId", "field_text": "int64_t", - "_N": 19, + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "queuedPartitionedSplitsWeight", - "field_text": "int64_t", - "_N": 20, + "field_type": "PlanNodeId", + "field_name": "planNodeId", + "field_text": "PlanNodeId", + "_N": 2, "field_local": true }, { - "field_type": "long", - "field_name": "runningPartitionedSplitsWeight", - "field_text": "int64_t", - "_N": 21, + "field_type": "Split", + "field_name": "split", + "field_text": "Split", + "_N": 3, "field_local": true } ] }, { - "class_name": "CacheQuotaScope", - "enum": true, - "elements": [ + "class_name": "TaskSource", + "struct": true, + "fields": [ { - "element": "GLOBAL", - "_N": 1 + "field_type": "PlanNodeId", + "field_name": "planNodeId", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": true }, { - "element": "SCHEMA", - "_N": 2 + "field_type": "Set", + "field_name": "splits", + "field_text": "List", + "_N": 2, + "field_local": true }, { - "element": "TABLE", - "_N": 3 + "field_type": "Set", + "field_name": "noMoreSplitsForLifespan", + "field_text": "List", + "_N": 3, + "field_local": true }, { - "element": "PARTITION", + "field_type": "boolean", + "field_name": "noMoreSplits", + "field_text": "bool", "_N": 4, - "_last": true + "field_local": true } ] }, { - "class_name": "CacheQuotaRequirement", + "class_name": "TypeVariableConstraint", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct TypeVariableConstraint {\n String name = {};\n bool comparableRequired = {};\n bool orderableRequired = {};\n String variadicBound = {};\n bool nonDecimalNumericRequired = {};\n String boundedBy = {};\n};\nvoid to_json(json& j, const TypeVariableConstraint& p);\nvoid from_json(const json& j, TypeVariableConstraint& p);\n\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { - "field_type": "CacheQuotaScope", - "field_name": "cacheQuotaScope", - "field_text": "CacheQuotaScope", + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "quota", - "field_text": "DataSize", - "optional": true, + "field_type": "boolean", + "field_name": "comparableRequired", + "field_text": "bool", "_N": 2, "field_local": true + }, + { + "field_type": "boolean", + "field_name": "orderableRequired", + "field_text": "bool", + "_N": 3, + "field_local": true + }, + { + "field_type": "String", + "field_name": "variadicBound", + "field_text": "String", + "_N": 4, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "nonDecimalNumericRequired", + "field_text": "bool", + "_N": 5, + "field_local": true } ] }, { - "class_name": "OperatorInfo", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const OperatorInfo& p) {}\nvoid from_json(const json& j, OperatorInfo& p) {}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct OperatorInfo {};\nvoid to_json(json& j, const OperatorInfo& p);\nvoid from_json(const json& j, OperatorInfo& p);\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "BlockedReason", + "class_name": "SelectedRoleType", "enum": true, "elements": [ { - "element": "WAITING_FOR_MEMORY", - "_N": 1, + "element": "ROLE", + "_N": 1 + }, + { + "element": "ALL", + "_N": 2 + }, + { + "element": "NONE", + "_N": 3, "_last": true } ] }, { - "class_name": "OperatorStats", + "class_name": "SelectedRole", "struct": true, "fields": [ { - "field_type": "int", - "field_name": "stageId", - "field_text": "int", + "field_type": "SelectedRoleType", + "field_name": "type", + "field_text": "SelectedRoleType", "_N": 1, "field_local": true }, { - "field_type": "int", - "field_name": "stageExecutionId", - "field_text": "int", + "field_type": "Optional", + "field_name": "role", + "field_text": "String", + "optional": true, "_N": 2, "field_local": true + } + ] + }, + { + "class_name": "ExchangeNodeScope", + "enum": true, + "elements": [ + { + "element": "LOCAL", + "_N": 1 }, { - "field_type": "int", - "field_name": "pipelineId", - "field_text": "int", - "_N": 3, - "field_local": true + "element": "REMOTE_STREAMING", + "_N": 2 }, { - "field_type": "int", - "field_name": "operatorId", - "field_text": "int", - "_N": 4, - "field_local": true + "element": "REMOTE_MATERIALIZED", + "_N": 3, + "_last": true + } + ] + }, + { + "class_name": "SortOrder", + "enum": true, + "elements": [ + { + "element": "ASC_NULLS_FIRST", + "_N": 1 }, { - "field_type": "PlanNodeId", - "field_name": "planNodeId", - "field_text": "PlanNodeId", - "_N": 5, - "field_local": true + "element": "ASC_NULLS_LAST", + "_N": 2 }, { - "field_type": "String", - "field_name": "operatorType", - "field_text": "String", - "_N": 6, - "field_local": true + "element": "DESC_NULLS_FIRST", + "_N": 3 }, { - "field_type": "long", - "field_name": "totalDrivers", - "field_text": "int64_t", - "_N": 7, + "element": "DESC_NULLS_LAST", + "_N": 4, + "_last": true + } + ] + }, + { + "class_name": "Ordering", + "struct": true, + "fields": [ + { + "field_type": "VariableReferenceExpression", + "field_name": "variable", + "field_text": "VariableReferenceExpression", + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "addInputCalls", - "field_text": "int64_t", - "_N": 8, + "field_type": "SortOrder", + "field_name": "sortOrder", + "field_text": "SortOrder", + "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "OrderingScheme", + "struct": true, + "fields": [ { - "field_type": "Duration", - "field_name": "addInputWall", - "field_text": "Duration", - "_N": 9, + "field_type": "List", + "field_name": "orderBy", + "field_text": "List", + "_N": 1, "field_local": true - }, + } + ] + }, + { + "class_name": "ExchangeNodeType", + "enum": true, + "elements": [ { - "field_type": "Duration", - "field_name": "addInputCpu", - "field_text": "Duration", - "_N": 10, - "field_local": true + "element": "GATHER", + "_N": 1 }, { - "field_type": "DataSize", - "field_name": "addInputAllocation", - "field_text": "DataSize", - "_N": 11, - "field_local": true + "element": "REPARTITION", + "_N": 2 }, { - "field_type": "DataSize", - "field_name": "rawInputDataSize", - "field_text": "DataSize", - "_N": 12, - "field_local": true - }, + "element": "REPLICATE", + "_N": 3 + } + ] + }, + { + "class_name": "ConnectorPartitioningHandle", + "field_name": "connectorPartitioningHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "field_type": "long", - "field_name": "rawInputPositions", - "field_text": "int64_t", - "_N": 13, - "field_local": true + "type": "SystemPartitioningHandle", + "name": "systemPartitioningHandle", + "key": "$remote", + "_N": 1 }, { - "field_type": "DataSize", - "field_name": "inputDataSize", - "field_text": "DataSize", - "_N": 14, - "field_local": true + "type": "HivePartitioningHandle", + "name": "hivePartitioningHandle", + "key": "hive", + "_N": 2 }, { - "field_type": "long", - "field_name": "inputPositions", - "field_text": "int64_t", - "_N": 15, - "field_local": true - }, + "type": "TpchPartitioningHandle", + "name": "tpchPartitioningHandle", + "key": "tpch", + "_N": 3, + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorPartitioningHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorPartitioningHandle\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"tpch\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorPartitioningHandle\");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "PartitioningHandle", + "struct": true, + "fields": [ { - "field_type": "double", - "field_name": "sumSquaredInputPositions", - "field_text": "double", - "_N": 16, + "field_type": "Optional", + "field_name": "connectorId", + "field_text": "ConnectorId", + "optional": true, + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "getOutputCalls", - "field_text": "int64_t", - "_N": 17, + "field_type": "Optional", + "field_name": "transactionHandle", + "field_text": "ConnectorTransactionHandle", + "optional": true, + "_N": 2, "field_local": true }, { - "field_type": "Duration", - "field_name": "getOutputWall", - "field_text": "Duration", - "_N": 18, - "field_local": true - }, + "field_type": "ConnectorPartitioningHandle", + "field_name": "connectorHandle", + "field_text": "ConnectorPartitioningHandle", + "_N": 3, + "field_local": true, + "optional": true + } + ] + }, + { + "class_name": "RowExpression", + "field_name": "rowExpression", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "field_type": "Duration", - "field_name": "getOutputCpu", - "field_text": "Duration", - "_N": 19, - "field_local": true + "type": "CallExpression", + "name": "callExpression", + "key": "call", + "_N": 1 }, { - "field_type": "DataSize", - "field_name": "getOutputAllocation", - "field_text": "DataSize", - "_N": 20, - "field_local": true + "type": "ConstantExpression", + "name": "constantExpression", + "key": "constant", + "_N": 2 }, { - "field_type": "DataSize", - "field_name": "outputDataSize", - "field_text": "DataSize", - "_N": 21, - "field_local": true + "type": "SpecialFormExpression", + "name": "specialFormExpression", + "key": "special", + "_N": 3 }, { - "field_type": "long", - "field_name": "outputPositions", - "field_text": "int64_t", - "_N": 22, - "field_local": true + "type": "LambdaDefinitionExpression", + "name": "lambdaDefinitionExpression", + "key": "lambda", + "_N": 4 }, { - "field_type": "DataSize", - "field_name": "physicalWrittenDataSize", - "field_text": "DataSize", - "_N": 23, - "field_local": true - }, + "type": "VariableReferenceExpression", + "name": "variableReferenceExpression", + "key": "variable", + "_N": 5, + "_last": true + } + ], + "fields": [ { - "field_type": "Duration", - "field_name": "additionalCpu", - "field_text": "Duration", - "_N": 24, - "field_local": true - }, + "field_type": "std::shared_ptr", + "field_name": "sourceLocation", + "field_text": "std::shared_ptr", + "_N": 1, + "field_local": true, + "last": true + } + ] + }, + { + "class_name": "Partitioning", + "struct": true, + "fields": [ { - "field_type": "Duration", - "field_name": "blockedWall", - "field_text": "Duration", - "_N": 25, + "field_type": "PartitioningHandle", + "field_name": "handle", + "field_text": "PartitioningHandle", + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "finishCalls", - "field_text": "int64_t", - "_N": 26, + "field_type": "List", + "field_name": "arguments", + "field_text": "List>", + "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "PartitioningScheme", + "struct": true, + "fields": [ { - "field_type": "Duration", - "field_name": "finishWall", - "field_text": "Duration", - "_N": 27, + "field_type": "Partitioning", + "field_name": "partitioning", + "field_text": "Partitioning", + "_N": 1, "field_local": true }, { - "field_type": "Duration", - "field_name": "finishCpu", - "field_text": "Duration", - "_N": 28, + "field_type": "List", + "field_name": "outputLayout", + "field_text": "List", + "_N": 2, "field_local": true }, { - "field_type": "DataSize", - "field_name": "finishAllocation", - "field_text": "DataSize", - "_N": 29, + "field_type": "Optional", + "field_name": "hashColumn", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 3, "field_local": true }, { - "field_type": "DataSize", - "field_name": "userMemoryReservation", - "field_text": "DataSize", - "_N": 30, + "field_type": "boolean", + "field_name": "replicateNullsAndAny", + "field_text": "bool", + "_N": 4, "field_local": true }, { - "field_type": "DataSize", - "field_name": "revocableMemoryReservation", - "field_text": "DataSize", - "_N": 31, + "field_type": "Optional", + "field_name": "bucketToPartition", + "field_text": "List", + "optional": true, + "_N": 5, "field_local": true - }, + } + ] + }, + { + "class_name": "ExchangeNode", + "struct": true, + "fields": [ { - "field_type": "DataSize", - "field_name": "systemMemoryReservation", - "field_text": "DataSize", - "_N": 32, - "field_local": true + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false }, { - "field_type": "DataSize", - "field_name": "peakUserMemoryReservation", - "field_text": "DataSize", - "_N": 33, + "field_type": "ExchangeNodeType", + "field_name": "type", + "field_text": "ExchangeNodeType", + "_N": 2, "field_local": true }, { - "field_type": "DataSize", - "field_name": "peakSystemMemoryReservation", - "field_text": "DataSize", - "_N": 34, + "field_type": "ExchangeNodeScope", + "field_name": "scope", + "field_text": "ExchangeNodeScope", + "_N": 3, "field_local": true }, { - "field_type": "DataSize", - "field_name": "peakTotalMemoryReservation", - "field_text": "DataSize", - "_N": 35, + "field_type": "PartitioningScheme", + "field_name": "partitioningScheme", + "field_text": "PartitioningScheme", + "_N": 4, "field_local": true }, { - "field_type": "DataSize", - "field_name": "spilledDataSize", - "field_text": "DataSize", - "_N": 36, + "field_type": "List", + "field_name": "sources", + "field_text": "List>", + "_N": 5, "field_local": true }, { - "field_type": "Optional", - "field_name": "blockedReason", - "field_text": "BlockedReason", - "optional": true, - "_N": 37, + "field_type": "List>", + "field_name": "inputs", + "field_text": "List>", + "_N": 6, "field_local": true }, { - "field_type": "OperatorInfo", - "field_name": "info", - "field_text": "OperatorInfo", - "_N": 38, + "field_type": "boolean", + "field_name": "ensureSourceOrdering", + "field_text": "bool", + "_N": 7, "field_local": true }, { - "field_type": "RuntimeStats", - "field_name": "runtimeStats", - "field_text": "RuntimeStats", - "_N": 39, + "field_type": "Optional", + "field_name": "orderingScheme", + "field_text": "OrderingScheme", + "optional": true, + "_N": 8, "field_local": true } - ] + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.ExchangeNode" }, { - "class_name": "DriverStats", + "class_name": "RemoteSourceNode", "struct": true, "fields": [ { - "field_type": "Lifespan", - "field_name": "lifespan", - "field_text": "Lifespan", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 1, - "field_local": true + "field_local": false }, { - "field_type": "DateTime", - "field_name": "createTime", - "field_text": "DateTime", + "field_type": "List", + "field_name": "sourceFragmentIds", + "field_text": "List", "_N": 2, "field_local": true }, { - "field_type": "DateTime", - "field_name": "startTime", - "field_text": "DateTime", + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "DateTime", - "field_name": "endTime", - "field_text": "DateTime", + "field_type": "boolean", + "field_name": "ensureSourceOrdering", + "field_text": "bool", "_N": 4, "field_local": true }, { - "field_type": "Duration", - "field_name": "queuedTime", - "field_text": "Duration", + "field_type": "Optional", + "field_name": "orderingScheme", + "field_text": "OrderingScheme", + "optional": true, "_N": 5, "field_local": true }, { - "field_type": "Duration", - "field_name": "elapsedTime", - "field_text": "Duration", + "field_type": "ExchangeNode.Type", + "field_name": "exchangeType", + "field_text": "ExchangeNodeType", "_N": 6, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.RemoteSourceNode" + }, + { + "class_name": "Parameter", + "struct": true, + "fields": [ { - "field_type": "DataSize", - "field_name": "userMemoryReservation", - "field_text": "DataSize", - "_N": 7, + "field_type": "String", + "field_name": "name", + "field_text": "String", + "_N": 1, "field_local": true }, { - "field_type": "DataSize", - "field_name": "revocableMemoryReservation", - "field_text": "DataSize", - "_N": 8, + "field_type": "TypeSignature", + "field_name": "type", + "field_text": "TypeSignature", + "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "Bound", + "enum": true, + "elements": [ { - "field_type": "DataSize", - "field_name": "systemMemoryReservation", - "field_text": "DataSize", - "_N": 9, - "field_local": true + "element": "BELOW", + "_N": 1 }, { - "field_type": "Duration", - "field_name": "totalScheduledTime", - "field_text": "Duration", - "_N": 10, - "field_local": true + "element": "EXACTLY", + "_N": 2 }, { - "field_type": "Duration", - "field_name": "totalCpuTime", - "field_text": "Duration", - "_N": 11, - "field_local": true - }, + "element": "ABOVE", + "_N": 3, + "_last": true + } + ] + }, + { + "class_name": "Block", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Block& p) {\n j = p.data;\n}\n\nvoid from_json(const json& j, Block& p) {\n p.data = std::string(j);\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct Block {\n std::string data;\n};\n\nvoid to_json(json& j, const Block& p);\n\nvoid from_json(const json& j, Block& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "Marker", + "struct": true, + "fields": [ { - "field_type": "Duration", - "field_name": "totalBlockedTime", - "field_text": "Duration", - "_N": 12, + "field_type": "Type", + "field_name": "type", + "field_text": "Type", + "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "fullyBlocked", - "field_text": "bool", - "_N": 13, + "field_type": "Optional", + "field_name": "valueBlock", + "field_text": "Block", + "optional": true, + "_N": 2, "field_local": true }, { - "field_type": "Set", - "field_name": "blockedReasons", - "field_text": "List", - "_N": 14, + "field_type": "Bound", + "field_name": "bound", + "field_text": "Bound", + "_N": 3, "field_local": true - }, + } + ] + }, + { + "class_name": "Range", + "struct": true, + "fields": [ { - "field_type": "DataSize", - "field_name": "totalAllocation", - "field_text": "DataSize", - "_N": 15, + "field_type": "Marker", + "field_name": "low", + "field_text": "Marker", + "_N": 1, "field_local": true }, { - "field_type": "DataSize", - "field_name": "rawInputDataSize", - "field_text": "DataSize", - "_N": 16, + "field_type": "Marker", + "field_name": "high", + "field_text": "Marker", + "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "SortedRangeSet", + "struct": true, + "fields": [ { - "field_type": "long", - "field_name": "rawInputPositions", - "field_text": "int64_t", - "_N": 17, + "field_type": "Type", + "field_name": "type", + "field_text": "Type", + "_N": 1, "field_local": true }, { - "field_type": "Duration", - "field_name": "rawInputReadTime", - "field_text": "Duration", - "_N": 18, + "field_type": "List", + "field_name": "ranges", + "field_text": "List", + "_N": 2, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "ValueSet", + "json_key": "sortable" + }, + { + "class_name": "TaskState", + "enum": true, + "elements": [ { - "field_type": "DataSize", - "field_name": "processedInputDataSize", - "field_text": "DataSize", - "_N": 19, - "field_local": true + "element": "PLANNED", + "_N": 1 }, { - "field_type": "long", - "field_name": "processedInputPositions", - "field_text": "int64_t", - "_N": 20, - "field_local": true + "element": "RUNNING", + "_N": 2 }, { - "field_type": "DataSize", - "field_name": "outputDataSize", - "field_text": "DataSize", - "_N": 21, - "field_local": true + "element": "FINISHED", + "_N": 3 }, { - "field_type": "long", - "field_name": "outputPositions", - "field_text": "int64_t", - "_N": 22, - "field_local": true + "element": "CANCELED", + "_N": 4 }, { - "field_type": "DataSize", - "field_name": "physicalWrittenDataSize", - "field_text": "DataSize", - "_N": 23, - "field_local": true + "element": "ABORTED", + "_N": 5 }, { - "field_type": "List", - "field_name": "operatorStats", - "field_text": "List", - "_N": 24, - "field_local": true + "element": "FAILED", + "_N": 6, + "_last": true } ] }, { - "class_name": "ConnectorTransactionHandle", - "field_name": "connectorTransactionHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + "class_name": "ErrorLocation", + "struct": true, + "fields": [ { - "type": "HiveTransactionHandle", - "name": "hiveTransactionHandle", - "key": "hive", - "_N": 1 + "field_type": "int", + "field_name": "lineNumber", + "field_text": "int", + "_N": 1, + "field_local": true }, { - "type": "RemoteTransactionHandle", - "name": "remoteTransactionHandle", - "key": "$remote", + "field_type": "int", + "field_name": "columnNumber", + "field_text": "int", "_N": 2, - "_last": true + "field_local": true } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTransactionHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorTransactionHandle ConnectorTransactionHandle\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTransactionHandle\");\n}\n} // namespace facebook::presto::protocol" + ] }, { - "class_name": "ConnectorSplit", - "field_name": "connectorSplit", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + "class_name": "ErrorCause", + "enum": true, + "elements": [ { - "type": "HiveSplit", - "name": "hiveSplit", - "key": "hive", + "element": "UNKNOWN", "_N": 1 }, { - "type": "RemoteSplit", - "name": "remoteSplit", - "key": "$remote", + "element": "LOW_PARTITION_COUNT", "_N": 2 }, { - "type": "EmptySplit", - "name": "emptySplit", - "key": "$empty", + "element": "EXCEEDS_BROADCAST_MEMORY_LIMIT", "_N": 3, "_last": true } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (type == \"$empty\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorSplit\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorSplit\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (type == \"$empty\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorSplit\");\n}\n} // namespace facebook::presto::protocol" + ] }, { - "class_name": "SplitContext", - "struct": true, - "fields": [ + "class_name": "HostAddress", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nusing HostAddress = std::string;\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "ErrorType", + "enum": true, + "elements": [ { - "field_type": "boolean", - "field_name": "cacheable", - "field_text": "bool", - "_N": 1, - "field_local": true + "element": "USER_ERROR", + "_N": 1 + }, + { + "element": "INTERNAL_ERROR", + "_N": 2 + }, + { + "element": "INSUFFICIENT_RESOURCES", + "_N": 3 + }, + { + "element": "EXTERNAL", + "_N": 4, + "_last": true } ] }, { - "class_name": "Split", + "class_name": "ErrorCode", "struct": true, "fields": [ { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", + "field_type": "int", + "field_name": "code", + "field_text": "int", "_N": 1, "field_local": true }, { - "field_type": "ConnectorTransactionHandle", - "field_name": "transactionHandle", - "field_text": "ConnectorTransactionHandle", + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 2, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "ConnectorSplit", - "field_name": "connectorSplit", - "field_text": "ConnectorSplit", + "field_type": "ErrorType", + "field_name": "type", + "field_text": "ErrorType", "_N": 3, - "field_local": true, - "optional": true - }, - { - "field_type": "Lifespan", - "field_name": "lifespan", - "field_text": "Lifespan", - "_N": 4, "field_local": true }, { - "field_type": "SplitContext", - "field_name": "splitContext", - "field_text": "SplitContext", - "_N": 5, + "field_type": "boolean", + "field_name": "retriable", + "field_text": "bool", + "_N": 4, "field_local": true } ] }, { - "class_name": "ConnectorMetadataUpdateHandle", - "field_name": "connectorMetadataUpdateHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ - { - "type": "HiveMetadataUpdateHandle", - "name": "hiveMetadataUpdateHandle", - "key": "hive", - "_N": 1, - "_last": true - } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorMetadataUpdateHandle\");\n}\n\nvoid from_json(\n const json& j,\n std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorMetadataUpdateHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorMetadataUpdateHandle\");\n}\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "MetadataUpdates", + "class_name": "ExecutionFailureInfo", "struct": true, "fields": [ { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", + "field_type": "String", + "field_name": "type", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "List", - "field_name": "metadataUpdates", - "field_text": "List>", + "field_type": "String", + "field_name": "message", + "field_text": "String", "_N": 2, "field_local": true - } - ] - }, - { - "class_name": "SourceLocation", - "struct": true, - "fields": [ + }, { - "field_type": "int", - "field_name": "line", - "field_text": "int", - "_N": 1, + "field_type": "ExecutionFailureInfo", + "field_name": "cause", + "field_text": "ExecutionFailureInfo", + "_N": 3, + "optional": true, "field_local": true }, { - "field_type": "int", - "field_name": "column", - "field_text": "int", - "_N": 2, + "field_type": "List", + "field_name": "suppressed", + "field_text": "List", + "_N": 4, "field_local": true - } - ] - }, - { - "class_name": "HiveBucketFilter", - "struct": true, - "fields": [ + }, { - "field_type": "Set", - "field_name": "bucketsToKeep", - "field_text": "List", - "_N": 1, + "field_type": "List", + "field_name": "stack", + "field_text": "List", + "_N": 5, "field_local": true - } - ] - }, - { - "class_name": "ValueSet", - "field_name": "valueSet", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + }, { - "type": "EquatableValueSet", - "name": "equatableValueSet", - "key": "equatable", - "_N": 1 + "field_type": "ErrorLocation", + "field_name": "errorLocation", + "field_text": "ErrorLocation", + "_N": 6, + "field_local": true }, { - "type": "SortedRangeSet", - "name": "sortedRangeSet", - "key": "sortable", - "_N": 2 + "field_type": "ErrorCode", + "field_name": "errorCode", + "field_text": "ErrorCode", + "_N": 7, + "field_local": true }, { - "type": "AllOrNoneValueSet", - "name": "allOrNoneValueSet", - "key": "allOrNone", - "_N": 3, - "_last": true + "field_type": "HostAddress", + "field_name": "remoteHost", + "field_text": "HostAddress", + "_N": 8, + "field_local": true + }, + { + "field_type": "ErrorCause", + "field_name": "errorCause", + "field_text": "ErrorCause", + "_N": 9, + "field_local": true } - ], - "fields": [] + ] }, { - "class_name": "Domain", + "class_name": "TaskStatus", "struct": true, "fields": [ { - "field_type": "ValueSet", - "field_name": "values", - "field_text": "ValueSet", + "field_type": "long", + "field_name": "taskInstanceIdLeastSignificantBits", + "field_text": "int64_t", "_N": 1, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "boolean", - "field_name": "nullAllowed", - "field_text": "bool", + "field_type": "long", + "field_name": "taskInstanceIdMostSignificantBits", + "field_text": "int64_t", "_N": 2, "field_local": true - } - ] - }, - { - "class_name": "TupleDomain", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\ntemplate \nstruct pointerDerefCompare {\n bool operator()(const std::shared_ptr& a, const std::shared_ptr& b)\n const {\n return *a < *b;\n }\n};\n\ntemplate \nstruct TupleDomain {\n std::shared_ptr> domains;\n};\n\ntemplate \nstruct TupleDomain> {\n std::shared_ptr, Domain, pointerDerefCompare>>\n domains;\n};\n\ntemplate \nstruct ColumnDomain {\n T column;\n Domain domain; // dependency\n};\n\n} // namespace facebook::presto::protocol\n\nnamespace nlohmann {\n\ntemplate \nstruct adl_serializer> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::ColumnDomain& p) {\n facebook::presto::protocol::to_json_key(\n j, \"column\", p.column, \"ColumnDomain\", \"T\", \"column\");\n facebook::presto::protocol::to_json_key(\n j, \"domain\", p.domain, \"ColumnDomain\", \"Domain\", \"domain\");\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::ColumnDomain& p) {\n facebook::presto::protocol::from_json_key(\n j, \"column\", p.column, \"ColumnDomain\", \"T\", \"column\");\n facebook::presto::protocol::from_json_key(\n j, \"domain\", p.domain, \"ColumnDomain\", \"Domain\", \"domain\");\n }\n};\n\ntemplate \nstruct adl_serializer> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::TupleDomain& tup) {\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>\n list;\n if (tup.domains != nullptr) {\n for (auto& el : *tup.domains) {\n facebook::presto::protocol::ColumnDomain domain;\n domain.column = el.first;\n domain.domain = el.second;\n list.push_back(domain);\n }\n }\n\n j[\"columnDomains\"] = list;\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::TupleDomain& tup) {\n if (j.count(\"columnDomains\") != 0U) {\n std::shared_ptr>\n map = std::make_shared<\n std::map>();\n\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>\n list = j.at(\"columnDomains\");\n for (const facebook::presto::protocol::ColumnDomain& value : list) {\n map->insert(std::make_pair(T(value.column), value.domain));\n }\n tup.domains = map;\n }\n }\n};\n\ntemplate \nstruct adl_serializer<\n facebook::presto::protocol::TupleDomain>> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::TupleDomain>& tup) {\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>>\n list;\n if (tup.domains != nullptr) {\n for (auto& el : *tup.domains) {\n facebook::presto::protocol::ColumnDomain> domain;\n domain.column = el.first;\n domain.domain = el.second;\n list.push_back(domain);\n }\n }\n\n j[\"columnDomains\"] = list;\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::TupleDomain>& tup) {\n if (j.count(\"columnDomains\") != 0U) {\n auto map = std::make_shared,\n facebook::presto::protocol::Domain,\n facebook::presto::protocol::pointerDerefCompare>>();\n\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>>\n list = j.at(\"columnDomains\");\n for (const facebook::presto::protocol::ColumnDomain>&\n value : list) {\n map->insert(\n std::make_pair(std::shared_ptr(value.column), value.domain));\n }\n tup.domains = map;\n }\n }\n};\n\n} // namespace nlohmann" - }, - { - "class_name": "RowExpression", - "field_name": "rowExpression", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + }, { - "type": "CallExpression", - "name": "callExpression", - "key": "call", - "_N": 1 + "field_type": "long", + "field_name": "version", + "field_text": "int64_t", + "_N": 3, + "field_local": true }, { - "type": "ConstantExpression", - "name": "constantExpression", - "key": "constant", - "_N": 2 + "field_type": "TaskState", + "field_name": "state", + "field_text": "TaskState", + "_N": 4, + "field_local": true }, { - "type": "SpecialFormExpression", - "name": "specialFormExpression", - "key": "special", - "_N": 3 + "field_type": "URI", + "field_name": "self", + "field_text": "URI", + "_N": 5, + "field_local": true }, { - "type": "LambdaDefinitionExpression", - "name": "lambdaDefinitionExpression", - "key": "lambda", - "_N": 4 + "field_type": "Set", + "field_name": "completedDriverGroups", + "field_text": "List", + "_N": 6, + "field_local": true }, { - "type": "VariableReferenceExpression", - "name": "variableReferenceExpression", - "key": "variable", - "_N": 5, - "_last": true - } - ], - "fields": [ + "field_type": "List", + "field_name": "failures", + "field_text": "List", + "_N": 7, + "field_local": true + }, { - "field_type": "std::shared_ptr", - "field_name": "sourceLocation", - "field_text": "std::shared_ptr", - "_N": 1, - "field_local": true, - "last": true - } - ] - }, - { - "class_name": "Form", - "enum": true, - "elements": [ + "field_type": "int", + "field_name": "queuedPartitionedDrivers", + "field_text": "int", + "_N": 8, + "field_local": true + }, { - "element": "IF", - "_N": 1 + "field_type": "int", + "field_name": "runningPartitionedDrivers", + "field_text": "int", + "_N": 9, + "field_local": true }, { - "element": "NULL_IF", - "_N": 2 + "field_type": "double", + "field_name": "outputBufferUtilization", + "field_text": "double", + "_N": 10, + "field_local": true }, { - "element": "SWITCH", - "_N": 3 + "field_type": "boolean", + "field_name": "outputBufferOverutilized", + "field_text": "bool", + "_N": 11, + "field_local": true }, { - "element": "WHEN", - "_N": 4 + "field_type": "long", + "field_name": "physicalWrittenDataSizeInBytes", + "field_text": "int64_t", + "_N": 12, + "field_local": true }, { - "element": "IS_NULL", - "_N": 5 + "field_type": "long", + "field_name": "memoryReservationInBytes", + "field_text": "int64_t", + "_N": 13, + "field_local": true }, { - "element": "COALESCE", - "_N": 6 + "field_type": "long", + "field_name": "systemMemoryReservationInBytes", + "field_text": "int64_t", + "_N": 14, + "field_local": true }, { - "element": "IN", - "_N": 7 + "field_type": "long", + "field_name": "peakNodeTotalMemoryReservationInBytes", + "field_text": "int64_t", + "_N": 15, + "field_local": true }, { - "element": "AND", - "_N": 8 + "field_type": "long", + "field_name": "fullGcCount", + "field_text": "int64_t", + "_N": 16, + "field_local": true }, { - "element": "OR", - "_N": 9 + "field_type": "long", + "field_name": "fullGcTimeInMillis", + "field_text": "int64_t", + "_N": 17, + "field_local": true }, { - "element": "DEREFERENCE", - "_N": 10 + "field_type": "long", + "field_name": "totalCpuTimeInNanos", + "field_text": "int64_t", + "_N": 18, + "field_local": true }, { - "element": "ROW_CONSTRUCTOR", - "_N": 11 + "field_type": "long", + "field_name": "taskAgeInMillis", + "field_text": "int64_t", + "_N": 19, + "field_local": true }, { - "element": "BIND", - "_N": 12, - "_last": true + "field_type": "long", + "field_name": "queuedPartitionedSplitsWeight", + "field_text": "int64_t", + "_N": 20, + "field_local": true + }, + { + "field_type": "long", + "field_name": "runningPartitionedSplitsWeight", + "field_text": "int64_t", + "_N": 21, + "field_local": true } ] }, { - "class_name": "SpecialFormExpression", + "class_name": "HiveBucketFilter", "struct": true, "fields": [ { - "field_type": "Optional", - "field_name": "sourceLocation", - "field_text": "SourceLocation", - "optional": true, + "field_type": "Set", + "field_name": "bucketsToKeep", + "field_text": "List", "_N": 1, - "field_local": false - }, + "field_local": true + } + ] + }, + { + "class_name": "EmptySplit", + "struct": true, + "fields": [ { - "field_type": "Form", - "field_name": "form", - "field_text": "Form", - "_N": 2, + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", + "_N": 1, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "ConnectorSplit", + "json_key": "$empty" + }, + { + "class_name": "Location", + "struct": true, + "fields": [ { - "field_type": "Type", - "field_name": "returnType", - "field_text": "Type", - "_N": 3, + "field_type": "String", + "field_name": "location", + "field_text": "String", + "_N": 1, + "field_local": true + } + ] + }, + { + "class_name": "RemoteSplit", + "struct": true, + "fields": [ + { + "field_type": "Location", + "field_name": "location", + "field_text": "Location", + "_N": 1, "field_local": true }, { - "field_type": "List", - "field_name": "arguments", - "field_text": "List>", - "_N": 4, + "field_type": "TaskId", + "field_name": "remoteSourceTaskId", + "field_text": "TaskId", + "_N": 2, "field_local": true } ], "subclass": true, - "super_class": "RowExpression", - "json_key": "special" + "super_class": "ConnectorSplit", + "json_key": "$remote" }, { - "class_name": "VariableReferenceExpression", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct VariableReferenceExpression : RowExpression {\n String name;\n Type type; // dependency\n\n VariableReferenceExpression() noexcept;\n explicit VariableReferenceExpression(const String& str) {\n _type = \"variable\";\n\n std::vector> parts;\n\n folly::split(\"<\", str, parts);\n name = parts[0];\n type = parts[1].substr(0, parts[1].length() - 1);\n }\n\n bool operator<(const VariableReferenceExpression& o) const {\n if (name == o.name) {\n return type < o.type;\n }\n\n return name < o.name;\n }\n};\n\nvoid to_json(json& j, const VariableReferenceExpression& p);\nvoid from_json(const json& j, VariableReferenceExpression& p);\n\nstd::string json_map_key(\n const facebook::presto::protocol::VariableReferenceExpression& p);\n\n} // namespace facebook::presto::protocol", + "class_name": "VariableStatsEstimate", "struct": true, "fields": [ { - "field_type": "Optional", - "field_name": "sourceLocation", - "field_text": "SourceLocation", - "optional": true, + "field_type": "double", + "field_name": "lowValue", + "field_text": "double", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "String", - "field_name": "name", - "field_text": "String", + "field_type": "double", + "field_name": "highValue", + "field_text": "double", "_N": 2, "field_local": true }, { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", + "field_type": "double", + "field_name": "nullsFraction", + "field_text": "double", "_N": 3, "field_local": true + }, + { + "field_type": "double", + "field_name": "averageRowSize", + "field_text": "double", + "_N": 4, + "field_local": true + }, + { + "field_type": "double", + "field_name": "distinctValuesCount", + "field_text": "double", + "_N": 5, + "field_local": true } - ], - "subclass": true, - "super_class": "RowExpression", - "json_key": "variable" + ] }, { - "class_name": "EquiJoinClause", + "class_name": "PlanNodeStatsEstimate", "struct": true, "fields": [ { - "field_type": "VariableReferenceExpression", - "field_name": "left", - "field_text": "VariableReferenceExpression", + "field_type": "double", + "field_name": "outputRowCount", + "field_text": "double", "_N": 1, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "right", - "field_text": "VariableReferenceExpression", + "field_type": "double", + "field_name": "totalSize", + "field_text": "double", "_N": 2, "field_local": true + }, + { + "field_type": "boolean", + "field_name": "confident", + "field_text": "bool", + "_N": 3, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "variableStatistics", + "field_text": "Map", + "_N": 4, + "field_local": true } ] }, { - "class_name": "SortOrder", + "class_name": "BoundType", "enum": true, "elements": [ { - "element": "ASC_NULLS_FIRST", + "element": "UNBOUNDED_PRECEDING", "_N": 1 }, { - "element": "ASC_NULLS_LAST", + "element": "PRECEDING", "_N": 2 }, { - "element": "DESC_NULLS_FIRST", + "element": "CURRENT_ROW", "_N": 3 }, { - "element": "DESC_NULLS_LAST", - "_N": 4, + "element": "FOLLOWING", + "_N": 4 + }, + { + "element": "UNBOUNDED_FOLLOWING", + "_N": 5, "_last": true } ] }, { - "class_name": "Ordering", - "struct": true, - "fields": [ + "class_name": "WindowType", + "enum": true, + "elements": [ { - "field_type": "VariableReferenceExpression", - "field_name": "variable", - "field_text": "VariableReferenceExpression", - "_N": 1, - "field_local": true + "element": "RANGE", + "_N": 1 }, { - "field_type": "SortOrder", - "field_name": "sortOrder", - "field_text": "SortOrder", - "_N": 2, - "field_local": true + "element": "ROWS", + "_N": 2 } ] }, { - "class_name": "OrderingScheme", + "class_name": "Frame", "struct": true, "fields": [ { - "field_type": "List", - "field_name": "orderBy", - "field_text": "List", + "field_type": "WindowType", + "field_name": "type", + "field_text": "WindowType", "_N": 1, "field_local": true + }, + { + "field_type": "BoundType", + "field_name": "startType", + "field_text": "BoundType", + "_N": 2, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "startValue", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 3, + "field_local": true + }, + { + "field_type": "BoundType", + "field_name": "endType", + "field_text": "BoundType", + "_N": 4, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "endValue", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 5, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "originalStartValue", + "field_text": "String", + "optional": true, + "_N": 6, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "originalEndValue", + "field_text": "String", + "optional": true, + "_N": 7, + "field_local": true } ] }, @@ -1426,46 +1686,96 @@ ] }, { - "class_name": "ColumnType", - "enum": true, - "elements": [ + "class_name": "FunctionHandle", + "field_name": "functionHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "element": "PARTITION_KEY", - "_N": 1 + "type": "BuiltInFunctionHandle", + "name": "builtInFunctionHandle", + "key": "$static", + "_N": 1, + "_last": true + } + ], + "fields": [] + }, + { + "class_name": "CallExpression", + "struct": true, + "fields": [ + { + "field_type": "Optional", + "field_name": "sourceLocation", + "field_text": "SourceLocation", + "optional": true, + "_N": 1, + "field_local": false }, { - "element": "REGULAR", - "_N": 2 + "field_type": "String", + "field_name": "displayName", + "field_text": "String", + "_N": 2, + "field_local": true }, { - "element": "SYNTHESIZED", - "_N": 3 + "field_type": "FunctionHandle", + "field_name": "functionHandle", + "field_text": "FunctionHandle", + "_N": 3, + "field_local": true, + "optional": true }, { - "element": "AGGREGATED", + "field_type": "Type", + "field_name": "returnType", + "field_text": "Type", "_N": 4, - "_last": true + "field_local": true + }, + { + "field_type": "List", + "field_name": "arguments", + "field_text": "List>", + "_N": 5, + "field_local": true } - ] + ], + "subclass": true, + "super_class": "RowExpression", + "json_key": "call" }, { - "class_name": "FunctionHandle", - "field_name": "functionHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + "class_name": "Function", + "struct": true, + "fields": [ { - "type": "BuiltInFunctionHandle", - "name": "builtInFunctionHandle", - "key": "$static", + "field_type": "CallExpression", + "field_name": "functionCall", + "field_text": "CallExpression", "_N": 1, - "_last": true + "field_local": true + }, + { + "field_type": "Frame", + "field_name": "frame", + "field_text": "Frame", + "_N": 2, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "ignoreNulls", + "field_text": "bool", + "_N": 3, + "field_local": true } - ], - "fields": [] + ] }, { - "class_name": "CallExpression", + "class_name": "WindowNode", "struct": true, "fields": [ { @@ -1474,41 +1784,86 @@ "field_text": "SourceLocation", "optional": true, "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "String", - "field_name": "displayName", - "field_text": "String", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 2, - "field_local": true + "field_local": false }, { - "field_type": "FunctionHandle", - "field_name": "functionHandle", - "field_text": "FunctionHandle", + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", "_N": 3, "field_local": true, "optional": true }, { - "field_type": "Type", - "field_name": "returnType", - "field_text": "Type", + "field_type": "Specification", + "field_name": "specification", + "field_text": "Specification", "_N": 4, "field_local": true }, { - "field_type": "List", - "field_name": "arguments", - "field_text": "List>", + "field_type": "Map", + "field_name": "windowFunctions", + "field_text": "Map", "_N": 5, "field_local": true + }, + { + "field_type": "Optional", + "field_name": "hashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 6, + "field_local": true + }, + { + "field_type": "Set", + "field_name": "prePartitionedInputs", + "field_text": "List", + "_N": 7, + "field_local": true + }, + { + "field_type": "int", + "field_name": "preSortedOrderPrefix", + "field_text": "int", + "_N": 8, + "field_local": true } ], "subclass": true, - "super_class": "RowExpression", - "json_key": "call" + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.WindowNode" + }, + { + "class_name": "ColumnType", + "enum": true, + "elements": [ + { + "element": "PARTITION_KEY", + "_N": 1 + }, + { + "element": "REGULAR", + "_N": 2 + }, + { + "element": "SYNTHESIZED", + "_N": 3 + }, + { + "element": "AGGREGATED", + "_N": 4, + "_last": true + } + ] }, { "class_name": "Aggregation", @@ -1639,891 +1994,666 @@ "json_key": "hive" }, { - "class_name": "BucketFunctionType", - "enum": true, - "elements": [ - { - "element": "HIVE_COMPATIBLE", - "_N": 1 - }, - { - "element": "PRESTO_NATIVE", - "_N": 2, - "_last": true - } - ] + "class_name": "OperatorInfo", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const OperatorInfo& p) {}\nvoid from_json(const json& j, OperatorInfo& p) {}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct OperatorInfo {};\nvoid to_json(json& j, const OperatorInfo& p);\nvoid from_json(const json& j, OperatorInfo& p);\n} // namespace facebook::presto::protocol" }, { - "class_name": "Order", + "class_name": "BlockedReason", "enum": true, "elements": [ { - "element": "ASCENDING", - "_N": 1 - }, - { - "element": "DESCENDING", - "_N": 2, + "element": "WAITING_FOR_MEMORY", + "_N": 1, "_last": true } ] }, { - "class_name": "SortingColumn", - "struct": true, - "fields": [ - { - "field_type": "String", - "field_name": "columnName", - "field_text": "String", - "_N": 1, - "field_local": true - }, - { - "field_type": "Order", - "field_name": "order", - "field_text": "Order", - "_N": 2, - "field_local": true - } - ] + "class_name": "Duration", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Duration& p) {\n j = p.toString();\n}\n\nvoid from_json(const json& j, Duration& p) {\n p = Duration(std::string(j));\n}\n\nstd::ostream& operator<<(std::ostream& os, const Duration& d) {\n return os << d.toString();\n}\n\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n\nnamespace facebook::presto::protocol {\n\nstd::ostream& operator<<(std::ostream& os, const Duration& d);\n\nvoid to_json(json& j, const Duration& p);\nvoid from_json(const json& j, Duration& p);\n\n} // namespace facebook::presto::protocol" }, { - "class_name": "HiveBucketProperty", + "class_name": "DataSize", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(nlohmann::json& j, const DataSize& p) {\n j = p.toString();\n}\n\nvoid from_json(const nlohmann::json& j, DataSize& p) {\n p = DataSize(std::string(j));\n}\n\nstd::ostream& operator<<(std::ostream& os, const DataSize& d) {\n return os << d.toString();\n}\n\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstd::ostream& operator<<(std::ostream& os, const DataSize& d);\n\nvoid to_json(nlohmann::json& j, const DataSize& p);\nvoid from_json(const nlohmann::json& j, DataSize& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "OperatorStats", "struct": true, "fields": [ { - "field_type": "List", - "field_name": "bucketedBy", - "field_text": "List", + "field_type": "int", + "field_name": "stageId", + "field_text": "int", "_N": 1, "field_local": true }, { "field_type": "int", - "field_name": "bucketCount", + "field_name": "stageExecutionId", "field_text": "int", "_N": 2, "field_local": true }, { - "field_type": "List", - "field_name": "sortedBy", - "field_text": "List", + "field_type": "int", + "field_name": "pipelineId", + "field_text": "int", "_N": 3, "field_local": true }, { - "field_type": "BucketFunctionType", - "field_name": "bucketFunctionType", - "field_text": "BucketFunctionType", + "field_type": "int", + "field_name": "operatorId", + "field_text": "int", "_N": 4, "field_local": true }, { - "field_type": "Optional>", - "field_name": "types", - "field_text": "List", - "optional": true, + "field_type": "PlanNodeId", + "field_name": "planNodeId", + "field_text": "PlanNodeId", "_N": 5, "field_local": true - } - ] - }, - { - "class_name": "DwrfEncryptionMetadata", - "struct": true, - "fields": [ + }, { - "field_type": "Map", - "field_name": "fieldToKeyData", - "field_text": "Map", - "_N": 1, + "field_type": "String", + "field_name": "operatorType", + "field_text": "String", + "_N": 6, "field_local": true }, { - "field_type": "Map", - "field_name": "extraMetadata", - "field_text": "Map", - "_N": 2, + "field_type": "long", + "field_name": "totalDrivers", + "field_text": "int64_t", + "_N": 7, "field_local": true }, { - "field_type": "String", - "field_name": "encryptionAlgorithm", - "field_text": "String", - "_N": 3, + "field_type": "long", + "field_name": "addInputCalls", + "field_text": "int64_t", + "_N": 8, "field_local": true }, { - "field_type": "String", - "field_name": "encryptionProvider", - "field_text": "String", - "_N": 4, + "field_type": "Duration", + "field_name": "addInputWall", + "field_text": "Duration", + "_N": 9, "field_local": true - } - ] - }, - { - "class_name": "EncryptionInformation", - "struct": true, - "fields": [ + }, { - "field_type": "Optional", - "field_name": "dwrfEncryptionMetadata", - "field_text": "DwrfEncryptionMetadata", - "optional": true, - "_N": 1, + "field_type": "Duration", + "field_name": "addInputCpu", + "field_text": "Duration", + "_N": 10, "field_local": true - } - ] - }, - { - "class_name": "HiveCompressionCodec", - "enum": true, - "elements": [ + }, { - "element": "NONE", - "_N": 1 + "field_type": "DataSize", + "field_name": "addInputAllocation", + "field_text": "DataSize", + "_N": 11, + "field_local": true + }, + { + "field_type": "DataSize", + "field_name": "rawInputDataSize", + "field_text": "DataSize", + "_N": 12, + "field_local": true + }, + { + "field_type": "long", + "field_name": "rawInputPositions", + "field_text": "int64_t", + "_N": 13, + "field_local": true + }, + { + "field_type": "DataSize", + "field_name": "inputDataSize", + "field_text": "DataSize", + "_N": 14, + "field_local": true }, { - "element": "SNAPPY", - "_N": 2 + "field_type": "long", + "field_name": "inputPositions", + "field_text": "int64_t", + "_N": 15, + "field_local": true }, { - "element": "GZIP", - "_N": 3 + "field_type": "double", + "field_name": "sumSquaredInputPositions", + "field_text": "double", + "_N": 16, + "field_local": true }, { - "element": "LZ4", - "_N": 4 + "field_type": "long", + "field_name": "getOutputCalls", + "field_text": "int64_t", + "_N": 17, + "field_local": true }, { - "element": "ZSTD", - "_N": 5, - "_last": true - } - ] - }, - { - "class_name": "HiveStorageFormat", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const HiveStorageFormat& p) {\n throw ParseError(\"Not implemented\");\n}\n\nstatic const std::pair HiveStorageFormat_enum_table[] =\n { // NOLINT: cert-err58-cpp\n {HiveStorageFormat::ORC, \"ORC\"},\n {HiveStorageFormat::DWRF, \"DWRF\"},\n {HiveStorageFormat::PARQUET, \"PARQUET\"},\n {HiveStorageFormat::AVRO, \"AVRO\"},\n {HiveStorageFormat::RCBINARY, \"RCBINARY\"},\n {HiveStorageFormat::RCTEXT, \"RCTEXT\"},\n {HiveStorageFormat::SEQUENCEFILE, \"SEQUENCEFILE\"},\n {HiveStorageFormat::JSON, \"JSON\"},\n {HiveStorageFormat::TEXTFILE, \"TEXTFILE\"},\n {HiveStorageFormat::CSV, \"CSV\"},\n {HiveStorageFormat::PAGEFILE, \"PAGEFILE\"}};\n\nvoid from_json(const json& j, HiveStorageFormat& e) {\n static_assert(\n std::is_enum::value,\n \"HiveStorageFormat must be an enum!\");\n const auto* it = std::find_if(\n std::begin(HiveStorageFormat_enum_table),\n std::end(HiveStorageFormat_enum_table),\n [&j](const std::pair& ej_pair) -> bool {\n return ej_pair.second == j;\n });\n e = ((it != std::end(HiveStorageFormat_enum_table))\n ? it\n : std::begin(HiveStorageFormat_enum_table))\n ->first;\n}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nenum class HiveStorageFormat {\n ORC,\n DWRF,\n PARQUET,\n AVRO,\n RCBINARY,\n RCTEXT,\n SEQUENCEFILE,\n JSON,\n TEXTFILE,\n CSV,\n PAGEFILE\n};\n\nvoid to_json(json& j, const HiveStorageFormat& p);\nvoid from_json(const json& j, HiveStorageFormat& p);\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "WriteMode", - "enum": true, - "elements": [ - { - "element": "STAGE_AND_MOVE_TO_TARGET_DIRECTORY", - "_N": 1 + "field_type": "Duration", + "field_name": "getOutputWall", + "field_text": "Duration", + "_N": 18, + "field_local": true }, { - "element": "DIRECT_TO_TARGET_NEW_DIRECTORY", - "_N": 2 + "field_type": "Duration", + "field_name": "getOutputCpu", + "field_text": "Duration", + "_N": 19, + "field_local": true }, { - "element": "DIRECT_TO_TARGET_EXISTING_DIRECTORY", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "TableType", - "enum": true, - "elements": [ + "field_type": "DataSize", + "field_name": "getOutputAllocation", + "field_text": "DataSize", + "_N": 20, + "field_local": true + }, { - "element": "NEW", - "_N": 1 + "field_type": "DataSize", + "field_name": "outputDataSize", + "field_text": "DataSize", + "_N": 21, + "field_local": true }, { - "element": "EXISTING", - "_N": 2 + "field_type": "long", + "field_name": "outputPositions", + "field_text": "int64_t", + "_N": 22, + "field_local": true }, { - "element": "TEMPORARY", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "LocationHandle", - "struct": true, - "fields": [ + "field_type": "DataSize", + "field_name": "physicalWrittenDataSize", + "field_text": "DataSize", + "_N": 23, + "field_local": true + }, { - "field_type": "String", - "field_name": "targetPath", - "field_text": "String", - "_N": 1, + "field_type": "Duration", + "field_name": "additionalCpu", + "field_text": "Duration", + "_N": 24, "field_local": true }, { - "field_type": "String", - "field_name": "writePath", - "field_text": "String", - "_N": 2, + "field_type": "Duration", + "field_name": "blockedWall", + "field_text": "Duration", + "_N": 25, "field_local": true }, { - "field_type": "Optional", - "field_name": "tempPath", - "field_text": "String", - "optional": true, - "_N": 3, + "field_type": "long", + "field_name": "finishCalls", + "field_text": "int64_t", + "_N": 26, "field_local": true }, { - "field_type": "TableType", - "field_name": "tableType", - "field_text": "TableType", - "_N": 4, + "field_type": "Duration", + "field_name": "finishWall", + "field_text": "Duration", + "_N": 27, "field_local": true }, { - "field_type": "WriteMode", - "field_name": "writeMode", - "field_text": "WriteMode", - "_N": 5, + "field_type": "Duration", + "field_name": "finishCpu", + "field_text": "Duration", + "_N": 28, "field_local": true - } - ] - }, - { - "class_name": "Column", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nstruct Column {\n String name;\n String type;\n\n Column() = default;\n explicit Column(const String& str) {\n name = str;\n }\n};\n\nvoid to_json(json& j, const Column& p);\nvoid from_json(const json& j, Column& p);\n\n} // namespace facebook::presto::protocol", - "struct": true, - "fields": [ + }, { - "field_type": "String", - "field_name": "name", - "field_text": "String", - "_N": 1, + "field_type": "DataSize", + "field_name": "finishAllocation", + "field_text": "DataSize", + "_N": 29, "field_local": true }, { - "field_type": "String", - "field_name": "type", - "field_text": "String", - "_N": 2, + "field_type": "DataSize", + "field_name": "userMemoryReservation", + "field_text": "DataSize", + "_N": 30, "field_local": true - } - ] - }, - { - "class_name": "StorageFormat", - "struct": true, - "fields": [ + }, { - "field_type": "String", - "field_name": "serDe", - "field_text": "String", - "_N": 1, + "field_type": "DataSize", + "field_name": "revocableMemoryReservation", + "field_text": "DataSize", + "_N": 31, "field_local": true }, { - "field_type": "String", - "field_name": "inputFormat", - "field_text": "String", - "_N": 2, + "field_type": "DataSize", + "field_name": "systemMemoryReservation", + "field_text": "DataSize", + "_N": 32, "field_local": true }, { - "field_type": "String", - "field_name": "outputFormat", - "field_text": "String", - "_N": 3, + "field_type": "DataSize", + "field_name": "peakUserMemoryReservation", + "field_text": "DataSize", + "_N": 33, "field_local": true - } - ] - }, - { - "class_name": "Storage", - "struct": true, - "fields": [ + }, { - "field_type": "StorageFormat", - "field_name": "storageFormat", - "field_text": "StorageFormat", - "_N": 1, + "field_type": "DataSize", + "field_name": "peakSystemMemoryReservation", + "field_text": "DataSize", + "_N": 34, "field_local": true }, { - "field_type": "String", - "field_name": "location", - "field_text": "String", - "_N": 2, + "field_type": "DataSize", + "field_name": "peakTotalMemoryReservation", + "field_text": "DataSize", + "_N": 35, "field_local": true }, { - "field_type": "Optional", - "field_name": "bucketProperty", - "field_text": "HiveBucketProperty", - "optional": true, - "_N": 3, + "field_type": "DataSize", + "field_name": "spilledDataSize", + "field_text": "DataSize", + "_N": 36, "field_local": true }, { - "field_type": "boolean", - "field_name": "skewed", - "field_text": "bool", - "_N": 4, + "field_type": "Optional", + "field_name": "blockedReason", + "field_text": "BlockedReason", + "optional": true, + "_N": 37, "field_local": true }, { - "field_type": "Map", - "field_name": "serdeParameters", - "field_text": "Map", - "_N": 5, + "field_type": "OperatorInfo", + "field_name": "info", + "field_text": "OperatorInfo", + "_N": 38, "field_local": true }, { - "field_type": "Map", - "field_name": "parameters", - "field_text": "Map", - "_N": 6, + "field_type": "RuntimeStats", + "field_name": "runtimeStats", + "field_text": "RuntimeStats", + "_N": 39, "field_local": true } ] }, { - "class_name": "PrestoTableType", + "class_name": "BufferState", "enum": true, "elements": [ { - "element": "MANAGED_TABLE", + "element": "OPEN", "_N": 1 }, { - "element": "EXTERNAL_TABLE", + "element": "NO_MORE_BUFFERS", "_N": 2 }, { - "element": "VIRTUAL_VIEW", + "element": "NO_MORE_PAGES", "_N": 3 }, { - "element": "MATERIALIZED_VIEW", + "element": "FLUSHING", "_N": 4 }, { - "element": "TEMPORARY_TABLE", + "element": "FINISHED", "_N": 5 }, { - "element": "OTHER", + "element": "FAILED", "_N": 6, "_last": true } ] }, { - "class_name": "Table", + "class_name": "PageBufferInfo", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "databaseName", - "field_text": "String", + "field_type": "int", + "field_name": "partition", + "field_text": "int", "_N": 1, "field_local": true }, { - "field_type": "String", - "field_name": "tableName", - "field_text": "String", + "field_type": "long", + "field_name": "bufferedPages", + "field_text": "int64_t", "_N": 2, "field_local": true }, { - "field_type": "String", - "field_name": "owner", - "field_text": "String", + "field_type": "long", + "field_name": "bufferedBytes", + "field_text": "int64_t", "_N": 3, "field_local": true }, { - "field_type": "PrestoTableType", - "field_name": "tableType", - "field_text": "PrestoTableType", + "field_type": "long", + "field_name": "rowsAdded", + "field_text": "int64_t", "_N": 4, "field_local": true }, { - "field_type": "Storage", - "field_name": "storage", - "field_text": "Storage", + "field_type": "long", + "field_name": "pagesAdded", + "field_text": "int64_t", "_N": 5, "field_local": true - }, - { - "field_type": "List", - "field_name": "dataColumns", - "field_text": "List", - "_N": 6, - "field_local": true - }, - { - "field_type": "List", - "field_name": "partitionColumns", - "field_text": "List", - "_N": 7, - "field_local": true - }, - { - "field_type": "Map", - "field_name": "parameters", - "field_text": "Map", - "_N": 8, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "viewOriginalText", - "field_text": "String", - "optional": true, - "_N": 9, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "viewExpandedText", - "field_text": "String", - "optional": true, - "_N": 10, - "field_local": true - } - ] - }, - { - "class_name": "SchemaTableName", - "struct": true, - "fields": [ - { - "field_type": "String", - "field_name": "schema", - "field_text": "String", - "_N": 1, - "field_local": true - }, - { - "field_type": "String", - "field_name": "table", - "field_text": "String", - "_N": 2, - "field_local": true } ] }, { - "class_name": "HivePageSinkMetadata", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const HivePageSinkMetadata& p) {\n j = json::object();\n to_json_key(\n j,\n \"schemaTableName\",\n p.schemaTableName,\n \"HivePageSinkMetadata\",\n \"SchemaTableName\",\n \"schemaTableName\");\n to_json_key(j, \"table\", p.table, \"HivePageSinkMetadata\", \"Table\", \"table\");\n}\n\nvoid from_json(const json& j, HivePageSinkMetadata& p) {\n from_json_key(\n j,\n \"schemaTableName\",\n p.schemaTableName,\n \"HivePageSinkMetadata\",\n \"SchemaTableName\",\n \"schemaTableName\");\n from_json_key(j, \"table\", p.table, \"HivePageSinkMetadata\", \"Table\", \"table\");\n}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// dependency Table\n// dependency SchemaTableName\n\nnamespace facebook::presto::protocol {\n\nstruct HivePageSinkMetadata {\n SchemaTableName schemaTableName = {};\n std::shared_ptr
table = {};\n // TODO Add modifiedPartitions\n};\nvoid to_json(json& j, const HivePageSinkMetadata& p);\nvoid from_json(const json& j, HivePageSinkMetadata& p);\n\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "HiveInsertTableHandle", + "class_name": "BufferInfo", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "schemaName", - "field_text": "String", + "field_type": "OutputBufferId", + "field_name": "bufferId", + "field_text": "OutputBufferId", "_N": 1, "field_local": true }, { - "field_type": "String", - "field_name": "tableName", - "field_text": "String", + "field_type": "boolean", + "field_name": "finished", + "field_text": "bool", "_N": 2, "field_local": true }, { - "field_type": "List", - "field_name": "inputColumns", - "field_text": "List", + "field_type": "int", + "field_name": "bufferedPages", + "field_text": "int", "_N": 3, "field_local": true }, { - "field_type": "HivePageSinkMetadata", - "field_name": "pageSinkMetadata", - "field_text": "HivePageSinkMetadata", + "field_type": "long", + "field_name": "pagesSent", + "field_text": "int64_t", "_N": 4, "field_local": true }, { - "field_type": "LocationHandle", - "field_name": "locationHandle", - "field_text": "LocationHandle", + "field_type": "PageBufferInfo", + "field_name": "pageBufferInfo", + "field_text": "PageBufferInfo", "_N": 5, "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketProperty", - "field_text": "HiveBucketProperty", - "optional": true, - "_N": 6, - "field_local": true - }, - { - "field_type": "List", - "field_name": "preferredOrderingColumns", - "field_text": "List", - "_N": 7, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "tableStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 8, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "partitionStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 9, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "actualStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 10, - "field_local": true - }, - { - "field_type": "HiveCompressionCodec", - "field_name": "compressionCodec", - "field_text": "HiveCompressionCodec", - "_N": 11, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "encryptionInformation", - "field_text": "EncryptionInformation", - "optional": true, - "_N": 12, - "field_local": true } - ], - "subclass": true, - "super_class": "ConnectorInsertTableHandle", - "json_key": "hive" + ] }, { - "class_name": "NodeVersion", + "class_name": "OutputBufferInfo", "struct": true, "fields": [ { "field_type": "String", - "field_name": "version", + "field_name": "type", "field_text": "String", "_N": 1, "field_local": true - } - ] - }, - { - "class_name": "ServerInfo", - "struct": true, - "fields": [ - { - "field_type": "NodeVersion", - "field_name": "nodeVersion", - "field_text": "NodeVersion", - "_N": 1, - "field_local": true }, { - "field_type": "String", - "field_name": "environment", - "field_text": "String", + "field_type": "BufferState", + "field_name": "state", + "field_text": "BufferState", "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "coordinator", - "field_text": "bool", - "_N": 3, - "field_local": true - }, - { - "field_type": "boolean", - "field_name": "starting", - "field_text": "bool", - "_N": 4, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "uptime", - "field_text": "Duration", - "optional": true, - "_N": 5, - "field_local": true - } - ] - }, - { - "class_name": "PlanNode", - "field_name": "planNode", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ - { - "type": "AggregationNode", - "name": "aggregationNode", - "key": ".AggregationNode", - "_N": 1 - }, - { - "type": "GroupIdNode", - "name": "groupIdNode", - "key": "com.facebook.presto.sql.planner.plan.GroupIdNode", - "_N": 2 + "field_type": "boolean", + "field_name": "canAddBuffers", + "field_text": "bool", + "_N": 3, + "field_local": true }, { - "type": "DistinctLimitNode", - "name": "distinctLimitNode", - "key": ".DistinctLimitNode", - "_N": 3 + "field_type": "boolean", + "field_name": "canAddPages", + "field_text": "bool", + "_N": 4, + "field_local": true }, { - "type": "EnforceSingleRowNode", - "name": "enforceSingleRowNode", - "key": "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode", - "_N": 4 + "field_type": "long", + "field_name": "totalBufferedBytes", + "field_text": "int64_t", + "_N": 5, + "field_local": true }, { - "type": "ExchangeNode", - "name": "exchangeNode", - "key": "com.facebook.presto.sql.planner.plan.ExchangeNode", - "_N": 5 + "field_type": "long", + "field_name": "totalBufferedPages", + "field_text": "int64_t", + "_N": 6, + "field_local": true }, { - "type": "FilterNode", - "name": "filterNode", - "key": ".FilterNode", - "_N": 6 + "field_type": "long", + "field_name": "totalRowsSent", + "field_text": "int64_t", + "_N": 7, + "field_local": true }, { - "type": "JoinNode", - "name": "joinNode", - "key": "com.facebook.presto.sql.planner.plan.JoinNode", - "_N": 7 + "field_type": "long", + "field_name": "totalPagesSent", + "field_text": "int64_t", + "_N": 8, + "field_local": true }, { - "type": "LimitNode", - "name": "limitNode", - "key": ".LimitNode", - "_N": 8 - }, + "field_type": "List", + "field_name": "buffers", + "field_text": "List", + "_N": 9, + "field_local": true + } + ] + }, + { + "class_name": "DriverStats", + "struct": true, + "fields": [ { - "type": "SortNode", - "name": "sortNode", - "key": "com.facebook.presto.sql.planner.plan.SortNode", - "_N": 9 + "field_type": "Lifespan", + "field_name": "lifespan", + "field_text": "Lifespan", + "_N": 1, + "field_local": true }, { - "type": "OutputNode", - "name": "outputNode", - "key": "com.facebook.presto.sql.planner.plan.OutputNode", - "_N": 10 + "field_type": "DateTime", + "field_name": "createTime", + "field_text": "DateTime", + "_N": 2, + "field_local": true }, { - "type": "ProjectNode", - "name": "projectNode", - "key": ".ProjectNode", - "_N": 11 + "field_type": "DateTime", + "field_name": "startTime", + "field_text": "DateTime", + "_N": 3, + "field_local": true }, { - "type": "RowNumberNode", - "name": "rowNumberNode", - "key": "com.facebook.presto.sql.planner.plan.RowNumberNode", - "_N": 12 + "field_type": "DateTime", + "field_name": "endTime", + "field_text": "DateTime", + "_N": 4, + "field_local": true }, { - "type": "RemoteSourceNode", - "name": "remoteSourceNode", - "key": "com.facebook.presto.sql.planner.plan.RemoteSourceNode", - "_N": 13 + "field_type": "Duration", + "field_name": "queuedTime", + "field_text": "Duration", + "_N": 5, + "field_local": true }, { - "type": "SemiJoinNode", - "name": "semiJoinNode", - "key": "com.facebook.presto.sql.planner.plan.SemiJoinNode", - "_N": 14 + "field_type": "Duration", + "field_name": "elapsedTime", + "field_text": "Duration", + "_N": 6, + "field_local": true }, { - "type": "TableScanNode", - "name": "tableScanNode", - "key": ".TableScanNode", - "_N": 15 + "field_type": "DataSize", + "field_name": "userMemoryReservation", + "field_text": "DataSize", + "_N": 7, + "field_local": true }, { - "type": "TableWriterNode", - "name": "tableWriterNode", - "key": "com.facebook.presto.sql.planner.plan.TableWriterNode", - "_N": 16 + "field_type": "DataSize", + "field_name": "revocableMemoryReservation", + "field_text": "DataSize", + "_N": 8, + "field_local": true }, { - "type": "TopNNode", - "name": "topNNode", - "key": ".TopNNode", - "_N": 17 + "field_type": "DataSize", + "field_name": "systemMemoryReservation", + "field_text": "DataSize", + "_N": 9, + "field_local": true }, { - "type": "UnnestNode", - "name": "unnestNode", - "key": "com.facebook.presto.sql.planner.plan.UnnestNode", - "_N": 18 + "field_type": "Duration", + "field_name": "totalScheduledTime", + "field_text": "Duration", + "_N": 10, + "field_local": true }, { - "type": "ValuesNode", - "name": "valuesNode", - "key": ".ValuesNode", - "_N": 19 + "field_type": "Duration", + "field_name": "totalCpuTime", + "field_text": "Duration", + "_N": 11, + "field_local": true }, { - "type": "AssignUniqueId", - "name": "assignUniqueId", - "key": "com.facebook.presto.sql.planner.plan.AssignUniqueId", - "_N": 20 + "field_type": "Duration", + "field_name": "totalBlockedTime", + "field_text": "Duration", + "_N": 12, + "field_local": true }, { - "type": "MergeJoinNode", - "name": "mergeJoinNode", - "key": "com.facebook.presto.sql.planner.plan.MergeJoinNode", - "_N": 21 + "field_type": "boolean", + "field_name": "fullyBlocked", + "field_text": "bool", + "_N": 13, + "field_local": true }, { - "type": "WindowNode", - "name": "windowNode", - "key": "com.facebook.presto.sql.planner.plan.WindowNode", - "_N": 22, - "_last": true - } - ], - "fields": [ - { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": true, - "last": true - } - ] - }, - { - "class_name": "Step", - "enum": true, - "elements": [ - { - "element": "SINGLE", - "_N": 1 + "field_type": "Set", + "field_name": "blockedReasons", + "field_text": "List", + "_N": 14, + "field_local": true }, { - "element": "PARTIAL", - "_N": 2 + "field_type": "DataSize", + "field_name": "totalAllocation", + "field_text": "DataSize", + "_N": 15, + "field_local": true }, { - "element": "FINAL", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "TopNNode", - "struct": true, - "fields": [ - { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": false + "field_type": "DataSize", + "field_name": "rawInputDataSize", + "field_text": "DataSize", + "_N": 16, + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", - "_N": 2, - "field_local": true, - "optional": true + "field_type": "long", + "field_name": "rawInputPositions", + "field_text": "int64_t", + "_N": 17, + "field_local": true }, { - "field_type": "long", - "field_name": "count", - "field_text": "int64_t", - "_N": 3, + "field_type": "Duration", + "field_name": "rawInputReadTime", + "field_text": "Duration", + "_N": 18, "field_local": true }, { - "field_type": "OrderingScheme", - "field_name": "orderingScheme", - "field_text": "OrderingScheme", - "_N": 4, + "field_type": "DataSize", + "field_name": "processedInputDataSize", + "field_text": "DataSize", + "_N": 19, "field_local": true }, { - "field_type": "Step", - "field_name": "step", - "field_text": "Step", - "_N": 5, + "field_type": "long", + "field_name": "processedInputPositions", + "field_text": "int64_t", + "_N": 20, "field_local": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": ".TopNNode" - }, - { - "class_name": "HiveMetadataUpdateHandle", - "struct": true, - "fields": [ + }, { - "field_type": "UUID", - "field_name": "requestId", - "field_text": "UUID", - "_N": 1, + "field_type": "DataSize", + "field_name": "outputDataSize", + "field_text": "DataSize", + "_N": 21, "field_local": true }, { - "field_type": "SchemaTableName", - "field_name": "schemaTableName", - "field_text": "SchemaTableName", - "_N": 2, + "field_type": "long", + "field_name": "outputPositions", + "field_text": "int64_t", + "_N": 22, "field_local": true }, { - "field_type": "Optional", - "field_name": "partitionName", - "field_text": "String", - "optional": true, - "_N": 3, + "field_type": "DataSize", + "field_name": "physicalWrittenDataSize", + "field_text": "DataSize", + "_N": 23, "field_local": true }, { - "field_type": "Optional", - "field_name": "fileName", - "field_text": "String", - "optional": true, - "_N": 4, + "field_type": "List", + "field_name": "operatorStats", + "field_text": "List", + "_N": 24, "field_local": true } - ], - "subclass": true, - "super_class": "ConnectorMetadataUpdateHandle", - "json_key": "hive" + ] }, { "class_name": "DistributionSnapshot", @@ -3168,385 +3298,227 @@ "field_type": "List", "field_name": "pipelines", "field_text": "List", - "_N": 40, - "field_local": true - }, - { - "field_type": "RuntimeStats", - "field_name": "runtimeStats", - "field_text": "RuntimeStats", - "_N": 41, - "field_local": true - } - ] - }, - { - "class_name": "ConnectorTableLayoutHandle", - "field_name": "connectorTableLayoutHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ - { - "type": "HiveTableLayoutHandle", - "name": "hiveTableLayoutHandle", - "key": "hive", - "_N": 1, - "_last": true - } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableLayoutHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorTableLayoutHandle ConnectorTableLayoutHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableLayoutHandle\");\n}\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "ConnectorTableHandle", - "field_name": "connectorTableHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ - { - "type": "HiveTableHandle", - "name": "hiveTableHandle", - "key": "hive", - "_N": 1, - "_last": true - } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) + \" ConnectorTableHandle ConnectorTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableHandle\");\n}\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "TableHandle", - "struct": true, - "fields": [ - { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", - "_N": 1, - "field_local": true - }, - { - "field_type": "ConnectorTableHandle", - "field_name": "connectorHandle", - "field_text": "ConnectorTableHandle", - "_N": 2, - "field_local": true, - "optional": true - }, - { - "field_type": "ConnectorTransactionHandle", - "field_name": "transaction", - "field_text": "ConnectorTransactionHandle", - "_N": 3, - "field_local": true, - "optional": true - }, - { - "field_type": "Optional", - "field_name": "connectorTableLayout", - "field_text": "ConnectorTableLayoutHandle", - "optional": true, - "_N": 4, - "field_local": true - } - ] - }, - { - "class_name": "DeleteHandle", - "struct": true, - "fields": [ - { - "field_type": "TableHandle", - "field_name": "handle", - "field_text": "TableHandle", - "_N": 1, - "field_local": true - }, - { - "field_type": "SchemaTableName", - "field_name": "schemaTableName", - "field_text": "SchemaTableName", - "_N": 2, - "field_local": true - } - ], - "subclass": true, - "super_class": "ExecutionWriterTarget", - "json_key": "DeleteHandle" - }, - { - "class_name": "ExchangeNodeType", - "enum": true, - "elements": [ - { - "element": "GATHER", - "_N": 1 - }, - { - "element": "REPARTITION", - "_N": 2 - }, - { - "element": "REPLICATE", - "_N": 3 - } - ] - }, - { - "class_name": "ExchangeNodeScope", - "enum": true, - "elements": [ - { - "element": "LOCAL", - "_N": 1 - }, - { - "element": "REMOTE_STREAMING", - "_N": 2 - }, - { - "element": "REMOTE_MATERIALIZED", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "ConnectorPartitioningHandle", - "field_name": "connectorPartitioningHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ - { - "type": "SystemPartitioningHandle", - "name": "systemPartitioningHandle", - "key": "$remote", - "_N": 1 - }, - { - "type": "HivePartitioningHandle", - "name": "hivePartitioningHandle", - "key": "hive", - "_N": 2, - "_last": true - } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (type == \"$remote\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorPartitioningHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorPartitioningHandle\");\n }\n\n if (type == \"$remote\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorPartitioningHandle\");\n}\n} // namespace facebook::presto::protocol" - }, - { - "class_name": "PartitioningHandle", - "struct": true, - "fields": [ - { - "field_type": "Optional", - "field_name": "connectorId", - "field_text": "ConnectorId", - "optional": true, - "_N": 1, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "transactionHandle", - "field_text": "ConnectorTransactionHandle", - "optional": true, - "_N": 2, + "_N": 40, "field_local": true }, { - "field_type": "ConnectorPartitioningHandle", - "field_name": "connectorHandle", - "field_text": "ConnectorPartitioningHandle", - "_N": 3, - "field_local": true, - "optional": true + "field_type": "RuntimeStats", + "field_name": "runtimeStats", + "field_text": "RuntimeStats", + "_N": 41, + "field_local": true } ] }, { - "class_name": "Partitioning", - "struct": true, - "fields": [ + "class_name": "ConnectorMetadataUpdateHandle", + "field_name": "connectorMetadataUpdateHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "field_type": "PartitioningHandle", - "field_name": "handle", - "field_text": "PartitioningHandle", + "type": "HiveMetadataUpdateHandle", + "name": "hiveMetadataUpdateHandle", + "key": "hive", "_N": 1, - "field_local": true - }, - { - "field_type": "List", - "field_name": "arguments", - "field_text": "List>", - "_N": 2, - "field_local": true + "_last": true } - ] + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorMetadataUpdateHandle\");\n}\n\nvoid from_json(\n const json& j,\n std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ConnectorMetadataUpdateHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorMetadataUpdateHandle\");\n}\n} // namespace facebook::presto::protocol" }, { - "class_name": "PartitioningScheme", + "class_name": "MetadataUpdates", "struct": true, "fields": [ { - "field_type": "Partitioning", - "field_name": "partitioning", - "field_text": "Partitioning", + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", "_N": 1, "field_local": true }, { - "field_type": "List", - "field_name": "outputLayout", - "field_text": "List", + "field_type": "List", + "field_name": "metadataUpdates", + "field_text": "List>", "_N": 2, "field_local": true - }, - { - "field_type": "Optional", - "field_name": "hashColumn", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 3, - "field_local": true - }, - { - "field_type": "boolean", - "field_name": "replicateNullsAndAny", - "field_text": "bool", - "_N": 4, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketToPartition", - "field_text": "List", - "optional": true, - "_N": 5, - "field_local": true } ] }, { - "class_name": "ExchangeNode", + "class_name": "TaskInfo", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "TaskId", + "field_name": "taskId", + "field_text": "TaskId", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "ExchangeNodeType", - "field_name": "type", - "field_text": "ExchangeNodeType", + "field_type": "TaskStatus", + "field_name": "taskStatus", + "field_text": "TaskStatus", "_N": 2, "field_local": true }, { - "field_type": "ExchangeNodeScope", - "field_name": "scope", - "field_text": "ExchangeNodeScope", + "field_type": "DateTime", + "field_name": "lastHeartbeat", + "field_text": "DateTime", "_N": 3, "field_local": true }, { - "field_type": "PartitioningScheme", - "field_name": "partitioningScheme", - "field_text": "PartitioningScheme", + "field_type": "OutputBufferInfo", + "field_name": "outputBuffers", + "field_text": "OutputBufferInfo", "_N": 4, "field_local": true }, { - "field_type": "List", - "field_name": "sources", - "field_text": "List>", + "field_type": "Set", + "field_name": "noMoreSplits", + "field_text": "List", "_N": 5, "field_local": true }, { - "field_type": "List>", - "field_name": "inputs", - "field_text": "List>", + "field_type": "TaskStats", + "field_name": "stats", + "field_text": "TaskStats", "_N": 6, "field_local": true }, { "field_type": "boolean", - "field_name": "ensureSourceOrdering", + "field_name": "needsPlan", "field_text": "bool", "_N": 7, "field_local": true }, { - "field_type": "Optional", - "field_name": "orderingScheme", - "field_text": "OrderingScheme", - "optional": true, + "field_type": "MetadataUpdates", + "field_name": "metadataUpdates", + "field_text": "MetadataUpdates", "_N": 8, "field_local": true + }, + { + "field_type": "String", + "field_name": "nodeId", + "field_text": "String", + "_N": 9, + "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.ExchangeNode" + ] }, { - "class_name": "RemoteSourceNode", + "class_name": "RemoteTransactionHandle", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "Optional", + "field_name": "dummy", + "field_text": "String", + "optional": true, "_N": 1, - "field_local": false + "field_local": true, + "last": true + } + ], + "subclass": true, + "super_class": "ConnectorTransactionHandle", + "json_key": "$remote" + }, + { + "class_name": "ConnectorTableLayoutHandle", + "field_name": "connectorTableLayoutHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ + { + "type": "HiveTableLayoutHandle", + "name": "hiveTableLayoutHandle", + "key": "hive", + "_N": 1 }, { - "field_type": "List", - "field_name": "sourceFragmentIds", - "field_text": "List", + "type": "TpchTableLayoutHandle", + "name": "tpchTableLayoutHandle", + "key": "tpch", "_N": 2, - "field_local": true + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableLayoutHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorTableLayoutHandle ConnectorTableLayoutHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableLayoutHandle\");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "ConnectorTableHandle", + "field_name": "connectorTableHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ + { + "type": "HiveTableHandle", + "name": "hiveTableHandle", + "key": "hive", + "_N": 1 }, { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", - "_N": 3, + "type": "TpchTableHandle", + "name": "tpchTableHandle", + "key": "tpch", + "_N": 2, + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableHandle\");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) + \" ConnectorTableHandle ConnectorTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n auto k = std::make_shared();\n j.get_to(*k);\n p = k;\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorTableHandle\");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "TableHandle", + "struct": true, + "fields": [ + { + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", + "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "ensureSourceOrdering", - "field_text": "bool", - "_N": 4, - "field_local": true + "field_type": "ConnectorTableHandle", + "field_name": "connectorHandle", + "field_text": "ConnectorTableHandle", + "_N": 2, + "field_local": true, + "optional": true }, { - "field_type": "Optional", - "field_name": "orderingScheme", - "field_text": "OrderingScheme", - "optional": true, - "_N": 5, - "field_local": true + "field_type": "ConnectorTransactionHandle", + "field_name": "transaction", + "field_text": "ConnectorTransactionHandle", + "_N": 3, + "field_local": true, + "optional": true }, { - "field_type": "ExchangeNode.Type", - "field_name": "exchangeType", - "field_text": "ExchangeNodeType", - "_N": 6, + "field_type": "Optional", + "field_name": "connectorTableLayout", + "field_text": "ConnectorTableLayoutHandle", + "optional": true, + "_N": 4, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.RemoteSourceNode" + ] }, { - "class_name": "FilterNode", + "class_name": "DeleteScanInfo", "struct": true, "fields": [ { @@ -3554,337 +3526,270 @@ "field_name": "id", "field_text": "PlanNodeId", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", - "_N": 2, - "field_local": true, - "optional": true - }, + "field_type": "TableHandle", + "field_name": "tableHandle", + "field_text": "TableHandle", + "_N": 2, + "field_local": true + } + ] + }, + { + "class_name": "Language", + "struct": true, + "fields": [ { - "field_type": "RowExpression", - "field_name": "predicate", - "field_text": "RowExpression", - "_N": 3, - "field_local": true, - "optional": true + "field_type": "String", + "field_name": "language", + "field_text": "String", + "_N": 1, + "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": ".FilterNode" + ] }, { - "class_name": "DistributionType", + "class_name": "NullCallClause", "enum": true, "elements": [ { - "element": "PARTITIONED", + "element": "RETURNS_NULL_ON_NULL_INPUT", "_N": 1 }, { - "element": "REPLICATED", + "element": "CALLED_ON_NULL_INPUT", "_N": 2, "_last": true } ] }, { - "class_name": "JoinNodeType", + "class_name": "Determinism", "enum": true, "elements": [ { - "element": "INNER", + "element": "DETERMINISTIC", "_N": 1 }, { - "element": "LEFT", + "element": "NOT_DETERMINISTIC", "_N": 2 - }, - { - "element": "RIGHT", - "_N": 3 - }, - { - "element": "FULL", - "_N": 4, - "_last": true } ] }, { - "class_name": "JoinNode", + "class_name": "RoutineCharacteristics", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": false - }, - { - "field_type": "JoinNodeType", - "field_name": "type", - "field_text": "JoinNodeType", - "_N": 2, - "field_local": true - }, - { - "field_type": "PlanNode", - "field_name": "left", - "field_text": "PlanNode", - "_N": 3, - "field_local": true, - "optional": true - }, - { - "field_type": "PlanNode", - "field_name": "right", - "field_text": "PlanNode", - "_N": 4, - "field_local": true, - "optional": true - }, - { - "field_type": "List", - "field_name": "criteria", - "field_text": "List", - "_N": 5, - "field_local": true - }, - { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", - "_N": 6, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "filter", - "field_text": "std::shared_ptr", + "field_type": "Optional", + "field_name": "language", + "field_text": "Language", "optional": true, - "_N": 7, + "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "leftHashVariable", - "field_text": "VariableReferenceExpression", + "field_type": "Optional", + "field_name": "determinism", + "field_text": "Determinism", "optional": true, - "_N": 8, + "_N": 2, "field_local": true }, { - "field_type": "Optional", - "field_name": "rightHashVariable", - "field_text": "VariableReferenceExpression", + "field_type": "Optional", + "field_name": "nullCallClause", + "field_text": "NullCallClause", "optional": true, - "_N": 9, + "_N": 3, "field_local": true - }, + } + ] + }, + { + "class_name": "MemoryAllocation", + "struct": true, + "fields": [ { - "field_type": "Optional", - "field_name": "distributionType", - "field_text": "DistributionType", - "optional": true, - "_N": 10, + "field_type": "String", + "field_name": "tag", + "field_text": "String", + "_N": 1, "field_local": true }, { - "field_type": "Map", - "field_name": "dynamicFilters", - "field_text": "Map", - "_N": 11, + "field_type": "long", + "field_name": "allocation", + "field_text": "int64_t", + "_N": 2, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.JoinNode" + ] }, { - "class_name": "MergeJoinNode", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct MergeJoinNode : public PlanNode {\n MergeJoinNode() noexcept;\n PlanNodeId id = {};\n // JoinNodeType is referenced as JoinNode.Type in Presto\n // Since presto_cpp codegen can't nicely handle inner class references\n // So a special hard-coded template is required here\n JoinNodeType type = {};\n std::shared_ptr left = {};\n std::shared_ptr right = {};\n // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto\n List criteria = {};\n List outputVariables = {};\n std::shared_ptr> filter = {};\n std::shared_ptr leftHashVariable = {};\n std::shared_ptr rightHashVariable = {};\n};\nvoid to_json(json& j, const MergeJoinNode& p);\nvoid from_json(const json& j, MergeJoinNode& p);\n} // namespace facebook::presto::protocol", + "class_name": "MemoryPoolInfo", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "long", + "field_name": "maxBytes", + "field_text": "int64_t", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "JoinNode.Type", - "field_name": "type", - "field_text": "JoinNode.Type", + "field_type": "long", + "field_name": "reservedBytes", + "field_text": "int64_t", "_N": 2, "field_local": true }, { - "field_type": "PlanNode", - "field_name": "left", - "field_text": "PlanNode", + "field_type": "long", + "field_name": "reservedRevocableBytes", + "field_text": "int64_t", "_N": 3, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "right", - "field_text": "PlanNode", + "field_type": "Map", + "field_name": "queryMemoryReservations", + "field_text": "Map", "_N": 4, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "List", - "field_name": "criteria", - "field_text": "List", + "field_type": "Map>", + "field_name": "queryMemoryAllocations", + "field_text": "Map>", "_N": 5, "field_local": true }, { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", + "field_type": "Map", + "field_name": "queryMemoryRevocableReservations", + "field_text": "Map", "_N": 6, "field_local": true - }, - { - "field_type": "Optional", - "field_name": "filter", - "field_text": "std::shared_ptr", - "optional": true, - "_N": 7, - "field_local": true - }, + } + ] + }, + { + "class_name": "MemoryInfo", + "struct": true, + "fields": [ { - "field_type": "Optional", - "field_name": "leftHashVariable", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 8, + "field_type": "DataSize", + "field_name": "totalNodeMemory", + "field_text": "DataSize", + "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "rightHashVariable", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 9, + "field_type": "Map", + "field_name": "pools", + "field_text": "Map", + "_N": 2, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.MergeJoinNode" + ] }, { - "class_name": "SystemPartitioning", - "enum": true, - "elements": [ - { - "element": "SINGLE", - "_N": 1 - }, - { - "element": "FIXED", - "_N": 2 - }, + "class_name": "PlanCostEstimate", + "struct": true, + "fields": [ { - "element": "SOURCE", - "_N": 3 + "field_type": "double", + "field_name": "cpuCost", + "field_text": "double", + "_N": 1, + "field_local": true }, { - "element": "SCALED", - "_N": 4 + "field_type": "double", + "field_name": "maxMemory", + "field_text": "double", + "_N": 2, + "field_local": true }, { - "element": "COORDINATOR_ONLY", - "_N": 5 + "field_type": "double", + "field_name": "maxMemoryWhenOutputting", + "field_text": "double", + "_N": 3, + "field_local": true }, { - "element": "ARBITRARY", - "_N": 6, - "_last": true + "field_type": "double", + "field_name": "networkCost", + "field_text": "double", + "_N": 4, + "field_local": true } ] }, { - "class_name": "SystemPartitionFunction", - "enum": true, - "elements": [ - { - "element": "SINGLE", - "_N": 1 - }, - { - "element": "HASH", - "_N": 2 - }, - { - "element": "ROUND_ROBIN", - "_N": 3 - }, + "class_name": "StatsAndCosts", + "struct": true, + "fields": [ { - "element": "BROADCAST", - "_N": 4 + "field_type": "Map", + "field_name": "stats", + "field_text": "Map", + "_N": 1, + "field_local": true }, { - "element": "UNKNOWN", - "_N": 5, - "_last": true + "field_type": "Map", + "field_name": "costs", + "field_text": "Map", + "_N": 2, + "field_local": true } ] }, { - "class_name": "SystemPartitioningHandle", + "class_name": "SchemaTableName", "struct": true, "fields": [ { - "field_type": "SystemPartitioning", - "field_name": "partitioning", - "field_text": "SystemPartitioning", + "field_type": "String", + "field_name": "schema", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "SystemPartitionFunction", - "field_name": "function", - "field_text": "SystemPartitionFunction", + "field_type": "String", + "field_name": "table", + "field_text": "String", "_N": 2, "field_local": true } - ], - "subclass": true, - "super_class": "ConnectorPartitioningHandle", - "json_key": "$remote" + ] }, { - "class_name": "ConnectorOutputTableHandle", - "field_name": "connectorOutputTableHandle", + "class_name": "ConnectorInsertTableHandle", + "field_name": "connectorInsertTableHandle", "abstract": true, "super_class": "JsonEncodedSubclass", "subclasses": [ { - "type": "HiveOutputTableHandle", - "name": "hiveOutputTableHandle", + "type": "HiveInsertTableHandle", + "name": "hiveInsertTableHandle", "key": "hive", "_N": 1, "_last": true } ], "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorOutputTableHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorOutputTableHandle ConnectorOutputTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k =\n std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorOutputTableHandle \");\n}\n} // namespace facebook::presto::protocol" + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorInsertTableHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorInsertTableHandle ConnectorInsertTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k =\n std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorInsertTableHandle \");\n}\n} // namespace facebook::presto::protocol" }, { - "class_name": "OutputTableHandle", + "class_name": "InsertTableHandle", "struct": true, "fields": [ { @@ -3903,9 +3808,9 @@ "optional": true }, { - "field_type": "ConnectorOutputTableHandle", + "field_type": "ConnectorInsertTableHandle", "field_name": "connectorHandle", - "field_text": "ConnectorOutputTableHandle", + "field_text": "ConnectorInsertTableHandle", "_N": 3, "field_local": true, "optional": true @@ -3913,13 +3818,13 @@ ] }, { - "class_name": "CreateHandle", + "class_name": "InsertHandle", "struct": true, "fields": [ { - "field_type": "OutputTableHandle", + "field_type": "InsertTableHandle", "field_name": "handle", - "field_text": "OutputTableHandle", + "field_text": "InsertTableHandle", "_N": 1, "field_local": true }, @@ -3933,176 +3838,106 @@ ], "subclass": true, "super_class": "ExecutionWriterTarget", - "json_key": "CreateHandle" + "json_key": "InsertHandle" }, { - "class_name": "BucketConversion", - "struct": true, - "fields": [ + "class_name": "Order", + "enum": true, + "elements": [ { - "field_type": "int", - "field_name": "tableBucketCount", - "field_text": "int", - "_N": 1, - "field_local": true + "element": "ASCENDING", + "_N": 1 }, { - "field_type": "int", - "field_name": "partitionBucketCount", - "field_text": "int", + "element": "DESCENDING", "_N": 2, - "field_local": true - }, - { - "field_type": "List", - "field_name": "bucketColumnHandles", - "field_text": "List", - "_N": 3, - "field_local": true + "_last": true } ] }, { - "class_name": "HiveOutputTableHandle", + "class_name": "SortingColumn", "struct": true, "fields": [ { "field_type": "String", - "field_name": "schemaName", + "field_name": "columnName", "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "String", - "field_name": "tableName", - "field_text": "String", + "field_type": "Order", + "field_name": "order", + "field_text": "Order", "_N": 2, "field_local": true - }, - { - "field_type": "List", - "field_name": "inputColumns", - "field_text": "List", - "_N": 3, - "field_local": true - }, - { - "field_type": "HivePageSinkMetadata", - "field_name": "pageSinkMetadata", - "field_text": "HivePageSinkMetadata", - "_N": 4, - "field_local": true - }, - { - "field_type": "LocationHandle", - "field_name": "locationHandle", - "field_text": "LocationHandle", - "_N": 5, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "tableStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 6, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "partitionStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 7, - "field_local": true - }, - { - "field_type": "HiveStorageFormat", - "field_name": "actualStorageFormat", - "field_text": "HiveStorageFormat", - "_N": 8, - "field_local": true - }, - { - "field_type": "HiveCompressionCodec", - "field_name": "compressionCodec", - "field_text": "HiveCompressionCodec", - "_N": 9, - "field_local": true - }, - { - "field_type": "List", - "field_name": "partitionedBy", - "field_text": "List", - "_N": 10, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketProperty", - "field_text": "HiveBucketProperty", - "optional": true, - "_N": 11, - "field_local": true - }, - { - "field_type": "List", - "field_name": "preferredOrderingColumns", - "field_text": "List", - "_N": 12, - "field_local": true - }, - { - "field_type": "String", - "field_name": "tableOwner", - "field_text": "String", - "_N": 13, - "field_local": true - }, + } + ] + }, + { + "class_name": "ColumnHandle", + "field_name": "columnHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "comparable": true, + "subclasses": [ { - "field_type": "Map", - "field_name": "additionalTableParameters", - "field_text": "Map", - "_N": 14, - "field_local": true + "type": "HiveColumnHandle", + "name": "hiveColumnHandle", + "key": "hive", + "_N": 1 }, { - "field_type": "Optional", - "field_name": "encryptionInformation", - "field_text": "EncryptionInformation", - "optional": true, - "_N": 15, - "field_local": true + "type": "TpchColumnHandle", + "name": "tpchColumnHandle", + "key": "tpch", + "_N": 2, + "_last": true } ], - "subclass": true, - "super_class": "ConnectorOutputTableHandle", - "json_key": "hive" + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ColumnHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ColumnHandle ColumnHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k = std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n if (getConnectorKey(type) == \"tpch\") {\n std::shared_ptr k = std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ColumnHandle \");\n}\n} // namespace facebook::presto::protocol" }, { - "class_name": "AllOrNoneValueSet", + "class_name": "TableScanNode", "struct": true, "fields": [ { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 1, + "field_local": false + }, + { + "field_type": "TableHandle", + "field_name": "table", + "field_text": "TableHandle", + "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "all", - "field_text": "bool", - "_N": 2, + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", + "_N": 3, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "assignments", + "field_text": "Map>", + "_N": 4, "field_local": true } ], "subclass": true, - "super_class": "ValueSet", - "json_key": "allOrNone" + "super_class": "PlanNode", + "json_key": ".TableScanNode" }, { - "class_name": "DistinctLimitNode", + "class_name": "FilterNode", "struct": true, "fields": [ { @@ -4121,303 +3956,282 @@ "optional": true }, { - "field_type": "long", - "field_name": "limit", - "field_text": "int64_t", + "field_type": "RowExpression", + "field_name": "predicate", + "field_text": "RowExpression", "_N": 3, - "field_local": true - }, - { - "field_type": "boolean", - "field_name": "partial", - "field_text": "bool", - "_N": 4, - "field_local": true - }, - { - "field_type": "List", - "field_name": "distinctVariables", - "field_text": "List", - "_N": 5, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "hashVariable", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 6, - "field_local": true + "field_local": true, + "optional": true } ], "subclass": true, "super_class": "PlanNode", - "json_key": ".DistinctLimitNode" + "json_key": ".FilterNode" }, { - "class_name": "ColumnHandle", - "field_name": "columnHandle", + "class_name": "ExecutionWriterTarget", + "field_name": "executionWriterTarget", "abstract": true, "super_class": "JsonEncodedSubclass", - "comparable": true, "subclasses": [ { - "type": "HiveColumnHandle", - "name": "hiveColumnHandle", - "key": "hive", - "_N": 1, + "type": "CreateHandle", + "name": "createHandle", + "key": "CreateHandle", + "_N": 1 + }, + { + "type": "InsertHandle", + "name": "insertHandle", + "key": "InsertHandle", + "_N": 2 + }, + { + "type": "DeleteHandle", + "name": "deleteHandle", + "key": "DeleteHandle", + "_N": 3, "_last": true } ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ColumnHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(std::string(e.what()) + \" ColumnHandle ColumnHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k = std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ColumnHandle \");\n}\n} // namespace facebook::presto::protocol" + "fields": [] }, { - "class_name": "HiveBucketHandle", + "class_name": "AnalyzeTableHandle", "struct": true, "fields": [ { - "field_type": "List", - "field_name": "columns", - "field_text": "List", + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", "_N": 1, "field_local": true }, { - "field_type": "int", - "field_name": "tableBucketCount", - "field_text": "int", + "field_type": "ConnectorTransactionHandle", + "field_name": "transactionHandle", + "field_text": "ConnectorTransactionHandle", "_N": 2, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "int", - "field_name": "readBucketCount", - "field_text": "int", + "field_type": "ConnectorTableHandle", + "field_name": "connectorHandle", + "field_text": "ConnectorTableHandle", "_N": 3, - "field_local": true + "field_local": true, + "optional": true } ] }, { - "class_name": "HiveTableLayoutHandle", + "class_name": "TableWriteInfo", "struct": true, "fields": [ { - "field_type": "SchemaTableName", - "field_name": "schemaTableName", - "field_text": "SchemaTableName", + "field_type": "Optional", + "field_name": "writerTarget", + "field_text": "ExecutionWriterTarget", + "optional": true, "_N": 1, "field_local": true }, { - "field_type": "String", - "field_name": "tablePath", - "field_text": "String", + "field_type": "Optional", + "field_name": "analyzeTableHandle", + "field_text": "AnalyzeTableHandle", + "optional": true, "_N": 2, "field_local": true }, { - "field_type": "List", - "field_name": "partitionColumns", - "field_text": "List", + "field_type": "Optional", + "field_name": "deleteScanInfo", + "field_text": "DeleteScanInfo", + "optional": true, "_N": 3, "field_local": true - }, - { - "field_type": "List", - "field_name": "dataColumns", - "field_text": "List", - "_N": 4, - "field_local": true - }, - { - "field_type": "Map", - "field_name": "tableParameters", - "field_text": "Map", - "_N": 5, - "field_local": true - }, + } + ] + }, + { + "class_name": "EnforceSingleRowNode", + "struct": true, + "fields": [ { - "field_type": "TupleDomain", - "field_name": "domainPredicate", - "field_text": "TupleDomain", - "_N": 6, - "field_local": true + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false }, { - "field_type": "RowExpression", - "field_name": "remainingPredicate", - "field_text": "RowExpression", - "_N": 7, + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, "field_local": true, "optional": true - }, - { - "field_type": "Map", - "field_name": "predicateColumns", - "field_text": "Map", - "_N": 8, - "field_local": true - }, - { - "field_type": "TupleDomain", - "field_name": "partitionColumnPredicate", - "field_text": "TupleDomain>", - "_N": 9, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketHandle", - "field_text": "HiveBucketHandle", - "optional": true, - "_N": 10, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketFilter", - "field_text": "HiveBucketFilter", - "optional": true, - "_N": 11, - "field_local": true - }, - { - "field_type": "boolean", - "field_name": "pushdownFilterEnabled", - "field_text": "bool", - "_N": 12, - "field_local": true - }, - { - "field_type": "String", - "field_name": "layoutString", - "field_text": "String", - "_N": 13, - "field_local": true - }, + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode" + }, + { + "class_name": "EquiJoinClause", + "struct": true, + "fields": [ { - "field_type": "Optional>", - "field_name": "requestedColumns", - "field_text": "List", - "optional": true, - "_N": 14, + "field_type": "VariableReferenceExpression", + "field_name": "left", + "field_text": "VariableReferenceExpression", + "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "partialAggregationsPushedDown", - "field_text": "bool", - "_N": 15, + "field_type": "VariableReferenceExpression", + "field_name": "right", + "field_text": "VariableReferenceExpression", + "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "DistributionType", + "enum": true, + "elements": [ { - "field_type": "boolean", - "field_name": "appendRowNumber", - "field_text": "bool", - "_N": 16, - "field_local": true + "element": "PARTITIONED", + "_N": 1 + }, + { + "element": "REPLICATED", + "_N": 2, + "_last": true } - ], - "subclass": true, - "super_class": "ConnectorTableLayoutHandle", - "json_key": "hive" + ] }, { - "class_name": "BufferType", + "class_name": "JoinNodeType", "enum": true, "elements": [ { - "element": "PARTITIONED", + "element": "INNER", "_N": 1 }, { - "element": "BROADCAST", + "element": "LEFT", "_N": 2 }, { - "element": "ARBITRARY", + "element": "RIGHT", "_N": 3 }, { - "element": "DISCARDING", - "_N": 4 - }, - { - "element": "SPOOLING", - "_N": 5, + "element": "FULL", + "_N": 4, "_last": true } ] }, { - "class_name": "OutputBuffers", + "class_name": "JoinNode", "struct": true, "fields": [ { - "field_type": "BufferType", - "field_name": "type", - "field_text": "BufferType", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 1, - "field_local": true + "field_local": false }, { - "field_type": "long", - "field_name": "version", - "field_text": "int64_t", + "field_type": "JoinNodeType", + "field_name": "type", + "field_text": "JoinNodeType", "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "noMoreBufferIds", - "field_text": "bool", + "field_type": "PlanNode", + "field_name": "left", + "field_text": "PlanNode", "_N": 3, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "Map", - "field_name": "buffers", - "field_text": "Map", + "field_type": "PlanNode", + "field_name": "right", + "field_text": "PlanNode", "_N": 4, + "field_local": true, + "optional": true + }, + { + "field_type": "List", + "field_name": "criteria", + "field_text": "List", + "_N": 5, "field_local": true - } - ] - }, - { - "class_name": "Assignments", - "struct": true, - "fields": [ + }, { - "field_type": "Map", - "field_name": "assignments", - "field_text": "Map>", - "_N": 1, + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", + "_N": 6, "field_local": true - } - ] - }, - { - "class_name": "HiveTransactionHandle", - "struct": true, - "fields": [ + }, { - "field_type": "UUID", - "field_name": "uuid", - "field_text": "UUID", - "_N": 1, + "field_type": "Optional", + "field_name": "filter", + "field_text": "std::shared_ptr", + "optional": true, + "_N": 7, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "leftHashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 8, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "rightHashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 9, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "distributionType", + "field_text": "DistributionType", + "optional": true, + "_N": 10, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "dynamicFilters", + "field_text": "Map", + "_N": 11, "field_local": true } ], "subclass": true, - "super_class": "ConnectorTransactionHandle", - "json_key": "hive" + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.JoinNode" }, { - "class_name": "TableWriterNode", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nTableWriterNode::TableWriterNode() noexcept {\n _type = \"com.facebook.presto.sql.planner.plan.TableWriterNode\";\n}\n\nvoid to_json(json& j, const TableWriterNode& p) {\n j = json::object();\n j[\"@type\"] = \"com.facebook.presto.sql.planner.plan.TableWriterNode\";\n to_json_key(j, \"id\", p.id, \"TableWriterNode\", \"PlanNodeId\", \"id\");\n to_json_key(j, \"source\", p.source, \"TableWriterNode\", \"PlanNode\", \"source\");\n to_json_key(\n j,\n \"rowCountVariable\",\n p.rowCountVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"rowCountVariable\");\n to_json_key(\n j,\n \"fragmentVariable\",\n p.fragmentVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"fragmentVariable\");\n to_json_key(\n j,\n \"tableCommitContextVariable\",\n p.tableCommitContextVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"tableCommitContextVariable\");\n to_json_key(\n j,\n \"columns\",\n p.columns,\n \"TableWriterNode\",\n \"List\",\n \"columns\");\n to_json_key(\n j,\n \"columnNames\",\n p.columnNames,\n \"TableWriterNode\",\n \"List\",\n \"columnNames\");\n to_json_key(\n j,\n \"notNullColumnVariables\",\n p.notNullColumnVariables,\n \"TableWriterNode\",\n \"List\",\n \"notNullColumnVariables\");\n to_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n to_json_key(\n j,\n \"preferredShufflePartitioningScheme\",\n p.preferredShufflePartitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"preferredShufflePartitioningScheme\");\n}\n\nvoid from_json(const json& j, TableWriterNode& p) {\n p._type = j[\"@type\"];\n from_json_key(j, \"id\", p.id, \"TableWriterNode\", \"PlanNodeId\", \"id\");\n from_json_key(j, \"source\", p.source, \"TableWriterNode\", \"PlanNode\", \"source\");\n from_json_key(\n j,\n \"rowCountVariable\",\n p.rowCountVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"rowCountVariable\");\n from_json_key(\n j,\n \"fragmentVariable\",\n p.fragmentVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"fragmentVariable\");\n from_json_key(\n j,\n \"tableCommitContextVariable\",\n p.tableCommitContextVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"tableCommitContextVariable\");\n from_json_key(\n j,\n \"columns\",\n p.columns,\n \"TableWriterNode\",\n \"List\",\n \"columns\");\n from_json_key(\n j,\n \"columnNames\",\n p.columnNames,\n \"TableWriterNode\",\n \"List\",\n \"columnNames\");\n from_json_key(\n j,\n \"notNullColumnVariables\",\n p.notNullColumnVariables,\n \"TableWriterNode\",\n \"List\",\n \"notNullColumnVariables\");\n from_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n from_json_key(\n j,\n \"preferredShufflePartitioningScheme\",\n p.preferredShufflePartitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"preferredShufflePartitioningScheme\");\n}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct TableWriterNode : public PlanNode {\n std::shared_ptr source = {};\n // TODO Add target\n VariableReferenceExpression rowCountVariable = {};\n VariableReferenceExpression fragmentVariable = {};\n VariableReferenceExpression tableCommitContextVariable = {};\n List columns = {};\n List columnNames = {};\n List notNullColumnVariables = {};\n std::shared_ptr partitioningScheme = {};\n std::shared_ptr preferredShufflePartitioningScheme = {};\n // TODO Add statisticsAggregation\n\n TableWriterNode() noexcept;\n};\nvoid to_json(json& j, const TableWriterNode& p);\nvoid from_json(const json& j, TableWriterNode& p);\n} // namespace facebook::presto::protocol", + "class_name": "MergeJoinNode", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct MergeJoinNode : public PlanNode {\n MergeJoinNode() noexcept;\n PlanNodeId id = {};\n // JoinNodeType is referenced as JoinNode.Type in Presto\n // Since presto_cpp codegen can't nicely handle inner class references\n // So a special hard-coded template is required here\n JoinNodeType type = {};\n std::shared_ptr left = {};\n std::shared_ptr right = {};\n // EquiJoinClause is referenced as JoinNode.EquiJoinClause in Presto\n List criteria = {};\n List outputVariables = {};\n std::shared_ptr> filter = {};\n std::shared_ptr leftHashVariable = {};\n std::shared_ptr rightHashVariable = {};\n};\nvoid to_json(json& j, const MergeJoinNode& p);\nvoid from_json(const json& j, MergeJoinNode& p);\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { @@ -4427,914 +4241,1138 @@ "_N": 1, "field_local": false }, + { + "field_type": "JoinNode.Type", + "field_name": "type", + "field_text": "JoinNode.Type", + "_N": 2, + "field_local": true + }, { "field_type": "PlanNode", - "field_name": "source", + "field_name": "left", "field_text": "PlanNode", - "_N": 2, + "_N": 3, "field_local": true, "optional": true }, { - "field_type": "Optional", - "field_name": "target", - "field_text": "WriterTarget", - "optional": true, - "_N": 3, + "field_type": "PlanNode", + "field_name": "right", + "field_text": "PlanNode", + "_N": 4, + "field_local": true, + "optional": true + }, + { + "field_type": "List", + "field_name": "criteria", + "field_text": "List", + "_N": 5, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "rowCountVariable", - "field_text": "VariableReferenceExpression", - "_N": 4, + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", + "_N": 6, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "fragmentVariable", + "field_type": "Optional", + "field_name": "filter", + "field_text": "std::shared_ptr", + "optional": true, + "_N": 7, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "leftHashVariable", "field_text": "VariableReferenceExpression", - "_N": 5, + "optional": true, + "_N": 8, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "tableCommitContextVariable", + "field_type": "Optional", + "field_name": "rightHashVariable", "field_text": "VariableReferenceExpression", - "_N": 6, + "optional": true, + "_N": 9, + "field_local": true + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.MergeJoinNode" + }, + { + "class_name": "BucketConversion", + "struct": true, + "fields": [ + { + "field_type": "int", + "field_name": "tableBucketCount", + "field_text": "int", + "_N": 1, "field_local": true }, { - "field_type": "List", - "field_name": "columns", - "field_text": "List", - "_N": 7, + "field_type": "int", + "field_name": "partitionBucketCount", + "field_text": "int", + "_N": 2, "field_local": true }, { - "field_type": "List", - "field_name": "columnNames", - "field_text": "List", - "_N": 8, + "field_type": "List", + "field_name": "bucketColumnHandles", + "field_text": "List", + "_N": 3, "field_local": true + } + ] + }, + { + "class_name": "CacheQuotaScope", + "enum": true, + "elements": [ + { + "element": "GLOBAL", + "_N": 1 + }, + { + "element": "SCHEMA", + "_N": 2 + }, + { + "element": "TABLE", + "_N": 3 }, { - "field_type": "Set", - "field_name": "notNullColumnVariables", - "field_text": "List", - "_N": 9, + "element": "PARTITION", + "_N": 4, + "_last": true + } + ] + }, + { + "class_name": "CacheQuotaRequirement", + "struct": true, + "fields": [ + { + "field_type": "CacheQuotaScope", + "field_name": "cacheQuotaScope", + "field_text": "CacheQuotaScope", + "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "partitioningScheme", - "field_text": "PartitioningScheme", + "field_type": "Optional", + "field_name": "quota", + "field_text": "DataSize", "optional": true, - "_N": 10, + "_N": 2, "field_local": true + } + ] + }, + { + "class_name": "NodeSelectionStrategy", + "enum": true, + "elements": [ + { + "element": "HARD_AFFINITY", + "_N": 1 }, { - "field_type": "Optional", - "field_name": "preferredShufflePartitioningScheme", - "field_text": "PartitioningScheme", - "optional": true, - "_N": 11, - "field_local": true + "element": "SOFT_AFFINITY", + "_N": 2 }, { - "field_type": "Optional", - "field_name": "statisticsAggregation", - "field_text": "StatisticAggregations", - "optional": true, - "_N": 12, - "field_local": true + "element": "NO_PREFERENCE", + "_N": 3, + "_last": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.TableWriterNode" - }, - { - "class_name": "Block", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const Block& p) {\n j = p.data;\n}\n\nvoid from_json(const json& j, Block& p) {\n p.data = std::string(j);\n}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct Block {\n std::string data;\n};\n\nvoid to_json(json& j, const Block& p);\n\nvoid from_json(const json& j, Block& p);\n\n} // namespace facebook::presto::protocol" + ] }, { - "class_name": "ValueEntry", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nclass ValueEntry {\n public:\n Type type;\n std::shared_ptr block;\n};\n\nvoid to_json(json& j, const ValueEntry& p);\nvoid from_json(const json& j, ValueEntry& p);\n\n} // namespace facebook::presto::protocol", + "class_name": "Column", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nstruct Column {\n String name;\n String type;\n\n Column() = default;\n explicit Column(const String& str) {\n name = str;\n }\n};\n\nvoid to_json(json& j, const Column& p);\nvoid from_json(const json& j, Column& p);\n\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "Block", - "field_name": "block", - "field_text": "Block", + "field_type": "String", + "field_name": "type", + "field_text": "String", "_N": 2, "field_local": true } ] }, { - "class_name": "EquatableValueSet", + "class_name": "TableToPartitionMapping", "struct": true, "fields": [ { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", + "field_type": "Optional>", + "field_name": "tableToPartitionColumns", + "field_text": "Map", + "optional": true, "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "whiteList", - "field_text": "bool", + "field_type": "Map", + "field_name": "partitionSchemaDifference", + "field_text": "Map", "_N": 2, "field_local": true - }, - { - "field_type": "Set", - "field_name": "entries", - "field_text": "List", - "_N": 3, - "field_local": true } - ], - "subclass": true, - "super_class": "ValueSet", - "json_key": "equatable" + ] }, { - "class_name": "PageBufferInfo", + "class_name": "HiveFileSplit", "struct": true, "fields": [ { - "field_type": "int", - "field_name": "partition", - "field_text": "int", + "field_type": "String", + "field_name": "path", + "field_text": "String", "_N": 1, "field_local": true }, { "field_type": "long", - "field_name": "bufferedPages", + "field_name": "start", "field_text": "int64_t", "_N": 2, "field_local": true }, { "field_type": "long", - "field_name": "bufferedBytes", + "field_name": "length", "field_text": "int64_t", "_N": 3, "field_local": true }, { "field_type": "long", - "field_name": "rowsAdded", + "field_name": "fileSize", "field_text": "int64_t", "_N": 4, "field_local": true }, { "field_type": "long", - "field_name": "pagesAdded", + "field_name": "fileModifiedTime", "field_text": "int64_t", "_N": 5, "field_local": true + }, + { + "field_type": "Optional", + "field_name": "extraFileInfo", + "field_text": "String", + "optional": true, + "_N": 6, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "customSplitInfo", + "field_text": "Map", + "_N": 7, + "field_local": true } ] }, { - "class_name": "BufferInfo", + "class_name": "HivePartitionKey", "struct": true, "fields": [ { - "field_type": "OutputBufferId", - "field_name": "bufferId", - "field_text": "OutputBufferId", + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "finished", - "field_text": "bool", + "field_type": "Optional", + "field_name": "value", + "field_text": "String", + "optional": true, "_N": 2, "field_local": true - }, + } + ] + }, + { + "class_name": "StorageFormat", + "struct": true, + "fields": [ { - "field_type": "int", - "field_name": "bufferedPages", - "field_text": "int", - "_N": 3, + "field_type": "String", + "field_name": "serDe", + "field_text": "String", + "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "pagesSent", - "field_text": "int64_t", - "_N": 4, + "field_type": "String", + "field_name": "inputFormat", + "field_text": "String", + "_N": 2, "field_local": true }, { - "field_type": "PageBufferInfo", - "field_name": "pageBufferInfo", - "field_text": "PageBufferInfo", - "_N": 5, + "field_type": "String", + "field_name": "outputFormat", + "field_text": "String", + "_N": 3, "field_local": true } ] }, { - "class_name": "FunctionKind", + "class_name": "BucketFunctionType", "enum": true, "elements": [ { - "element": "SCALAR", + "element": "HIVE_COMPATIBLE", "_N": 1 }, { - "element": "AGGREGATE", - "_N": 2 + "element": "PRESTO_NATIVE", + "_N": 2, + "_last": true + } + ] + }, + { + "class_name": "HiveBucketProperty", + "struct": true, + "fields": [ + { + "field_type": "List", + "field_name": "bucketedBy", + "field_text": "List", + "_N": 1, + "field_local": true }, { - "element": "WINDOW", + "field_type": "int", + "field_name": "bucketCount", + "field_text": "int", + "_N": 2, + "field_local": true + }, + { + "field_type": "List", + "field_name": "sortedBy", + "field_text": "List", "_N": 3, - "_last": true + "field_local": true + }, + { + "field_type": "BucketFunctionType", + "field_name": "bucketFunctionType", + "field_text": "BucketFunctionType", + "_N": 4, + "field_local": true + }, + { + "field_type": "Optional>", + "field_name": "types", + "field_text": "List", + "optional": true, + "_N": 5, + "field_local": true } ] }, { - "class_name": "LongVariableConstraint", + "class_name": "Storage", "struct": true, "fields": [ + { + "field_type": "StorageFormat", + "field_name": "storageFormat", + "field_text": "StorageFormat", + "_N": 1, + "field_local": true + }, { "field_type": "String", - "field_name": "name", + "field_name": "location", "field_text": "String", - "_N": 1, + "_N": 2, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "bucketProperty", + "field_text": "HiveBucketProperty", + "optional": true, + "_N": 3, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "skewed", + "field_text": "bool", + "_N": 4, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "serdeParameters", + "field_text": "Map", + "_N": 5, "field_local": true }, { - "field_type": "String", - "field_name": "expression", - "field_text": "String", - "_N": 2, + "field_type": "Map", + "field_name": "parameters", + "field_text": "Map", + "_N": 6, "field_local": true } ] }, { - "class_name": "TypeVariableConstraint", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct TypeVariableConstraint {\n String name = {};\n bool comparableRequired = {};\n bool orderableRequired = {};\n String variadicBound = {};\n bool nonDecimalNumericRequired = {};\n String boundedBy = {};\n};\nvoid to_json(json& j, const TypeVariableConstraint& p);\nvoid from_json(const json& j, TypeVariableConstraint& p);\n\n} // namespace facebook::presto::protocol", + "class_name": "HiveSplit", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "name", - "field_text": "String", + "field_type": "HiveFileSplit", + "field_name": "fileSplit", + "field_text": "HiveFileSplit", "_N": 1, "field_local": true }, { - "field_type": "boolean", - "field_name": "comparableRequired", - "field_text": "bool", + "field_type": "String", + "field_name": "database", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "orderableRequired", - "field_text": "bool", + "field_type": "String", + "field_name": "table", + "field_text": "String", "_N": 3, "field_local": true }, { "field_type": "String", - "field_name": "variadicBound", + "field_name": "partitionName", "field_text": "String", "_N": 4, "field_local": true }, { - "field_type": "boolean", - "field_name": "nonDecimalNumericRequired", - "field_text": "bool", + "field_type": "Storage", + "field_name": "storage", + "field_text": "Storage", "_N": 5, "field_local": true - } - ] - }, - { - "class_name": "Signature", - "struct": true, - "fields": [ + }, { - "field_type": "QualifiedObjectName", - "field_name": "name", - "field_text": "QualifiedObjectName", - "_N": 1, + "field_type": "List", + "field_name": "partitionKeys", + "field_text": "List", + "_N": 6, "field_local": true }, { - "field_type": "FunctionKind", - "field_name": "kind", - "field_text": "FunctionKind", - "_N": 2, + "field_type": "List", + "field_name": "addresses", + "field_text": "List", + "_N": 7, "field_local": true }, { - "field_type": "List", - "field_name": "typeVariableConstraints", - "field_text": "List", - "_N": 3, + "field_type": "OptionalInt", + "field_name": "readBucketNumber", + "field_text": "int", + "optional": true, + "_N": 8, "field_local": true }, { - "field_type": "List", - "field_name": "longVariableConstraints", - "field_text": "List", - "_N": 4, + "field_type": "OptionalInt", + "field_name": "tableBucketNumber", + "field_text": "int", + "optional": true, + "_N": 9, "field_local": true }, { - "field_type": "TypeSignature", - "field_name": "returnType", - "field_text": "TypeSignature", - "_N": 5, + "field_type": "NodeSelectionStrategy", + "field_name": "nodeSelectionStrategy", + "field_text": "NodeSelectionStrategy", + "_N": 10, "field_local": true }, { - "field_type": "List", - "field_name": "argumentTypes", - "field_text": "List", - "_N": 6, + "field_type": "int", + "field_name": "partitionDataColumnCount", + "field_text": "int", + "_N": 11, + "field_local": true + }, + { + "field_type": "TableToPartitionMapping", + "field_name": "tableToPartitionMapping", + "field_text": "TableToPartitionMapping", + "_N": 12, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "bucketConversion", + "field_text": "BucketConversion", + "optional": true, + "_N": 13, "field_local": true }, { "field_type": "boolean", - "field_name": "variableArity", + "field_name": "s3SelectPushdownEnabled", "field_text": "bool", - "_N": 7, + "_N": 14, "field_local": true - } - ] - }, - { - "class_name": "BuiltInFunctionHandle", - "struct": true, - "fields": [ + }, { - "field_type": "Signature", - "field_name": "signature", - "field_text": "Signature", - "_N": 1, + "field_type": "CacheQuotaRequirement", + "field_name": "cacheQuota", + "field_text": "CacheQuotaRequirement", + "_N": 15, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "encryptionMetadata", + "field_text": "EncryptionInformation", + "optional": true, + "_N": 16, + "field_local": true + }, + { + "field_type": "Set", + "field_name": "redundantColumnDomains", + "field_text": "List>", + "_N": 17, + "field_local": true + }, + { + "field_type": "SplitWeight", + "field_name": "splitWeight", + "field_text": "SplitWeight", + "_N": 18, "field_local": true } ], "subclass": true, - "super_class": "FunctionHandle", - "json_key": "$static" + "super_class": "ConnectorSplit", + "json_key": "hive" }, { - "class_name": "LambdaDefinitionExpression", + "class_name": "AssignUniqueId", "struct": true, "fields": [ { - "field_type": "Optional", - "field_name": "sourceLocation", - "field_text": "SourceLocation", - "optional": true, + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 1, "field_local": false }, { - "field_type": "List", - "field_name": "argumentTypes", - "field_text": "List", + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", "_N": 2, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "List", - "field_name": "arguments", - "field_text": "List", + "field_type": "VariableReferenceExpression", + "field_name": "idVariable", + "field_text": "VariableReferenceExpression", "_N": 3, "field_local": true + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.AssignUniqueId" + }, + { + "class_name": "TpchColumnHandle", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// TpchColumnHandle is special since we require an implementation of\n// operator<().\n\nnamespace facebook::presto::protocol {\nstruct TpchColumnHandle : public ColumnHandle {\n String columnName = {};\n Type type = {};\n\n TpchColumnHandle() noexcept;\n\n bool operator<(const ColumnHandle& o) const override {\n return columnName < dynamic_cast(o).columnName;\n }\n};\nvoid to_json(json& j, const TpchColumnHandle& p);\nvoid from_json(const json& j, TpchColumnHandle& p);\n} // namespace facebook::presto::protocol", + "struct": true, + "fields": [ + { + "field_type": "String", + "field_name": "columnName", + "field_text": "String", + "_N": 1, + "field_local": true }, { - "field_type": "RowExpression", - "field_name": "body", - "field_text": "RowExpression", - "_N": 4, - "field_local": true, - "optional": true + "field_type": "Type", + "field_name": "type", + "field_text": "Type", + "_N": 2, + "field_local": true } ], "subclass": true, - "super_class": "RowExpression", - "json_key": "lambda" + "super_class": "ColumnHandle", + "json_key": "tpch" }, { - "class_name": "BoundType", + "class_name": "PrestoTableType", "enum": true, "elements": [ { - "element": "UNBOUNDED_PRECEDING", + "element": "MANAGED_TABLE", "_N": 1 }, { - "element": "PRECEDING", + "element": "EXTERNAL_TABLE", "_N": 2 }, { - "element": "CURRENT_ROW", + "element": "VIRTUAL_VIEW", "_N": 3 }, { - "element": "FOLLOWING", + "element": "MATERIALIZED_VIEW", "_N": 4 }, { - "element": "UNBOUNDED_FOLLOWING", - "_N": 5, - "_last": true - } - ] - }, - { - "class_name": "WindowType", - "enum": true, - "elements": [ - { - "element": "RANGE", - "_N": 1 + "element": "TEMPORARY_TABLE", + "_N": 5 }, { - "element": "ROWS", - "_N": 2 + "element": "OTHER", + "_N": 6, + "_last": true } ] }, { - "class_name": "Frame", + "class_name": "Table", "struct": true, "fields": [ { - "field_type": "WindowType", - "field_name": "type", - "field_text": "WindowType", + "field_type": "String", + "field_name": "databaseName", + "field_text": "String", "_N": 1, "field_local": true }, - { - "field_type": "BoundType", - "field_name": "startType", - "field_text": "BoundType", + { + "field_type": "String", + "field_name": "tableName", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "Optional", - "field_name": "startValue", - "field_text": "VariableReferenceExpression", - "optional": true, + "field_type": "String", + "field_name": "owner", + "field_text": "String", "_N": 3, "field_local": true }, { - "field_type": "BoundType", - "field_name": "endType", - "field_text": "BoundType", + "field_type": "PrestoTableType", + "field_name": "tableType", + "field_text": "PrestoTableType", "_N": 4, "field_local": true }, { - "field_type": "Optional", - "field_name": "endValue", - "field_text": "VariableReferenceExpression", - "optional": true, + "field_type": "Storage", + "field_name": "storage", + "field_text": "Storage", "_N": 5, "field_local": true }, + { + "field_type": "List", + "field_name": "dataColumns", + "field_text": "List", + "_N": 6, + "field_local": true + }, + { + "field_type": "List", + "field_name": "partitionColumns", + "field_text": "List", + "_N": 7, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "parameters", + "field_text": "Map", + "_N": 8, + "field_local": true + }, { "field_type": "Optional", - "field_name": "originalStartValue", + "field_name": "viewOriginalText", "field_text": "String", "optional": true, - "_N": 6, + "_N": 9, "field_local": true }, { "field_type": "Optional", - "field_name": "originalEndValue", + "field_name": "viewExpandedText", "field_text": "String", "optional": true, - "_N": 7, + "_N": 10, "field_local": true } ] }, { - "class_name": "Function", + "class_name": "HivePageSinkMetadata", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const HivePageSinkMetadata& p) {\n j = json::object();\n to_json_key(\n j,\n \"schemaTableName\",\n p.schemaTableName,\n \"HivePageSinkMetadata\",\n \"SchemaTableName\",\n \"schemaTableName\");\n to_json_key(j, \"table\", p.table, \"HivePageSinkMetadata\", \"Table\", \"table\");\n}\n\nvoid from_json(const json& j, HivePageSinkMetadata& p) {\n from_json_key(\n j,\n \"schemaTableName\",\n p.schemaTableName,\n \"HivePageSinkMetadata\",\n \"SchemaTableName\",\n \"schemaTableName\");\n from_json_key(j, \"table\", p.table, \"HivePageSinkMetadata\", \"Table\", \"table\");\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// dependency Table\n// dependency SchemaTableName\n\nnamespace facebook::presto::protocol {\n\nstruct HivePageSinkMetadata {\n SchemaTableName schemaTableName = {};\n std::shared_ptr
table = {};\n // TODO Add modifiedPartitions\n};\nvoid to_json(json& j, const HivePageSinkMetadata& p);\nvoid from_json(const json& j, HivePageSinkMetadata& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "HiveMetadataUpdateHandle", "struct": true, "fields": [ { - "field_type": "CallExpression", - "field_name": "functionCall", - "field_text": "CallExpression", + "field_type": "UUID", + "field_name": "requestId", + "field_text": "UUID", "_N": 1, "field_local": true }, { - "field_type": "Frame", - "field_name": "frame", - "field_text": "Frame", + "field_type": "SchemaTableName", + "field_name": "schemaTableName", + "field_text": "SchemaTableName", "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "ignoreNulls", - "field_text": "bool", + "field_type": "Optional", + "field_name": "partitionName", + "field_text": "String", + "optional": true, "_N": 3, "field_local": true + }, + { + "field_type": "Optional", + "field_name": "fileName", + "field_text": "String", + "optional": true, + "_N": 4, + "field_local": true + } + ], + "subclass": true, + "super_class": "ConnectorMetadataUpdateHandle", + "json_key": "hive" + }, + { + "class_name": "NodeVersion", + "struct": true, + "fields": [ + { + "field_type": "String", + "field_name": "version", + "field_text": "String", + "_N": 1, + "field_local": true } ] }, { - "class_name": "VariableStatsEstimate", + "class_name": "ServerInfo", "struct": true, "fields": [ { - "field_type": "double", - "field_name": "lowValue", - "field_text": "double", + "field_type": "NodeVersion", + "field_name": "nodeVersion", + "field_text": "NodeVersion", "_N": 1, "field_local": true }, { - "field_type": "double", - "field_name": "highValue", - "field_text": "double", + "field_type": "String", + "field_name": "environment", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "double", - "field_name": "nullsFraction", - "field_text": "double", + "field_type": "boolean", + "field_name": "coordinator", + "field_text": "bool", "_N": 3, "field_local": true }, { - "field_type": "double", - "field_name": "averageRowSize", - "field_text": "double", + "field_type": "boolean", + "field_name": "starting", + "field_text": "bool", "_N": 4, "field_local": true }, { - "field_type": "double", - "field_name": "distinctValuesCount", - "field_text": "double", + "field_type": "Optional", + "field_name": "uptime", + "field_text": "Duration", + "optional": true, "_N": 5, "field_local": true } ] }, { - "class_name": "PlanNodeStatsEstimate", + "class_name": "ConnectorOutputTableHandle", + "field_name": "connectorOutputTableHandle", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ + { + "type": "HiveOutputTableHandle", + "name": "hiveOutputTableHandle", + "key": "hive", + "_N": 1, + "_last": true + } + ], + "fields": [], + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorOutputTableHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorOutputTableHandle ConnectorOutputTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k =\n std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorOutputTableHandle \");\n}\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "OutputTableHandle", "struct": true, "fields": [ { - "field_type": "double", - "field_name": "outputRowCount", - "field_text": "double", + "field_type": "ConnectorId", + "field_name": "connectorId", + "field_text": "ConnectorId", "_N": 1, "field_local": true }, { - "field_type": "double", - "field_name": "totalSize", - "field_text": "double", + "field_type": "ConnectorTransactionHandle", + "field_name": "transactionHandle", + "field_text": "ConnectorTransactionHandle", "_N": 2, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "boolean", - "field_name": "confident", - "field_text": "bool", + "field_type": "ConnectorOutputTableHandle", + "field_name": "connectorHandle", + "field_text": "ConnectorOutputTableHandle", "_N": 3, + "field_local": true, + "optional": true + } + ] + }, + { + "class_name": "CreateHandle", + "struct": true, + "fields": [ + { + "field_type": "OutputTableHandle", + "field_name": "handle", + "field_text": "OutputTableHandle", + "_N": 1, "field_local": true }, { - "field_type": "Map", - "field_name": "variableStatistics", - "field_text": "Map", - "_N": 4, + "field_type": "SchemaTableName", + "field_name": "schemaTableName", + "field_text": "SchemaTableName", + "_N": 2, "field_local": true } - ] + ], + "subclass": true, + "super_class": "ExecutionWriterTarget", + "json_key": "CreateHandle" }, { - "class_name": "PlanCostEstimate", + "class_name": "NodeStatus", "struct": true, "fields": [ { - "field_type": "double", - "field_name": "cpuCost", - "field_text": "double", + "field_type": "String", + "field_name": "nodeId", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "double", - "field_name": "maxMemory", - "field_text": "double", + "field_type": "NodeVersion", + "field_name": "nodeVersion", + "field_text": "NodeVersion", "_N": 2, "field_local": true }, { - "field_type": "double", - "field_name": "maxMemoryWhenOutputting", - "field_text": "double", + "field_type": "String", + "field_name": "environment", + "field_text": "String", "_N": 3, "field_local": true }, { - "field_type": "double", - "field_name": "networkCost", - "field_text": "double", + "field_type": "boolean", + "field_name": "coordinator", + "field_text": "bool", "_N": 4, "field_local": true - } - ] - }, - { - "class_name": "StatsAndCosts", - "struct": true, - "fields": [ + }, { - "field_type": "Map", - "field_name": "stats", - "field_text": "Map", - "_N": 1, + "field_type": "Duration", + "field_name": "uptime", + "field_text": "Duration", + "_N": 5, + "field_local": true + }, + { + "field_type": "String", + "field_name": "externalAddress", + "field_text": "String", + "_N": 6, + "field_local": true + }, + { + "field_type": "String", + "field_name": "internalAddress", + "field_text": "String", + "_N": 7, "field_local": true }, { - "field_type": "Map", - "field_name": "costs", - "field_text": "Map", - "_N": 2, + "field_type": "MemoryInfo", + "field_name": "memoryInfo", + "field_text": "MemoryInfo", + "_N": 8, "field_local": true - } - ] - }, - { - "class_name": "Bound", - "enum": true, - "elements": [ - { - "element": "BELOW", - "_N": 1 }, { - "element": "EXACTLY", - "_N": 2 + "field_type": "int", + "field_name": "processors", + "field_text": "int", + "_N": 9, + "field_local": true }, { - "element": "ABOVE", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "Marker", - "struct": true, - "fields": [ - { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", - "_N": 1, + "field_type": "double", + "field_name": "processCpuLoad", + "field_text": "double", + "_N": 10, "field_local": true }, { - "field_type": "Optional", - "field_name": "valueBlock", - "field_text": "Block", - "optional": true, - "_N": 2, + "field_type": "double", + "field_name": "systemCpuLoad", + "field_text": "double", + "_N": 11, "field_local": true }, { - "field_type": "Bound", - "field_name": "bound", - "field_text": "Bound", - "_N": 3, + "field_type": "long", + "field_name": "heapUsed", + "field_text": "int64_t", + "_N": 12, "field_local": true - } - ] - }, - { - "class_name": "TableToPartitionMapping", - "struct": true, - "fields": [ + }, { - "field_type": "Optional>", - "field_name": "tableToPartitionColumns", - "field_text": "Map", - "optional": true, - "_N": 1, + "field_type": "long", + "field_name": "heapAvailable", + "field_text": "int64_t", + "_N": 13, "field_local": true }, { - "field_type": "Map", - "field_name": "partitionSchemaDifference", - "field_text": "Map", - "_N": 2, + "field_type": "long", + "field_name": "nonHeapUsed", + "field_text": "int64_t", + "_N": 14, "field_local": true } ] }, { - "class_name": "TableScanNode", + "class_name": "HiveTableHandle", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "String", + "field_name": "schemaName", + "field_text": "String", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "TableHandle", - "field_name": "table", - "field_text": "TableHandle", + "field_type": "String", + "field_name": "tableName", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", + "field_type": "Optional>>", + "field_name": "analyzePartitionValues", + "field_text": "List>", + "optional": true, "_N": 3, "field_local": true - }, - { - "field_type": "Map", - "field_name": "assignments", - "field_text": "Map>", - "_N": 4, - "field_local": true } ], "subclass": true, - "super_class": "PlanNode", - "json_key": ".TableScanNode" + "super_class": "ConnectorTableHandle", + "json_key": "hive" }, { - "class_name": "EmptySplit", + "class_name": "HiveTransactionHandle", "struct": true, "fields": [ { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", + "field_type": "UUID", + "field_name": "uuid", + "field_text": "UUID", "_N": 1, "field_local": true } ], "subclass": true, - "super_class": "ConnectorSplit", - "json_key": "$empty" + "super_class": "ConnectorTransactionHandle", + "json_key": "hive" }, { - "class_name": "ValuesNode", + "class_name": "TableWriterNode", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nTableWriterNode::TableWriterNode() noexcept {\n _type = \"com.facebook.presto.sql.planner.plan.TableWriterNode\";\n}\n\nvoid to_json(json& j, const TableWriterNode& p) {\n j = json::object();\n j[\"@type\"] = \"com.facebook.presto.sql.planner.plan.TableWriterNode\";\n to_json_key(j, \"id\", p.id, \"TableWriterNode\", \"PlanNodeId\", \"id\");\n to_json_key(j, \"source\", p.source, \"TableWriterNode\", \"PlanNode\", \"source\");\n to_json_key(\n j,\n \"rowCountVariable\",\n p.rowCountVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"rowCountVariable\");\n to_json_key(\n j,\n \"fragmentVariable\",\n p.fragmentVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"fragmentVariable\");\n to_json_key(\n j,\n \"tableCommitContextVariable\",\n p.tableCommitContextVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"tableCommitContextVariable\");\n to_json_key(\n j,\n \"columns\",\n p.columns,\n \"TableWriterNode\",\n \"List\",\n \"columns\");\n to_json_key(\n j,\n \"columnNames\",\n p.columnNames,\n \"TableWriterNode\",\n \"List\",\n \"columnNames\");\n to_json_key(\n j,\n \"notNullColumnVariables\",\n p.notNullColumnVariables,\n \"TableWriterNode\",\n \"List\",\n \"notNullColumnVariables\");\n to_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n to_json_key(\n j,\n \"preferredShufflePartitioningScheme\",\n p.preferredShufflePartitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"preferredShufflePartitioningScheme\");\n}\n\nvoid from_json(const json& j, TableWriterNode& p) {\n p._type = j[\"@type\"];\n from_json_key(j, \"id\", p.id, \"TableWriterNode\", \"PlanNodeId\", \"id\");\n from_json_key(j, \"source\", p.source, \"TableWriterNode\", \"PlanNode\", \"source\");\n from_json_key(\n j,\n \"rowCountVariable\",\n p.rowCountVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"rowCountVariable\");\n from_json_key(\n j,\n \"fragmentVariable\",\n p.fragmentVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"fragmentVariable\");\n from_json_key(\n j,\n \"tableCommitContextVariable\",\n p.tableCommitContextVariable,\n \"TableWriterNode\",\n \"VariableReferenceExpression\",\n \"tableCommitContextVariable\");\n from_json_key(\n j,\n \"columns\",\n p.columns,\n \"TableWriterNode\",\n \"List\",\n \"columns\");\n from_json_key(\n j,\n \"columnNames\",\n p.columnNames,\n \"TableWriterNode\",\n \"List\",\n \"columnNames\");\n from_json_key(\n j,\n \"notNullColumnVariables\",\n p.notNullColumnVariables,\n \"TableWriterNode\",\n \"List\",\n \"notNullColumnVariables\");\n from_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n from_json_key(\n j,\n \"preferredShufflePartitioningScheme\",\n p.preferredShufflePartitioningScheme,\n \"TableWriterNode\",\n \"PartitioningScheme\",\n \"preferredShufflePartitioningScheme\");\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nstruct TableWriterNode : public PlanNode {\n std::shared_ptr source = {};\n // TODO Add target\n VariableReferenceExpression rowCountVariable = {};\n VariableReferenceExpression fragmentVariable = {};\n VariableReferenceExpression tableCommitContextVariable = {};\n List columns = {};\n List columnNames = {};\n List notNullColumnVariables = {};\n std::shared_ptr partitioningScheme = {};\n std::shared_ptr preferredShufflePartitioningScheme = {};\n // TODO Add statisticsAggregation\n\n TableWriterNode() noexcept;\n};\nvoid to_json(json& j, const TableWriterNode& p);\nvoid from_json(const json& j, TableWriterNode& p);\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ - { - "field_type": "Optional", - "field_name": "location", - "field_text": "SourceLocation", - "optional": true, - "_N": 1, - "field_local": true - }, { "field_type": "PlanNodeId", "field_name": "id", "field_text": "PlanNodeId", - "_N": 2, + "_N": 1, "field_local": false }, { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, + "field_local": true, + "optional": true + }, + { + "field_type": "Optional", + "field_name": "target", + "field_text": "WriterTarget", + "optional": true, "_N": 3, "field_local": true }, { - "field_type": "List>", - "field_name": "rows", - "field_text": "List>>", + "field_type": "VariableReferenceExpression", + "field_name": "rowCountVariable", + "field_text": "VariableReferenceExpression", "_N": 4, "field_local": true }, { - "field_type": "Optional", - "field_name": "valuesNodeLabel", - "field_text": "String", - "optional": true, + "field_type": "VariableReferenceExpression", + "field_name": "fragmentVariable", + "field_text": "VariableReferenceExpression", "_N": 5, "field_local": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": ".ValuesNode" - }, - { - "class_name": "RuntimeUnit", - "enum": true, - "elements": [ - { - "element": "NONE", - "_N": 1 - }, - { - "element": "NANO", - "_N": 2 }, { - "element": "BYTE", - "_N": 3, - "_last": true - } - ] - }, - { - "class_name": "RuntimeMetric", - "struct": true, - "fields": [ - { - "field_type": "String", - "field_name": "name", - "field_text": "String", - "_N": 1, + "field_type": "VariableReferenceExpression", + "field_name": "tableCommitContextVariable", + "field_text": "VariableReferenceExpression", + "_N": 6, "field_local": true }, { - "field_type": "RuntimeUnit", - "field_name": "unit", - "field_text": "RuntimeUnit", - "_N": 2, + "field_type": "List", + "field_name": "columns", + "field_text": "List", + "_N": 7, "field_local": true }, { - "field_type": "long", - "field_name": "sum", - "field_text": "int64_t", - "_N": 3, + "field_type": "List", + "field_name": "columnNames", + "field_text": "List", + "_N": 8, "field_local": true }, { - "field_type": "long", - "field_name": "count", - "field_text": "int64_t", - "_N": 4, + "field_type": "Set", + "field_name": "notNullColumnVariables", + "field_text": "List", + "_N": 9, "field_local": true }, { - "field_type": "long", - "field_name": "max", - "field_text": "int64_t", - "_N": 5, + "field_type": "Optional", + "field_name": "partitioningScheme", + "field_text": "PartitioningScheme", + "optional": true, + "_N": 10, "field_local": true }, { - "field_type": "long", - "field_name": "min", - "field_text": "int64_t", - "_N": 6, + "field_type": "Optional", + "field_name": "preferredShufflePartitioningScheme", + "field_text": "PartitioningScheme", + "optional": true, + "_N": 11, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "statisticsAggregation", + "field_text": "StatisticAggregations", + "optional": true, + "_N": 12, "field_local": true } - ] + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.TableWriterNode" }, { - "class_name": "AssignUniqueId", + "class_name": "DistinctLimitNode", "struct": true, "fields": [ { @@ -5353,1811 +5391,2113 @@ "optional": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "idVariable", - "field_text": "VariableReferenceExpression", + "field_type": "long", + "field_name": "limit", + "field_text": "int64_t", "_N": 3, "field_local": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.AssignUniqueId" - }, - { - "class_name": "RemoteTransactionHandle", - "struct": true, - "fields": [ + }, { - "field_type": "Optional", - "field_name": "dummy", - "field_text": "String", + "field_type": "boolean", + "field_name": "partial", + "field_text": "bool", + "_N": 4, + "field_local": true + }, + { + "field_type": "List", + "field_name": "distinctVariables", + "field_text": "List", + "_N": 5, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "hashVariable", + "field_text": "VariableReferenceExpression", "optional": true, - "_N": 1, - "field_local": true, - "last": true + "_N": 6, + "field_local": true } ], "subclass": true, - "super_class": "ConnectorTransactionHandle", - "json_key": "$remote" + "super_class": "PlanNode", + "json_key": ".DistinctLimitNode" }, { - "class_name": "ConnectorInsertTableHandle", - "field_name": "connectorInsertTableHandle", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + "class_name": "Form", + "enum": true, + "elements": [ { - "type": "HiveInsertTableHandle", - "name": "hiveInsertTableHandle", - "key": "hive", - "_N": 1, + "element": "IF", + "_N": 1 + }, + { + "element": "NULL_IF", + "_N": 2 + }, + { + "element": "SWITCH", + "_N": 3 + }, + { + "element": "WHEN", + "_N": 4 + }, + { + "element": "IS_NULL", + "_N": 5 + }, + { + "element": "COALESCE", + "_N": 6 + }, + { + "element": "IN", + "_N": 7 + }, + { + "element": "AND", + "_N": 8 + }, + { + "element": "OR", + "_N": 9 + }, + { + "element": "DEREFERENCE", + "_N": 10 + }, + { + "element": "ROW_CONSTRUCTOR", + "_N": 11 + }, + { + "element": "BIND", + "_N": 12, "_last": true } - ], - "fields": [], - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\nvoid to_json(json& j, const std::shared_ptr& p) {\n if (p == nullptr) {\n return;\n }\n String type = p->_type;\n\n if (getConnectorKey(type) == \"hive\") {\n j = *std::static_pointer_cast(p);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorInsertTableHandle \");\n}\n\nvoid from_json(const json& j, std::shared_ptr& p) {\n String type;\n try {\n type = p->getSubclassKey(j);\n } catch (json::parse_error& e) {\n throw ParseError(\n std::string(e.what()) +\n \" ConnectorInsertTableHandle ConnectorInsertTableHandle\");\n }\n\n if (getConnectorKey(type) == \"hive\") {\n std::shared_ptr k =\n std::make_shared();\n j.get_to(*k);\n p = std::static_pointer_cast(k);\n return;\n }\n\n throw TypeError(type + \" no abstract type ConnectorInsertTableHandle \");\n}\n} // namespace facebook::presto::protocol" + ] }, { - "class_name": "InsertTableHandle", + "class_name": "SpecialFormExpression", "struct": true, "fields": [ { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", + "field_type": "Optional", + "field_name": "sourceLocation", + "field_text": "SourceLocation", + "optional": true, "_N": 1, - "field_local": true + "field_local": false }, { - "field_type": "ConnectorTransactionHandle", - "field_name": "transactionHandle", - "field_text": "ConnectorTransactionHandle", + "field_type": "Form", + "field_name": "form", + "field_text": "Form", "_N": 2, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "ConnectorInsertTableHandle", - "field_name": "connectorHandle", - "field_text": "ConnectorInsertTableHandle", + "field_type": "Type", + "field_name": "returnType", + "field_text": "Type", "_N": 3, - "field_local": true, - "optional": true + "field_local": true + }, + { + "field_type": "List", + "field_name": "arguments", + "field_text": "List>", + "_N": 4, + "field_local": true } - ] + ], + "subclass": true, + "super_class": "RowExpression", + "json_key": "special" }, { - "class_name": "RefreshMaterializedViewHandle", + "class_name": "ValueEntry", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nclass ValueEntry {\n public:\n Type type;\n std::shared_ptr block;\n};\n\nvoid to_json(json& j, const ValueEntry& p);\nvoid from_json(const json& j, ValueEntry& p);\n\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { - "field_type": "InsertTableHandle", - "field_name": "handle", - "field_text": "InsertTableHandle", + "field_type": "Type", + "field_name": "type", + "field_text": "Type", "_N": 1, "field_local": true }, { - "field_type": "SchemaTableName", - "field_name": "schemaTableName", - "field_text": "SchemaTableName", + "field_type": "Block", + "field_name": "block", + "field_text": "Block", "_N": 2, "field_local": true } ] }, { - "class_name": "HiveFileSplit", + "class_name": "EquatableValueSet", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "path", - "field_text": "String", + "field_type": "Type", + "field_name": "type", + "field_text": "Type", "_N": 1, "field_local": true }, { - "field_type": "long", - "field_name": "start", - "field_text": "int64_t", + "field_type": "boolean", + "field_name": "whiteList", + "field_text": "bool", "_N": 2, "field_local": true }, { - "field_type": "long", - "field_name": "length", - "field_text": "int64_t", + "field_type": "Set", + "field_name": "entries", + "field_text": "List", "_N": 3, "field_local": true - }, - { - "field_type": "long", - "field_name": "fileSize", - "field_text": "int64_t", - "_N": 4, - "field_local": true - }, - { - "field_type": "long", - "field_name": "fileModifiedTime", - "field_text": "int64_t", - "_N": 5, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "extraFileInfo", - "field_text": "String", - "optional": true, - "_N": 6, - "field_local": true - }, - { - "field_type": "Map", - "field_name": "customSplitInfo", - "field_text": "Map", - "_N": 7, - "field_local": true } - ] + ], + "subclass": true, + "super_class": "ValueSet", + "json_key": "equatable" }, { - "class_name": "StageExecutionStrategy", + "class_name": "AggregationNodeStep", "enum": true, "elements": [ { - "element": "UNGROUPED_EXECUTION", + "element": "PARTIAL", "_N": 1 }, { - "element": "FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION", + "element": "FINAL", "_N": 2 }, { - "element": "DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION", + "element": "INTERMEDIATE", "_N": 3 }, { - "element": "RECOVERABLE_GROUPED_EXECUTION", + "element": "SINGLE", "_N": 4, "_last": true } ] }, { - "class_name": "StageExecutionDescriptor", + "class_name": "GroupingSetDescriptor", "struct": true, "fields": [ { - "field_type": "StageExecutionStrategy", - "field_name": "stageExecutionStrategy", - "field_text": "StageExecutionStrategy", + "field_type": "List", + "field_name": "groupingKeys", + "field_text": "List", "_N": 1, "field_local": true }, { - "field_type": "Set", - "field_name": "groupedExecutionScanNodes", - "field_text": "List", + "field_type": "int", + "field_name": "groupingSetCount", + "field_text": "int", "_N": 2, "field_local": true }, { - "field_type": "int", - "field_name": "totalLifespans", - "field_text": "int", + "field_type": "Set", + "field_name": "globalGroupingSets", + "field_text": "List", "_N": 3, "field_local": true } ] }, { - "class_name": "PlanFragment", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const PlanFragment& p) {\n j = json::object();\n to_json_key(j, \"id\", p.id, \"PlanFragment\", \"PlanFragmentId\", \"id\");\n to_json_key(j, \"root\", p.root, \"PlanFragment\", \"PlanNode\", \"root\");\n to_json_key(\n j,\n \"variables\",\n p.variables,\n \"PlanFragment\",\n \"List\",\n \"variables\");\n to_json_key(\n j,\n \"partitioning\",\n p.partitioning,\n \"PlanFragment\",\n \"PartitioningHandle\",\n \"partitioning\");\n to_json_key(\n j,\n \"tableScanSchedulingOrder\",\n p.tableScanSchedulingOrder,\n \"PlanFragment\",\n \"List\",\n \"tableScanSchedulingOrder\");\n to_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"PlanFragment\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n to_json_key(\n j,\n \"stageExecutionDescriptor\",\n p.stageExecutionDescriptor,\n \"PlanFragment\",\n \"StageExecutionDescriptor\",\n \"stageExecutionDescriptor\");\n to_json_key(\n j,\n \"outputTableWriterFragment\",\n p.outputTableWriterFragment,\n \"PlanFragment\",\n \"bool\",\n \"outputTableWriterFragment\");\n to_json_key(\n j,\n \"jsonRepresentation\",\n p.jsonRepresentation,\n \"PlanFragment\",\n \"String\",\n \"jsonRepresentation\");\n}\n\nvoid from_json(const json& j, PlanFragment& p) {\n from_json_key(j, \"id\", p.id, \"PlanFragment\", \"PlanFragmentId\", \"id\");\n from_json_key(j, \"root\", p.root, \"PlanFragment\", \"PlanNode\", \"root\");\n from_json_key(\n j,\n \"variables\",\n p.variables,\n \"PlanFragment\",\n \"List\",\n \"variables\");\n from_json_key(\n j,\n \"partitioning\",\n p.partitioning,\n \"PlanFragment\",\n \"PartitioningHandle\",\n \"partitioning\");\n from_json_key(\n j,\n \"tableScanSchedulingOrder\",\n p.tableScanSchedulingOrder,\n \"PlanFragment\",\n \"List\",\n \"tableScanSchedulingOrder\");\n from_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"PlanFragment\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n from_json_key(\n j,\n \"stageExecutionDescriptor\",\n p.stageExecutionDescriptor,\n \"PlanFragment\",\n \"StageExecutionDescriptor\",\n \"stageExecutionDescriptor\");\n from_json_key(\n j,\n \"outputTableWriterFragment\",\n p.outputTableWriterFragment,\n \"PlanFragment\",\n \"bool\",\n \"outputTableWriterFragment\");\n from_json_key(\n j,\n \"jsonRepresentation\",\n p.jsonRepresentation,\n \"PlanFragment\",\n \"String\",\n \"jsonRepresentation\");\n}\n} // namespace facebook::presto::protocol", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\nstruct PlanFragment {\n PlanFragmentId id = {};\n std::shared_ptr root = {};\n List variables = {};\n PartitioningHandle partitioning = {};\n List tableScanSchedulingOrder = {};\n PartitioningScheme partitioningScheme = {};\n StageExecutionDescriptor stageExecutionDescriptor = {};\n bool outputTableWriterFragment = {};\n std::shared_ptr jsonRepresentation = {};\n};\nvoid to_json(json& j, const PlanFragment& p);\nvoid from_json(const json& j, PlanFragment& p);\n} // namespace facebook::presto::protocol", + "class_name": "AggregationNode", "struct": true, "fields": [ { - "field_type": "PlanFragmentId", + "field_type": "PlanNodeId", "field_name": "id", - "field_text": "PlanFragmentId", + "field_text": "PlanNodeId", "_N": 1, - "field_local": true + "field_local": false }, { "field_type": "PlanNode", - "field_name": "root", + "field_name": "source", "field_text": "PlanNode", "_N": 2, "field_local": true, "optional": true }, { - "field_type": "Set", - "field_name": "variables", - "field_text": "List", + "field_type": "Map", + "field_name": "aggregations", + "field_text": "Map", "_N": 3, "field_local": true }, { - "field_type": "PartitioningHandle", - "field_name": "partitioning", - "field_text": "PartitioningHandle", + "field_type": "GroupingSetDescriptor", + "field_name": "groupingSets", + "field_text": "GroupingSetDescriptor", "_N": 4, "field_local": true }, { - "field_type": "List", - "field_name": "tableScanSchedulingOrder", - "field_text": "List", + "field_type": "List", + "field_name": "preGroupedVariables", + "field_text": "List", "_N": 5, "field_local": true }, { - "field_type": "PartitioningScheme", - "field_name": "partitioningScheme", - "field_text": "PartitioningScheme", + "field_type": "AggregationNodeStep", + "field_name": "step", + "field_text": "AggregationNodeStep", "_N": 6, "field_local": true }, { - "field_type": "StageExecutionDescriptor", - "field_name": "stageExecutionDescriptor", - "field_text": "StageExecutionDescriptor", + "field_type": "Optional", + "field_name": "hashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, "_N": 7, "field_local": true }, { - "field_type": "boolean", - "field_name": "outputTableWriterFragment", - "field_text": "bool", + "field_type": "Optional", + "field_name": "groupIdVariable", + "field_text": "VariableReferenceExpression", + "optional": true, "_N": 8, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": ".AggregationNode" + }, + { + "class_name": "ConstantExpression", + "struct": true, + "fields": [ { - "field_type": "StatsAndCosts", - "field_name": "statsAndCosts", - "field_text": "StatsAndCosts", - "_N": 9, + "field_type": "Block", + "field_name": "valueBlock", + "field_text": "Block", + "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "jsonRepresentation", - "field_text": "String", - "optional": true, - "_N": 10, + "field_type": "Type", + "field_name": "type", + "field_text": "Type", + "_N": 2, "field_local": true } - ] + ], + "subclass": true, + "super_class": "RowExpression", + "json_key": "constant" }, { - "class_name": "SelectedRoleType", - "enum": true, - "elements": [ + "class_name": "ValueSet", + "field_name": "valueSet", + "abstract": true, + "super_class": "JsonEncodedSubclass", + "subclasses": [ { - "element": "ROLE", + "type": "EquatableValueSet", + "name": "equatableValueSet", + "key": "equatable", "_N": 1 }, { - "element": "ALL", + "type": "SortedRangeSet", + "name": "sortedRangeSet", + "key": "sortable", "_N": 2 }, { - "element": "NONE", + "type": "AllOrNoneValueSet", + "name": "allOrNoneValueSet", + "key": "allOrNone", "_N": 3, "_last": true } - ] + ], + "fields": [] }, { - "class_name": "SelectedRole", + "class_name": "Domain", "struct": true, "fields": [ { - "field_type": "SelectedRoleType", - "field_name": "type", - "field_text": "SelectedRoleType", + "field_type": "ValueSet", + "field_name": "values", + "field_text": "ValueSet", "_N": 1, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "Optional", - "field_name": "role", - "field_text": "String", - "optional": true, + "field_type": "boolean", + "field_name": "nullAllowed", + "field_text": "bool", "_N": 2, "field_local": true } ] }, { - "class_name": "Language", + "class_name": "TupleDomain", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\ntemplate \nstruct pointerDerefCompare {\n bool operator()(const std::shared_ptr& a, const std::shared_ptr& b)\n const {\n return *a < *b;\n }\n};\n\ntemplate \nstruct TupleDomain {\n std::shared_ptr> domains;\n};\n\ntemplate \nstruct TupleDomain> {\n std::shared_ptr, Domain, pointerDerefCompare>>\n domains;\n};\n\ntemplate \nstruct ColumnDomain {\n T column;\n Domain domain; // dependency\n};\n\n} // namespace facebook::presto::protocol\n\nnamespace nlohmann {\n\ntemplate \nstruct adl_serializer> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::ColumnDomain& p) {\n facebook::presto::protocol::to_json_key(\n j, \"column\", p.column, \"ColumnDomain\", \"T\", \"column\");\n facebook::presto::protocol::to_json_key(\n j, \"domain\", p.domain, \"ColumnDomain\", \"Domain\", \"domain\");\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::ColumnDomain& p) {\n facebook::presto::protocol::from_json_key(\n j, \"column\", p.column, \"ColumnDomain\", \"T\", \"column\");\n facebook::presto::protocol::from_json_key(\n j, \"domain\", p.domain, \"ColumnDomain\", \"Domain\", \"domain\");\n }\n};\n\ntemplate \nstruct adl_serializer> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::TupleDomain& tup) {\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>\n list;\n if (tup.domains != nullptr) {\n for (auto& el : *tup.domains) {\n facebook::presto::protocol::ColumnDomain domain;\n domain.column = el.first;\n domain.domain = el.second;\n list.push_back(domain);\n }\n }\n\n j[\"columnDomains\"] = list;\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::TupleDomain& tup) {\n if (j.count(\"columnDomains\") != 0U) {\n std::shared_ptr>\n map = std::make_shared<\n std::map>();\n\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>\n list = j.at(\"columnDomains\");\n for (const facebook::presto::protocol::ColumnDomain& value : list) {\n map->insert(std::make_pair(T(value.column), value.domain));\n }\n tup.domains = map;\n }\n }\n};\n\ntemplate \nstruct adl_serializer<\n facebook::presto::protocol::TupleDomain>> {\n static void to_json(\n json& j,\n const facebook::presto::protocol::TupleDomain>& tup) {\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>>\n list;\n if (tup.domains != nullptr) {\n for (auto& el : *tup.domains) {\n facebook::presto::protocol::ColumnDomain> domain;\n domain.column = el.first;\n domain.domain = el.second;\n list.push_back(domain);\n }\n }\n\n j[\"columnDomains\"] = list;\n }\n\n static void from_json(\n const json& j,\n facebook::presto::protocol::TupleDomain>& tup) {\n if (j.count(\"columnDomains\") != 0U) {\n auto map = std::make_shared,\n facebook::presto::protocol::Domain,\n facebook::presto::protocol::pointerDerefCompare>>();\n\n facebook::presto::protocol::List<\n facebook::presto::protocol::ColumnDomain>>\n list = j.at(\"columnDomains\");\n for (const facebook::presto::protocol::ColumnDomain>&\n value : list) {\n map->insert(\n std::make_pair(std::shared_ptr(value.column), value.domain));\n }\n tup.domains = map;\n }\n }\n};\n\n} // namespace nlohmann" + }, + { + "class_name": "TpchTableHandle", "struct": true, "fields": [ { "field_type": "String", - "field_name": "language", + "field_name": "tableName", "field_text": "String", "_N": 1, "field_local": true + }, + { + "field_type": "double", + "field_name": "scaleFactor", + "field_text": "double", + "_N": 2, + "field_local": true } - ] + ], + "subclass": true, + "super_class": "ConnectorTableHandle", + "json_key": "tpch" }, { - "class_name": "Determinism", - "enum": true, - "elements": [ + "class_name": "TpchTableLayoutHandle", + "struct": true, + "fields": [ { - "element": "DETERMINISTIC", - "_N": 1 + "field_type": "TpchTableHandle", + "field_name": "table", + "field_text": "TpchTableHandle", + "_N": 1, + "field_local": true }, { - "element": "NOT_DETERMINISTIC", - "_N": 2 + "field_type": "TupleDomain", + "field_name": "predicate", + "field_text": "TupleDomain>", + "_N": 2, + "field_local": true } - ] + ], + "subclass": true, + "super_class": "ConnectorTableLayoutHandle", + "json_key": "tpch" }, { - "class_name": "NullCallClause", + "class_name": "HiveStorageFormat", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const HiveStorageFormat& p) {\n throw ParseError(\"Not implemented\");\n}\n\nstatic const std::pair HiveStorageFormat_enum_table[] =\n { // NOLINT: cert-err58-cpp\n {HiveStorageFormat::ORC, \"ORC\"},\n {HiveStorageFormat::DWRF, \"DWRF\"},\n {HiveStorageFormat::PARQUET, \"PARQUET\"},\n {HiveStorageFormat::AVRO, \"AVRO\"},\n {HiveStorageFormat::RCBINARY, \"RCBINARY\"},\n {HiveStorageFormat::RCTEXT, \"RCTEXT\"},\n {HiveStorageFormat::SEQUENCEFILE, \"SEQUENCEFILE\"},\n {HiveStorageFormat::JSON, \"JSON\"},\n {HiveStorageFormat::TEXTFILE, \"TEXTFILE\"},\n {HiveStorageFormat::CSV, \"CSV\"},\n {HiveStorageFormat::PAGEFILE, \"PAGEFILE\"}};\n\nvoid from_json(const json& j, HiveStorageFormat& e) {\n static_assert(\n std::is_enum::value,\n \"HiveStorageFormat must be an enum!\");\n const auto* it = std::find_if(\n std::begin(HiveStorageFormat_enum_table),\n std::end(HiveStorageFormat_enum_table),\n [&j](const std::pair& ej_pair) -> bool {\n return ej_pair.second == j;\n });\n e = ((it != std::end(HiveStorageFormat_enum_table))\n ? it\n : std::begin(HiveStorageFormat_enum_table))\n ->first;\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nenum class HiveStorageFormat {\n ORC,\n DWRF,\n PARQUET,\n AVRO,\n RCBINARY,\n RCTEXT,\n SEQUENCEFILE,\n JSON,\n TEXTFILE,\n CSV,\n PAGEFILE\n};\n\nvoid to_json(json& j, const HiveStorageFormat& p);\nvoid from_json(const json& j, HiveStorageFormat& p);\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "HiveCompressionCodec", "enum": true, "elements": [ { - "element": "RETURNS_NULL_ON_NULL_INPUT", + "element": "NONE", "_N": 1 }, { - "element": "CALLED_ON_NULL_INPUT", - "_N": 2, + "element": "SNAPPY", + "_N": 2 + }, + { + "element": "GZIP", + "_N": 3 + }, + { + "element": "LZ4", + "_N": 4 + }, + { + "element": "ZSTD", + "_N": 5, "_last": true } ] }, { - "class_name": "RoutineCharacteristics", - "struct": true, - "fields": [ + "class_name": "WriteMode", + "enum": true, + "elements": [ { - "field_type": "Optional", - "field_name": "language", - "field_text": "Language", - "optional": true, - "_N": 1, - "field_local": true + "element": "STAGE_AND_MOVE_TO_TARGET_DIRECTORY", + "_N": 1 }, { - "field_type": "Optional", - "field_name": "determinism", - "field_text": "Determinism", - "optional": true, - "_N": 2, - "field_local": true + "element": "DIRECT_TO_TARGET_NEW_DIRECTORY", + "_N": 2 }, { - "field_type": "Optional", - "field_name": "nullCallClause", - "field_text": "NullCallClause", - "optional": true, + "element": "DIRECT_TO_TARGET_EXISTING_DIRECTORY", "_N": 3, - "field_local": true + "_last": true } ] }, { - "class_name": "Parameter", - "struct": true, - "fields": [ + "class_name": "TableType", + "enum": true, + "elements": [ { - "field_type": "String", - "field_name": "name", - "field_text": "String", - "_N": 1, - "field_local": true + "element": "NEW", + "_N": 1 }, { - "field_type": "TypeSignature", - "field_name": "type", - "field_text": "TypeSignature", - "_N": 2, - "field_local": true + "element": "EXISTING", + "_N": 2 + }, + { + "element": "TEMPORARY", + "_N": 3, + "_last": true } ] }, { - "class_name": "SqlInvokedFunction", + "class_name": "LocationHandle", "struct": true, "fields": [ { - "field_type": "List", - "field_name": "parameters", - "field_text": "List", + "field_type": "String", + "field_name": "targetPath", + "field_text": "String", "_N": 1, "field_local": true }, { "field_type": "String", - "field_name": "description", + "field_name": "writePath", "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "RoutineCharacteristics", - "field_name": "routineCharacteristics", - "field_text": "RoutineCharacteristics", + "field_type": "Optional", + "field_name": "tempPath", + "field_text": "String", + "optional": true, "_N": 3, "field_local": true }, { - "field_type": "String", - "field_name": "body", - "field_text": "String", + "field_type": "TableType", + "field_name": "tableType", + "field_text": "TableType", "_N": 4, "field_local": true }, { - "field_type": "Signature", - "field_name": "signature", - "field_text": "Signature", + "field_type": "WriteMode", + "field_name": "writeMode", + "field_text": "WriteMode", "_N": 5, "field_local": true - }, - { - "field_type": "SqlFunctionId", - "field_name": "functionId", - "field_text": "SqlFunctionId", - "_N": 6, - "field_local": true } ] }, { - "class_name": "SessionRepresentation", + "class_name": "HiveInsertTableHandle", "struct": true, "fields": [ { "field_type": "String", - "field_name": "queryId", + "field_name": "schemaName", "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "transactionId", - "field_text": "TransactionId", - "optional": true, + "field_type": "String", + "field_name": "tableName", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "boolean", - "field_name": "clientTransactionSupport", - "field_text": "bool", + "field_type": "List", + "field_name": "inputColumns", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "String", - "field_name": "user", - "field_text": "String", + "field_type": "HivePageSinkMetadata", + "field_name": "pageSinkMetadata", + "field_text": "HivePageSinkMetadata", "_N": 4, "field_local": true }, { - "field_type": "Optional", - "field_name": "principal", - "field_text": "String", - "optional": true, + "field_type": "LocationHandle", + "field_name": "locationHandle", + "field_text": "LocationHandle", "_N": 5, "field_local": true }, { - "field_type": "Optional", - "field_name": "source", - "field_text": "String", + "field_type": "Optional", + "field_name": "bucketProperty", + "field_text": "HiveBucketProperty", "optional": true, "_N": 6, "field_local": true }, { - "field_type": "Optional", - "field_name": "catalog", - "field_text": "String", - "optional": true, + "field_type": "List", + "field_name": "preferredOrderingColumns", + "field_text": "List", "_N": 7, "field_local": true }, { - "field_type": "Optional", - "field_name": "schema", - "field_text": "String", - "optional": true, + "field_type": "HiveStorageFormat", + "field_name": "tableStorageFormat", + "field_text": "HiveStorageFormat", "_N": 8, "field_local": true }, { - "field_type": "Optional", - "field_name": "traceToken", - "field_text": "String", - "optional": true, + "field_type": "HiveStorageFormat", + "field_name": "partitionStorageFormat", + "field_text": "HiveStorageFormat", "_N": 9, "field_local": true }, { - "field_type": "TimeZoneKey", - "field_name": "timeZoneKey", - "field_text": "TimeZoneKey", + "field_type": "HiveStorageFormat", + "field_name": "actualStorageFormat", + "field_text": "HiveStorageFormat", "_N": 10, "field_local": true }, { - "field_type": "Locale", - "field_name": "locale", - "field_text": "Locale", + "field_type": "HiveCompressionCodec", + "field_name": "compressionCodec", + "field_text": "HiveCompressionCodec", "_N": 11, "field_local": true }, { - "field_type": "Optional", - "field_name": "remoteUserAddress", - "field_text": "String", + "field_type": "Optional", + "field_name": "encryptionInformation", + "field_text": "EncryptionInformation", "optional": true, "_N": 12, "field_local": true + } + ], + "subclass": true, + "super_class": "ConnectorInsertTableHandle", + "json_key": "hive" + }, + { + "class_name": "Assignments", + "struct": true, + "fields": [ + { + "field_type": "Map", + "field_name": "assignments", + "field_text": "Map>", + "_N": 1, + "field_local": true + } + ] + }, + { + "class_name": "Locality", + "enum": true, + "elements": [ + { + "element": "UNKNOWN", + "_N": 1 + }, + { + "element": "LOCAL", + "_N": 2 + }, + { + "element": "REMOTE", + "_N": 3, + "_last": true + } + ] + }, + { + "class_name": "ProjectNode", + "struct": true, + "fields": [ + { + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false + }, + { + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, + "field_local": true, + "optional": true + }, + { + "field_type": "Assignments", + "field_name": "assignments", + "field_text": "Assignments", + "_N": 3, + "field_local": true }, { - "field_type": "Optional", - "field_name": "userAgent", - "field_text": "String", - "optional": true, - "_N": 13, + "field_type": "Locality", + "field_name": "locality", + "field_text": "Locality", + "_N": 4, "field_local": true + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": ".ProjectNode" + }, + { + "class_name": "Step", + "enum": true, + "elements": [ + { + "element": "SINGLE", + "_N": 1 }, { - "field_type": "Optional", - "field_name": "clientInfo", - "field_text": "String", - "optional": true, - "_N": 14, + "element": "PARTIAL", + "_N": 2 + }, + { + "element": "FINAL", + "_N": 3, + "_last": true + } + ] + }, + { + "class_name": "TopNNode", + "struct": true, + "fields": [ + { + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false + }, + { + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, + "field_local": true, + "optional": true + }, + { + "field_type": "long", + "field_name": "count", + "field_text": "int64_t", + "_N": 3, + "field_local": true + }, + { + "field_type": "OrderingScheme", + "field_name": "orderingScheme", + "field_text": "OrderingScheme", + "_N": 4, + "field_local": true + }, + { + "field_type": "Step", + "field_name": "step", + "field_text": "Step", + "_N": 5, "field_local": true + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": ".TopNNode" + }, + { + "class_name": "SemiJoinNode", + "struct": true, + "fields": [ + { + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false }, { - "field_type": "Set", - "field_name": "clientTags", - "field_text": "List", - "_N": 15, - "field_local": true + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, + "field_local": true, + "optional": true }, { - "field_type": "ResourceEstimates", - "field_name": "resourceEstimates", - "field_text": "ResourceEstimates", - "_N": 16, - "field_local": true + "field_type": "PlanNode", + "field_name": "filteringSource", + "field_text": "PlanNode", + "_N": 3, + "field_local": true, + "optional": true }, { - "field_type": "long", - "field_name": "startTime", - "field_text": "int64_t", - "_N": 17, + "field_type": "VariableReferenceExpression", + "field_name": "sourceJoinVariable", + "field_text": "VariableReferenceExpression", + "_N": 4, "field_local": true }, { - "field_type": "Map", - "field_name": "systemProperties", - "field_text": "Map", - "_N": 18, + "field_type": "VariableReferenceExpression", + "field_name": "filteringSourceJoinVariable", + "field_text": "VariableReferenceExpression", + "_N": 5, "field_local": true }, { - "field_type": "Map>", - "field_name": "catalogProperties", - "field_text": "Map>", - "_N": 19, + "field_type": "VariableReferenceExpression", + "field_name": "semiJoinOutput", + "field_text": "VariableReferenceExpression", + "_N": 6, "field_local": true }, { - "field_type": "Map>", - "field_name": "unprocessedCatalogProperties", - "field_text": "Map>", - "_N": 20, + "field_type": "Optional", + "field_name": "sourceHashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 7, "field_local": true }, { - "field_type": "Map", - "field_name": "roles", - "field_text": "Map", - "_N": 21, + "field_type": "Optional", + "field_name": "filteringSourceHashVariable", + "field_text": "VariableReferenceExpression", + "optional": true, + "_N": 8, "field_local": true }, { - "field_type": "Map", - "field_name": "preparedStatements", - "field_text": "Map", - "_N": 22, + "field_type": "Optional", + "field_name": "distributionType", + "field_text": "DistributionType", + "optional": true, + "_N": 9, "field_local": true }, { - "field_type": "Map", - "field_name": "sessionFunctions", - "field_text": "Map", - "_N": 23, + "field_type": "Map", + "field_name": "dynamicFilters", + "field_text": "Map", + "_N": 10, "field_local": true } - ] + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.SemiJoinNode" }, { - "class_name": "ScheduledSplit", - "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\nnamespace facebook::presto::protocol {\n\nstruct ScheduledSplit {\n long sequenceId = {};\n PlanNodeId planNodeId = {}; // dependency\n Split split = {};\n\n bool operator<(const ScheduledSplit& o) const {\n return sequenceId < o.sequenceId;\n }\n};\n\nvoid to_json(json& j, const ScheduledSplit& p);\nvoid from_json(const json& j, ScheduledSplit& p);\n\n} // namespace facebook::presto::protocol", + "class_name": "HiveOutputTableHandle", "struct": true, "fields": [ { - "field_type": "long", - "field_name": "sequenceId", - "field_text": "int64_t", + "field_type": "String", + "field_name": "schemaName", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "PlanNodeId", - "field_name": "planNodeId", - "field_text": "PlanNodeId", + "field_type": "String", + "field_name": "tableName", + "field_text": "String", "_N": 2, "field_local": true }, { - "field_type": "Split", - "field_name": "split", - "field_text": "Split", + "field_type": "List", + "field_name": "inputColumns", + "field_text": "List", "_N": 3, "field_local": true - } - ] - }, - { - "class_name": "TaskSource", - "struct": true, - "fields": [ + }, { - "field_type": "PlanNodeId", - "field_name": "planNodeId", - "field_text": "PlanNodeId", - "_N": 1, + "field_type": "HivePageSinkMetadata", + "field_name": "pageSinkMetadata", + "field_text": "HivePageSinkMetadata", + "_N": 4, "field_local": true }, { - "field_type": "Set", - "field_name": "splits", - "field_text": "List", - "_N": 2, + "field_type": "LocationHandle", + "field_name": "locationHandle", + "field_text": "LocationHandle", + "_N": 5, "field_local": true }, { - "field_type": "Set", - "field_name": "noMoreSplitsForLifespan", - "field_text": "List", - "_N": 3, + "field_type": "HiveStorageFormat", + "field_name": "tableStorageFormat", + "field_text": "HiveStorageFormat", + "_N": 6, "field_local": true }, { - "field_type": "boolean", - "field_name": "noMoreSplits", - "field_text": "bool", - "_N": 4, + "field_type": "HiveStorageFormat", + "field_name": "partitionStorageFormat", + "field_text": "HiveStorageFormat", + "_N": 7, "field_local": true - } - ] - }, - { - "class_name": "DeleteScanInfo", - "struct": true, - "fields": [ + }, { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, + "field_type": "HiveStorageFormat", + "field_name": "actualStorageFormat", + "field_text": "HiveStorageFormat", + "_N": 8, "field_local": true }, { - "field_type": "TableHandle", - "field_name": "tableHandle", - "field_text": "TableHandle", - "_N": 2, + "field_type": "HiveCompressionCodec", + "field_name": "compressionCodec", + "field_text": "HiveCompressionCodec", + "_N": 9, "field_local": true - } - ] - }, - { - "class_name": "ExecutionWriterTarget", - "field_name": "executionWriterTarget", - "abstract": true, - "super_class": "JsonEncodedSubclass", - "subclasses": [ + }, { - "type": "CreateHandle", - "name": "createHandle", - "key": "CreateHandle", - "_N": 1 + "field_type": "List", + "field_name": "partitionedBy", + "field_text": "List", + "_N": 10, + "field_local": true }, { - "type": "InsertHandle", - "name": "insertHandle", - "key": "InsertHandle", - "_N": 2 + "field_type": "Optional", + "field_name": "bucketProperty", + "field_text": "HiveBucketProperty", + "optional": true, + "_N": 11, + "field_local": true }, { - "type": "DeleteHandle", - "name": "deleteHandle", - "key": "DeleteHandle", - "_N": 3, - "_last": true + "field_type": "List", + "field_name": "preferredOrderingColumns", + "field_text": "List", + "_N": 12, + "field_local": true + }, + { + "field_type": "String", + "field_name": "tableOwner", + "field_text": "String", + "_N": 13, + "field_local": true + }, + { + "field_type": "Map", + "field_name": "additionalTableParameters", + "field_text": "Map", + "_N": 14, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "encryptionInformation", + "field_text": "EncryptionInformation", + "optional": true, + "_N": 15, + "field_local": true } ], - "fields": [] + "subclass": true, + "super_class": "ConnectorOutputTableHandle", + "json_key": "hive" }, { - "class_name": "AnalyzeTableHandle", - "struct": true, - "fields": [ + "class_name": "FunctionKind", + "enum": true, + "elements": [ { - "field_type": "ConnectorId", - "field_name": "connectorId", - "field_text": "ConnectorId", - "_N": 1, - "field_local": true + "element": "SCALAR", + "_N": 1 }, - { - "field_type": "ConnectorTransactionHandle", - "field_name": "transactionHandle", - "field_text": "ConnectorTransactionHandle", - "_N": 2, - "field_local": true, - "optional": true + { + "element": "AGGREGATE", + "_N": 2 }, { - "field_type": "ConnectorTableHandle", - "field_name": "connectorHandle", - "field_text": "ConnectorTableHandle", + "element": "WINDOW", "_N": 3, - "field_local": true, - "optional": true + "_last": true } ] }, { - "class_name": "TableWriteInfo", + "class_name": "LongVariableConstraint", "struct": true, "fields": [ { - "field_type": "Optional", - "field_name": "writerTarget", - "field_text": "ExecutionWriterTarget", - "optional": true, + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "analyzeTableHandle", - "field_text": "AnalyzeTableHandle", - "optional": true, + "field_type": "String", + "field_name": "expression", + "field_text": "String", "_N": 2, "field_local": true - }, - { - "field_type": "Optional", - "field_name": "deleteScanInfo", - "field_text": "DeleteScanInfo", - "optional": true, - "_N": 3, - "field_local": true } ] }, { - "class_name": "TaskUpdateRequest", + "class_name": "Signature", "struct": true, "fields": [ { - "field_type": "SessionRepresentation", - "field_name": "session", - "field_text": "SessionRepresentation", + "field_type": "QualifiedObjectName", + "field_name": "name", + "field_text": "QualifiedObjectName", "_N": 1, "field_local": true }, { - "field_type": "Map", - "field_name": "extraCredentials", - "field_text": "Map", + "field_type": "FunctionKind", + "field_name": "kind", + "field_text": "FunctionKind", "_N": 2, "field_local": true }, { - "field_type": "Optional", - "field_name": "fragment", - "field_text": "String", - "optional": true, + "field_type": "List", + "field_name": "typeVariableConstraints", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "List", - "field_name": "sources", - "field_text": "List", + "field_type": "List", + "field_name": "longVariableConstraints", + "field_text": "List", "_N": 4, "field_local": true }, { - "field_type": "OutputBuffers", - "field_name": "outputIds", - "field_text": "OutputBuffers", + "field_type": "TypeSignature", + "field_name": "returnType", + "field_text": "TypeSignature", "_N": 5, "field_local": true }, { - "field_type": "Optional", - "field_name": "tableWriteInfo", - "field_text": "TableWriteInfo", - "optional": true, + "field_type": "List", + "field_name": "argumentTypes", + "field_text": "List", "_N": 6, "field_local": true + }, + { + "field_type": "boolean", + "field_name": "variableArity", + "field_text": "bool", + "_N": 7, + "field_local": true } ] }, { - "class_name": "GroupIdNode", + "class_name": "BuiltInFunctionHandle", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "Signature", + "field_name": "signature", + "field_text": "Signature", "_N": 1, - "field_local": false + "field_local": true + } + ], + "subclass": true, + "super_class": "FunctionHandle", + "json_key": "$static" + }, + { + "class_name": "AllOrNoneValueSet", + "struct": true, + "fields": [ + { + "field_type": "Type", + "field_name": "type", + "field_text": "Type", + "_N": 1, + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", + "field_type": "boolean", + "field_name": "all", + "field_text": "bool", "_N": 2, - "field_local": true, - "optional": true - }, + "field_local": true + } + ], + "subclass": true, + "super_class": "ValueSet", + "json_key": "allOrNone" + }, + { + "class_name": "DeleteHandle", + "struct": true, + "fields": [ { - "field_type": "List>", - "field_name": "groupingSets", - "field_text": "List>", - "_N": 3, + "field_type": "TableHandle", + "field_name": "handle", + "field_text": "TableHandle", + "_N": 1, "field_local": true }, { - "field_type": "Map", - "field_name": "groupingColumns", - "field_text": "Map", - "_N": 4, + "field_type": "SchemaTableName", + "field_name": "schemaTableName", + "field_text": "SchemaTableName", + "_N": 2, "field_local": true - }, + } + ], + "subclass": true, + "super_class": "ExecutionWriterTarget", + "json_key": "DeleteHandle" + }, + { + "class_name": "RefreshMaterializedViewHandle", + "struct": true, + "fields": [ { - "field_type": "List", - "field_name": "aggregationArguments", - "field_text": "List", - "_N": 5, + "field_type": "InsertTableHandle", + "field_name": "handle", + "field_text": "InsertTableHandle", + "_N": 1, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "groupIdVariable", - "field_text": "VariableReferenceExpression", - "_N": 6, + "field_type": "SchemaTableName", + "field_name": "schemaTableName", + "field_text": "SchemaTableName", + "_N": 2, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.GroupIdNode" + ] }, { - "class_name": "Locality", - "enum": true, - "elements": [ + "class_name": "HiveBucketHandle", + "struct": true, + "fields": [ { - "element": "UNKNOWN", - "_N": 1 + "field_type": "List", + "field_name": "columns", + "field_text": "List", + "_N": 1, + "field_local": true }, { - "element": "LOCAL", - "_N": 2 + "field_type": "int", + "field_name": "tableBucketCount", + "field_text": "int", + "_N": 2, + "field_local": true }, { - "element": "REMOTE", + "field_type": "int", + "field_name": "readBucketCount", + "field_text": "int", "_N": 3, - "_last": true + "field_local": true } ] }, { - "class_name": "ProjectNode", + "class_name": "HiveTableLayoutHandle", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "SchemaTableName", + "field_name": "schemaTableName", + "field_text": "SchemaTableName", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", + "field_type": "String", + "field_name": "tablePath", + "field_text": "String", "_N": 2, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "Assignments", - "field_name": "assignments", - "field_text": "Assignments", + "field_type": "List", + "field_name": "partitionColumns", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "Locality", - "field_name": "locality", - "field_text": "Locality", + "field_type": "List", + "field_name": "dataColumns", + "field_text": "List", "_N": 4, "field_local": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": ".ProjectNode" - }, - { - "class_name": "LimitNodeStep", - "enum": true, - "elements": [ - { - "element": "PARTIAL", - "_N": 1 }, { - "element": "FINAL", - "_N": 2, - "_last": true - } - ] - }, - { - "class_name": "LimitNode", - "struct": true, - "fields": [ + "field_type": "Map", + "field_name": "tableParameters", + "field_text": "Map", + "_N": 5, + "field_local": true + }, { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": false + "field_type": "TupleDomain", + "field_name": "domainPredicate", + "field_text": "TupleDomain", + "_N": 6, + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", - "_N": 2, + "field_type": "RowExpression", + "field_name": "remainingPredicate", + "field_text": "RowExpression", + "_N": 7, "field_local": true, "optional": true }, { - "field_type": "long", - "field_name": "count", - "field_text": "int64_t", - "_N": 3, + "field_type": "Map", + "field_name": "predicateColumns", + "field_text": "Map", + "_N": 8, "field_local": true }, { - "field_type": "LimitNodeStep", - "field_name": "step", - "field_text": "LimitNodeStep", - "_N": 4, + "field_type": "TupleDomain", + "field_name": "partitionColumnPredicate", + "field_text": "TupleDomain>", + "_N": 9, "field_local": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": ".LimitNode" - }, - { - "class_name": "UnnestNode", - "struct": true, - "fields": [ + }, { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": false + "field_type": "Optional", + "field_name": "bucketHandle", + "field_text": "HiveBucketHandle", + "optional": true, + "_N": 10, + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", - "_N": 2, - "field_local": true, - "optional": true + "field_type": "Optional", + "field_name": "bucketFilter", + "field_text": "HiveBucketFilter", + "optional": true, + "_N": 11, + "field_local": true }, { - "field_type": "List", - "field_name": "replicateVariables", - "field_text": "List", - "_N": 3, + "field_type": "boolean", + "field_name": "pushdownFilterEnabled", + "field_text": "bool", + "_N": 12, "field_local": true }, { - "field_type": "Map>", - "field_name": "unnestVariables", - "field_text": "Map>", - "_N": 4, + "field_type": "String", + "field_name": "layoutString", + "field_text": "String", + "_N": 13, "field_local": true }, { - "field_type": "Optional", - "field_name": "ordinalityVariable", - "field_text": "VariableReferenceExpression", + "field_type": "Optional>", + "field_name": "requestedColumns", + "field_text": "List", "optional": true, - "_N": 5, + "_N": 14, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "partialAggregationsPushedDown", + "field_text": "bool", + "_N": 15, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "appendRowNumber", + "field_text": "bool", + "_N": 16, "field_local": true } ], "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.UnnestNode" + "super_class": "ConnectorTableLayoutHandle", + "json_key": "hive" }, { - "class_name": "MemoryAllocation", - "struct": true, - "fields": [ + "class_name": "BufferType", + "enum": true, + "elements": [ { - "field_type": "String", - "field_name": "tag", - "field_text": "String", - "_N": 1, - "field_local": true + "element": "PARTITIONED", + "_N": 1 }, { - "field_type": "long", - "field_name": "allocation", - "field_text": "int64_t", - "_N": 2, - "field_local": true + "element": "BROADCAST", + "_N": 2 + }, + { + "element": "ARBITRARY", + "_N": 3 + }, + { + "element": "DISCARDING", + "_N": 4 + }, + { + "element": "SPOOLING", + "_N": 5, + "_last": true } ] }, { - "class_name": "MemoryPoolInfo", + "class_name": "OutputBuffers", "struct": true, "fields": [ { - "field_type": "long", - "field_name": "maxBytes", - "field_text": "int64_t", + "field_type": "BufferType", + "field_name": "type", + "field_text": "BufferType", "_N": 1, "field_local": true }, { "field_type": "long", - "field_name": "reservedBytes", + "field_name": "version", "field_text": "int64_t", "_N": 2, "field_local": true }, { - "field_type": "long", - "field_name": "reservedRevocableBytes", - "field_text": "int64_t", + "field_type": "boolean", + "field_name": "noMoreBufferIds", + "field_text": "bool", "_N": 3, "field_local": true }, { - "field_type": "Map", - "field_name": "queryMemoryReservations", - "field_text": "Map", + "field_type": "Map", + "field_name": "buffers", + "field_text": "Map", "_N": 4, "field_local": true + } + ] + }, + { + "class_name": "RuntimeUnit", + "enum": true, + "elements": [ + { + "element": "NONE", + "_N": 1 }, { - "field_type": "Map>", - "field_name": "queryMemoryAllocations", - "field_text": "Map>", - "_N": 5, - "field_local": true + "element": "NANO", + "_N": 2 }, { - "field_type": "Map", - "field_name": "queryMemoryRevocableReservations", - "field_text": "Map", - "_N": 6, - "field_local": true + "element": "BYTE", + "_N": 3, + "_last": true } ] }, { - "class_name": "RowNumberNode", + "class_name": "RuntimeMetric", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "String", + "field_name": "name", + "field_text": "String", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", + "field_type": "RuntimeUnit", + "field_name": "unit", + "field_text": "RuntimeUnit", "_N": 2, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "List", - "field_name": "partitionBy", - "field_text": "List", + "field_type": "long", + "field_name": "sum", + "field_text": "int64_t", "_N": 3, "field_local": true }, { - "field_type": "VariableReferenceExpression", - "field_name": "rowNumberVariable", - "field_text": "VariableReferenceExpression", + "field_type": "long", + "field_name": "count", + "field_text": "int64_t", "_N": 4, "field_local": true }, { - "field_type": "Optional", - "field_name": "maxRowCountPerPartition", - "field_text": "Integer", - "optional": true, + "field_type": "long", + "field_name": "max", + "field_text": "int64_t", "_N": 5, "field_local": true }, { - "field_type": "Optional", - "field_name": "hashVariable", - "field_text": "VariableReferenceExpression", - "optional": true, + "field_type": "long", + "field_name": "min", + "field_text": "int64_t", "_N": 6, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.RowNumberNode" + ] }, { - "class_name": "WindowNode", + "class_name": "UnnestNode", "struct": true, "fields": [ - { - "field_type": "Optional", - "field_name": "sourceLocation", - "field_text": "SourceLocation", - "optional": true, - "_N": 1, - "field_local": true - }, { "field_type": "PlanNodeId", "field_name": "id", "field_text": "PlanNodeId", - "_N": 2, + "_N": 1, "field_local": false }, { "field_type": "PlanNode", "field_name": "source", "field_text": "PlanNode", - "_N": 3, + "_N": 2, "field_local": true, "optional": true }, { - "field_type": "Specification", - "field_name": "specification", - "field_text": "Specification", - "_N": 4, + "field_type": "List", + "field_name": "replicateVariables", + "field_text": "List", + "_N": 3, "field_local": true }, { - "field_type": "Map", - "field_name": "windowFunctions", - "field_text": "Map", - "_N": 5, + "field_type": "Map>", + "field_name": "unnestVariables", + "field_text": "Map>", + "_N": 4, "field_local": true }, { "field_type": "Optional", - "field_name": "hashVariable", + "field_name": "ordinalityVariable", "field_text": "VariableReferenceExpression", "optional": true, - "_N": 6, - "field_local": true - }, - { - "field_type": "Set", - "field_name": "prePartitionedInputs", - "field_text": "List", - "_N": 7, - "field_local": true - }, - { - "field_type": "int", - "field_name": "preSortedOrderPrefix", - "field_text": "int", - "_N": 8, + "_N": 5, "field_local": true } ], "subclass": true, "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.WindowNode" + "json_key": "com.facebook.presto.sql.planner.plan.UnnestNode" }, { - "class_name": "HivePartitionKey", + "class_name": "ResourceEstimates", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "name", - "field_text": "String", + "field_type": "Optional", + "field_name": "executionTime", + "field_text": "Duration", + "optional": true, "_N": 1, "field_local": true }, { - "field_type": "Optional", - "field_name": "value", - "field_text": "String", + "field_type": "Optional", + "field_name": "cpuTime", + "field_text": "Duration", "optional": true, "_N": 2, "field_local": true + }, + { + "field_type": "Optional", + "field_name": "peakMemory", + "field_text": "DataSize", + "optional": true, + "_N": 3, + "field_local": true + }, + { + "field_type": "Optional", + "field_name": "peakTaskMemory", + "field_text": "DataSize", + "optional": true, + "_N": 4, + "field_local": true } ] }, { - "class_name": "BufferState", - "enum": true, - "elements": [ + "class_name": "SqlInvokedFunction", + "struct": true, + "fields": [ { - "element": "OPEN", - "_N": 1 + "field_type": "List", + "field_name": "parameters", + "field_text": "List", + "_N": 1, + "field_local": true }, { - "element": "NO_MORE_BUFFERS", - "_N": 2 + "field_type": "String", + "field_name": "description", + "field_text": "String", + "_N": 2, + "field_local": true }, { - "element": "NO_MORE_PAGES", - "_N": 3 + "field_type": "RoutineCharacteristics", + "field_name": "routineCharacteristics", + "field_text": "RoutineCharacteristics", + "_N": 3, + "field_local": true }, { - "element": "FLUSHING", - "_N": 4 + "field_type": "String", + "field_name": "body", + "field_text": "String", + "_N": 4, + "field_local": true }, { - "element": "FINISHED", - "_N": 5 + "field_type": "Signature", + "field_name": "signature", + "field_text": "Signature", + "_N": 5, + "field_local": true }, { - "element": "FAILED", + "field_type": "SqlFunctionId", + "field_name": "functionId", + "field_text": "SqlFunctionId", "_N": 6, - "_last": true + "field_local": true } ] }, { - "class_name": "OutputBufferInfo", + "class_name": "SessionRepresentation", "struct": true, "fields": [ { "field_type": "String", - "field_name": "type", + "field_name": "queryId", "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "BufferState", - "field_name": "state", - "field_text": "BufferState", + "field_type": "Optional", + "field_name": "transactionId", + "field_text": "TransactionId", + "optional": true, "_N": 2, "field_local": true }, { "field_type": "boolean", - "field_name": "canAddBuffers", + "field_name": "clientTransactionSupport", "field_text": "bool", "_N": 3, "field_local": true }, { - "field_type": "boolean", - "field_name": "canAddPages", - "field_text": "bool", + "field_type": "String", + "field_name": "user", + "field_text": "String", "_N": 4, "field_local": true }, { - "field_type": "long", - "field_name": "totalBufferedBytes", - "field_text": "int64_t", + "field_type": "Optional", + "field_name": "principal", + "field_text": "String", + "optional": true, "_N": 5, "field_local": true }, { - "field_type": "long", - "field_name": "totalBufferedPages", - "field_text": "int64_t", + "field_type": "Optional", + "field_name": "source", + "field_text": "String", + "optional": true, "_N": 6, "field_local": true }, { - "field_type": "long", - "field_name": "totalRowsSent", - "field_text": "int64_t", + "field_type": "Optional", + "field_name": "catalog", + "field_text": "String", + "optional": true, "_N": 7, "field_local": true }, { - "field_type": "long", - "field_name": "totalPagesSent", - "field_text": "int64_t", + "field_type": "Optional", + "field_name": "schema", + "field_text": "String", + "optional": true, "_N": 8, "field_local": true }, { - "field_type": "List", - "field_name": "buffers", - "field_text": "List", + "field_type": "Optional", + "field_name": "traceToken", + "field_text": "String", + "optional": true, "_N": 9, "field_local": true - } - ] - }, - { - "class_name": "ConstantExpression", - "struct": true, - "fields": [ - { - "field_type": "Block", - "field_name": "valueBlock", - "field_text": "Block", - "_N": 1, - "field_local": true - }, - { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", - "_N": 2, - "field_local": true - } - ], - "subclass": true, - "super_class": "RowExpression", - "json_key": "constant" - }, - { - "class_name": "InsertHandle", - "struct": true, - "fields": [ - { - "field_type": "InsertTableHandle", - "field_name": "handle", - "field_text": "InsertTableHandle", - "_N": 1, - "field_local": true - }, - { - "field_type": "SchemaTableName", - "field_name": "schemaTableName", - "field_text": "SchemaTableName", - "_N": 2, - "field_local": true - } - ], - "subclass": true, - "super_class": "ExecutionWriterTarget", - "json_key": "InsertHandle" - }, - { - "class_name": "MemoryInfo", - "struct": true, - "fields": [ - { - "field_type": "DataSize", - "field_name": "totalNodeMemory", - "field_text": "DataSize", - "_N": 1, - "field_local": true }, { - "field_type": "Map", - "field_name": "pools", - "field_text": "Map", - "_N": 2, - "field_local": true - } - ] - }, - { - "class_name": "NodeStatus", - "struct": true, - "fields": [ - { - "field_type": "String", - "field_name": "nodeId", - "field_text": "String", - "_N": 1, + "field_type": "TimeZoneKey", + "field_name": "timeZoneKey", + "field_text": "TimeZoneKey", + "_N": 10, "field_local": true }, { - "field_type": "NodeVersion", - "field_name": "nodeVersion", - "field_text": "NodeVersion", - "_N": 2, + "field_type": "Locale", + "field_name": "locale", + "field_text": "Locale", + "_N": 11, "field_local": true }, { - "field_type": "String", - "field_name": "environment", + "field_type": "Optional", + "field_name": "remoteUserAddress", "field_text": "String", - "_N": 3, + "optional": true, + "_N": 12, "field_local": true }, { - "field_type": "boolean", - "field_name": "coordinator", - "field_text": "bool", - "_N": 4, + "field_type": "Optional", + "field_name": "userAgent", + "field_text": "String", + "optional": true, + "_N": 13, "field_local": true }, { - "field_type": "Duration", - "field_name": "uptime", - "field_text": "Duration", - "_N": 5, + "field_type": "Optional", + "field_name": "clientInfo", + "field_text": "String", + "optional": true, + "_N": 14, "field_local": true }, { - "field_type": "String", - "field_name": "externalAddress", - "field_text": "String", - "_N": 6, + "field_type": "Set", + "field_name": "clientTags", + "field_text": "List", + "_N": 15, "field_local": true }, { - "field_type": "String", - "field_name": "internalAddress", - "field_text": "String", - "_N": 7, + "field_type": "ResourceEstimates", + "field_name": "resourceEstimates", + "field_text": "ResourceEstimates", + "_N": 16, "field_local": true }, { - "field_type": "MemoryInfo", - "field_name": "memoryInfo", - "field_text": "MemoryInfo", - "_N": 8, + "field_type": "long", + "field_name": "startTime", + "field_text": "int64_t", + "_N": 17, "field_local": true }, { - "field_type": "int", - "field_name": "processors", - "field_text": "int", - "_N": 9, + "field_type": "Map", + "field_name": "systemProperties", + "field_text": "Map", + "_N": 18, "field_local": true }, { - "field_type": "double", - "field_name": "processCpuLoad", - "field_text": "double", - "_N": 10, + "field_type": "Map>", + "field_name": "catalogProperties", + "field_text": "Map>", + "_N": 19, "field_local": true }, { - "field_type": "double", - "field_name": "systemCpuLoad", - "field_text": "double", - "_N": 11, + "field_type": "Map>", + "field_name": "unprocessedCatalogProperties", + "field_text": "Map>", + "_N": 20, "field_local": true }, { - "field_type": "long", - "field_name": "heapUsed", - "field_text": "int64_t", - "_N": 12, + "field_type": "Map", + "field_name": "roles", + "field_text": "Map", + "_N": 21, "field_local": true }, { - "field_type": "long", - "field_name": "heapAvailable", - "field_text": "int64_t", - "_N": 13, + "field_type": "Map", + "field_name": "preparedStatements", + "field_text": "Map", + "_N": 22, "field_local": true }, { - "field_type": "long", - "field_name": "nonHeapUsed", - "field_text": "int64_t", - "_N": 14, + "field_type": "Map", + "field_name": "sessionFunctions", + "field_text": "Map", + "_N": 23, "field_local": true } ] }, { - "class_name": "HiveTableHandle", + "class_name": "LambdaDefinitionExpression", "struct": true, "fields": [ { - "field_type": "String", - "field_name": "schemaName", - "field_text": "String", + "field_type": "Optional", + "field_name": "sourceLocation", + "field_text": "SourceLocation", + "optional": true, "_N": 1, - "field_local": true + "field_local": false }, { - "field_type": "String", - "field_name": "tableName", - "field_text": "String", + "field_type": "List", + "field_name": "argumentTypes", + "field_text": "List", "_N": 2, "field_local": true }, { - "field_type": "Optional>>", - "field_name": "analyzePartitionValues", - "field_text": "List>", - "optional": true, + "field_type": "List", + "field_name": "arguments", + "field_text": "List", "_N": 3, "field_local": true + }, + { + "field_type": "RowExpression", + "field_name": "body", + "field_text": "RowExpression", + "_N": 4, + "field_local": true, + "optional": true } ], "subclass": true, - "super_class": "ConnectorTableHandle", - "json_key": "hive" + "super_class": "RowExpression", + "json_key": "lambda" }, { - "class_name": "TaskInfo", + "class_name": "ValuesNode", "struct": true, "fields": [ { - "field_type": "TaskId", - "field_name": "taskId", - "field_text": "TaskId", + "field_type": "Optional", + "field_name": "location", + "field_text": "SourceLocation", + "optional": true, "_N": 1, "field_local": true }, { - "field_type": "TaskStatus", - "field_name": "taskStatus", - "field_text": "TaskStatus", + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", "_N": 2, - "field_local": true + "field_local": false }, { - "field_type": "DateTime", - "field_name": "lastHeartbeat", - "field_text": "DateTime", + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "OutputBufferInfo", - "field_name": "outputBuffers", - "field_text": "OutputBufferInfo", + "field_type": "List>", + "field_name": "rows", + "field_text": "List>>", "_N": 4, "field_local": true }, { - "field_type": "Set", - "field_name": "noMoreSplits", - "field_text": "List", + "field_type": "Optional", + "field_name": "valuesNodeLabel", + "field_text": "String", + "optional": true, "_N": 5, "field_local": true + } + ], + "subclass": true, + "super_class": "PlanNode", + "json_key": ".ValuesNode" + }, + { + "class_name": "SystemPartitionFunction", + "enum": true, + "elements": [ + { + "element": "SINGLE", + "_N": 1 }, { - "field_type": "TaskStats", - "field_name": "stats", - "field_text": "TaskStats", - "_N": 6, - "field_local": true + "element": "HASH", + "_N": 2 }, { - "field_type": "boolean", - "field_name": "needsPlan", - "field_text": "bool", - "_N": 7, - "field_local": true + "element": "ROUND_ROBIN", + "_N": 3 }, { - "field_type": "MetadataUpdates", - "field_name": "metadataUpdates", - "field_text": "MetadataUpdates", - "_N": 8, - "field_local": true + "element": "BROADCAST", + "_N": 4 }, { - "field_type": "String", - "field_name": "nodeId", - "field_text": "String", - "_N": 9, + "element": "UNKNOWN", + "_N": 5, + "_last": true + } + ] + }, + { + "class_name": "SystemPartitioning", + "enum": true, + "elements": [ + { + "element": "SINGLE", + "_N": 1 + }, + { + "element": "FIXED", + "_N": 2 + }, + { + "element": "SOURCE", + "_N": 3 + }, + { + "element": "SCALED", + "_N": 4 + }, + { + "element": "COORDINATOR_ONLY", + "_N": 5 + }, + { + "element": "ARBITRARY", + "_N": 6, + "_last": true + } + ] + }, + { + "class_name": "SystemPartitioningHandle", + "struct": true, + "fields": [ + { + "field_type": "SystemPartitioning", + "field_name": "partitioning", + "field_text": "SystemPartitioning", + "_N": 1, + "field_local": true + }, + { + "field_type": "SystemPartitionFunction", + "field_name": "function", + "field_text": "SystemPartitionFunction", + "_N": 2, "field_local": true } - ] + ], + "subclass": true, + "super_class": "ConnectorPartitioningHandle", + "json_key": "$remote" }, { - "class_name": "Range", + "class_name": "TpchPartitioningHandle", "struct": true, "fields": [ { - "field_type": "Marker", - "field_name": "low", - "field_text": "Marker", + "field_type": "String", + "field_name": "table", + "field_text": "String", "_N": 1, "field_local": true }, { - "field_type": "Marker", - "field_name": "high", - "field_text": "Marker", + "field_type": "long", + "field_name": "totalRows", + "field_text": "int64_t", "_N": 2, "field_local": true } - ] + ], + "subclass": true, + "super_class": "ConnectorPartitioningHandle", + "json_key": "tpch" }, { - "class_name": "SortedRangeSet", + "class_name": "TpchSplit", "struct": true, "fields": [ { - "field_type": "Type", - "field_name": "type", - "field_text": "Type", + "field_type": "TpchTableHandle", + "field_name": "tableHandle", + "field_text": "TpchTableHandle", "_N": 1, "field_local": true }, { - "field_type": "List", - "field_name": "ranges", - "field_text": "List", + "field_type": "int", + "field_name": "partNumber", + "field_text": "int", "_N": 2, "field_local": true + }, + { + "field_type": "int", + "field_name": "totalParts", + "field_text": "int", + "_N": 3, + "field_local": true + }, + { + "field_type": "List", + "field_name": "addresses", + "field_text": "List", + "_N": 4, + "field_local": true + }, + { + "field_type": "TupleDomain", + "field_name": "predicate", + "field_text": "TupleDomain>", + "_N": 5, + "field_local": true } ], "subclass": true, - "super_class": "ValueSet", - "json_key": "sortable" + "super_class": "ConnectorSplit", + "json_key": "tpch" }, { - "class_name": "AggregationNodeStep", + "class_name": "StageExecutionStrategy", "enum": true, "elements": [ { - "element": "PARTIAL", + "element": "UNGROUPED_EXECUTION", "_N": 1 }, { - "element": "FINAL", + "element": "FIXED_LIFESPAN_SCHEDULE_GROUPED_EXECUTION", "_N": 2 }, { - "element": "INTERMEDIATE", + "element": "DYNAMIC_LIFESPAN_SCHEDULE_GROUPED_EXECUTION", "_N": 3 }, { - "element": "SINGLE", + "element": "RECOVERABLE_GROUPED_EXECUTION", "_N": 4, "_last": true } ] }, { - "class_name": "GroupingSetDescriptor", + "class_name": "StageExecutionDescriptor", "struct": true, "fields": [ { - "field_type": "List", - "field_name": "groupingKeys", - "field_text": "List", + "field_type": "StageExecutionStrategy", + "field_name": "stageExecutionStrategy", + "field_text": "StageExecutionStrategy", "_N": 1, "field_local": true }, { - "field_type": "int", - "field_name": "groupingSetCount", - "field_text": "int", + "field_type": "Set", + "field_name": "groupedExecutionScanNodes", + "field_text": "List", "_N": 2, "field_local": true }, { - "field_type": "Set", - "field_name": "globalGroupingSets", - "field_text": "List", + "field_type": "int", + "field_name": "totalLifespans", + "field_text": "int", "_N": 3, "field_local": true } ] }, { - "class_name": "AggregationNode", + "class_name": "TaskUpdateRequest", "struct": true, "fields": [ { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", + "field_type": "SessionRepresentation", + "field_name": "session", + "field_text": "SessionRepresentation", "_N": 1, - "field_local": false + "field_local": true }, { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", + "field_type": "Map", + "field_name": "extraCredentials", + "field_text": "Map", "_N": 2, - "field_local": true, - "optional": true + "field_local": true }, { - "field_type": "Map", - "field_name": "aggregations", - "field_text": "Map", + "field_type": "Optional", + "field_name": "fragment", + "field_text": "String", + "optional": true, "_N": 3, "field_local": true }, { - "field_type": "GroupingSetDescriptor", - "field_name": "groupingSets", - "field_text": "GroupingSetDescriptor", + "field_type": "List", + "field_name": "sources", + "field_text": "List", "_N": 4, "field_local": true }, { - "field_type": "List", - "field_name": "preGroupedVariables", - "field_text": "List", + "field_type": "OutputBuffers", + "field_name": "outputIds", + "field_text": "OutputBuffers", "_N": 5, "field_local": true }, { - "field_type": "AggregationNodeStep", - "field_name": "step", - "field_text": "AggregationNodeStep", + "field_type": "Optional", + "field_name": "tableWriteInfo", + "field_text": "TableWriteInfo", + "optional": true, "_N": 6, "field_local": true + } + ] + }, + { + "class_name": "LimitNodeStep", + "enum": true, + "elements": [ + { + "element": "PARTIAL", + "_N": 1 }, { - "field_type": "Optional", - "field_name": "hashVariable", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 7, + "element": "FINAL", + "_N": 2, + "_last": true + } + ] + }, + { + "class_name": "LimitNode", + "struct": true, + "fields": [ + { + "field_type": "PlanNodeId", + "field_name": "id", + "field_text": "PlanNodeId", + "_N": 1, + "field_local": false + }, + { + "field_type": "PlanNode", + "field_name": "source", + "field_text": "PlanNode", + "_N": 2, + "field_local": true, + "optional": true + }, + { + "field_type": "long", + "field_name": "count", + "field_text": "int64_t", + "_N": 3, "field_local": true }, { - "field_type": "Optional", - "field_name": "groupIdVariable", - "field_text": "VariableReferenceExpression", - "optional": true, - "_N": 8, + "field_type": "LimitNodeStep", + "field_name": "step", + "field_text": "LimitNodeStep", + "_N": 4, "field_local": true } ], "subclass": true, "super_class": "PlanNode", - "json_key": ".AggregationNode" + "json_key": ".LimitNode" }, { "class_name": "HivePartitioningHandle", @@ -7207,7 +7547,11 @@ "json_key": "hive" }, { - "class_name": "SortNode", + "class_name": "JsonEncodedSubclass", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// dependency KeyedSubclass\n\nnamespace facebook::presto::protocol {\n\nstd::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) {\n return j[\"@type\"];\n}\n\n} // namespace facebook::presto::protocol" + }, + { + "class_name": "OutputNode", "struct": true, "fields": [ { @@ -7226,30 +7570,26 @@ "optional": true }, { - "field_type": "OrderingScheme", - "field_name": "orderingScheme", - "field_text": "OrderingScheme", + "field_type": "List", + "field_name": "columnNames", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "boolean", - "field_name": "isPartial", - "field_text": "bool", + "field_type": "List", + "field_name": "outputVariables", + "field_text": "List", "_N": 4, "field_local": true } ], "subclass": true, "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.SortNode" - }, - { - "class_name": "JsonEncodedSubclass", - "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\n// dependency KeyedSubclass\n\nnamespace facebook::presto::protocol {\n\nstd::string JsonEncodedSubclass::getSubclassKey(nlohmann::json j) {\n return j[\"@type\"];\n}\n\n} // namespace facebook::presto::protocol" + "json_key": "com.facebook.presto.sql.planner.plan.OutputNode" }, { - "class_name": "EnforceSingleRowNode", + "class_name": "SortNode", "struct": true, "fields": [ { @@ -7266,50 +7606,28 @@ "_N": 2, "field_local": true, "optional": true - } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.EnforceSingleRowNode" - }, - { - "class_name": "Location", - "struct": true, - "fields": [ - { - "field_type": "String", - "field_name": "location", - "field_text": "String", - "_N": 1, - "field_local": true - } - ] - }, - { - "class_name": "RemoteSplit", - "struct": true, - "fields": [ - { - "field_type": "Location", - "field_name": "location", - "field_text": "Location", - "_N": 1, - "field_local": true }, { - "field_type": "TaskId", - "field_name": "remoteSourceTaskId", - "field_text": "TaskId", - "_N": 2, + "field_type": "OrderingScheme", + "field_name": "orderingScheme", + "field_text": "OrderingScheme", + "_N": 3, + "field_local": true + }, + { + "field_type": "boolean", + "field_name": "isPartial", + "field_text": "bool", + "_N": 4, "field_local": true } ], "subclass": true, - "super_class": "ConnectorSplit", - "json_key": "$remote" + "super_class": "PlanNode", + "json_key": "com.facebook.presto.sql.planner.plan.SortNode" }, { - "class_name": "SemiJoinNode", + "class_name": "RowNumberNode", "struct": true, "fields": [ { @@ -7328,265 +7646,119 @@ "optional": true }, { - "field_type": "PlanNode", - "field_name": "filteringSource", - "field_text": "PlanNode", + "field_type": "List", + "field_name": "partitionBy", + "field_text": "List", "_N": 3, - "field_local": true, - "optional": true - }, - { - "field_type": "VariableReferenceExpression", - "field_name": "sourceJoinVariable", - "field_text": "VariableReferenceExpression", - "_N": 4, - "field_local": true - }, - { - "field_type": "VariableReferenceExpression", - "field_name": "filteringSourceJoinVariable", - "field_text": "VariableReferenceExpression", - "_N": 5, "field_local": true }, { "field_type": "VariableReferenceExpression", - "field_name": "semiJoinOutput", + "field_name": "rowNumberVariable", "field_text": "VariableReferenceExpression", - "_N": 6, + "_N": 4, "field_local": true }, { - "field_type": "Optional", - "field_name": "sourceHashVariable", - "field_text": "VariableReferenceExpression", + "field_type": "Optional", + "field_name": "maxRowCountPerPartition", + "field_text": "Integer", "optional": true, - "_N": 7, + "_N": 5, "field_local": true }, { "field_type": "Optional", - "field_name": "filteringSourceHashVariable", + "field_name": "hashVariable", "field_text": "VariableReferenceExpression", "optional": true, - "_N": 8, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "distributionType", - "field_text": "DistributionType", - "optional": true, - "_N": 9, - "field_local": true - }, - { - "field_type": "Map", - "field_name": "dynamicFilters", - "field_text": "Map", - "_N": 10, + "_N": 6, "field_local": true } ], "subclass": true, "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.SemiJoinNode" - }, - { - "class_name": "NodeSelectionStrategy", - "enum": true, - "elements": [ - { - "element": "HARD_AFFINITY", - "_N": 1 - }, - { - "element": "SOFT_AFFINITY", - "_N": 2 - }, - { - "element": "NO_PREFERENCE", - "_N": 3, - "_last": true - } - ] + "json_key": "com.facebook.presto.sql.planner.plan.RowNumberNode" }, { - "class_name": "HiveSplit", + "class_name": "PlanFragment", + "cinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\n\nvoid to_json(json& j, const PlanFragment& p) {\n j = json::object();\n to_json_key(j, \"id\", p.id, \"PlanFragment\", \"PlanFragmentId\", \"id\");\n to_json_key(j, \"root\", p.root, \"PlanFragment\", \"PlanNode\", \"root\");\n to_json_key(\n j,\n \"variables\",\n p.variables,\n \"PlanFragment\",\n \"List\",\n \"variables\");\n to_json_key(\n j,\n \"partitioning\",\n p.partitioning,\n \"PlanFragment\",\n \"PartitioningHandle\",\n \"partitioning\");\n to_json_key(\n j,\n \"tableScanSchedulingOrder\",\n p.tableScanSchedulingOrder,\n \"PlanFragment\",\n \"List\",\n \"tableScanSchedulingOrder\");\n to_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"PlanFragment\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n to_json_key(\n j,\n \"stageExecutionDescriptor\",\n p.stageExecutionDescriptor,\n \"PlanFragment\",\n \"StageExecutionDescriptor\",\n \"stageExecutionDescriptor\");\n to_json_key(\n j,\n \"outputTableWriterFragment\",\n p.outputTableWriterFragment,\n \"PlanFragment\",\n \"bool\",\n \"outputTableWriterFragment\");\n to_json_key(\n j,\n \"jsonRepresentation\",\n p.jsonRepresentation,\n \"PlanFragment\",\n \"String\",\n \"jsonRepresentation\");\n}\n\nvoid from_json(const json& j, PlanFragment& p) {\n from_json_key(j, \"id\", p.id, \"PlanFragment\", \"PlanFragmentId\", \"id\");\n from_json_key(j, \"root\", p.root, \"PlanFragment\", \"PlanNode\", \"root\");\n from_json_key(\n j,\n \"variables\",\n p.variables,\n \"PlanFragment\",\n \"List\",\n \"variables\");\n from_json_key(\n j,\n \"partitioning\",\n p.partitioning,\n \"PlanFragment\",\n \"PartitioningHandle\",\n \"partitioning\");\n from_json_key(\n j,\n \"tableScanSchedulingOrder\",\n p.tableScanSchedulingOrder,\n \"PlanFragment\",\n \"List\",\n \"tableScanSchedulingOrder\");\n from_json_key(\n j,\n \"partitioningScheme\",\n p.partitioningScheme,\n \"PlanFragment\",\n \"PartitioningScheme\",\n \"partitioningScheme\");\n from_json_key(\n j,\n \"stageExecutionDescriptor\",\n p.stageExecutionDescriptor,\n \"PlanFragment\",\n \"StageExecutionDescriptor\",\n \"stageExecutionDescriptor\");\n from_json_key(\n j,\n \"outputTableWriterFragment\",\n p.outputTableWriterFragment,\n \"PlanFragment\",\n \"bool\",\n \"outputTableWriterFragment\");\n from_json_key(\n j,\n \"jsonRepresentation\",\n p.jsonRepresentation,\n \"PlanFragment\",\n \"String\",\n \"jsonRepresentation\");\n}\n} // namespace facebook::presto::protocol", + "hinc": "/*\n * Licensed under the Apache License, Version 2.0 (the \"License\");\n * you may not use this file except in compliance with the License.\n * You may obtain a copy of the License at\n *\n * http://www.apache.org/licenses/LICENSE-2.0\n *\n * Unless required by applicable law or agreed to in writing, software\n * distributed under the License is distributed on an \"AS IS\" BASIS,\n * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.\n * See the License for the specific language governing permissions and\n * limitations under the License.\n */\n\nnamespace facebook::presto::protocol {\nstruct PlanFragment {\n PlanFragmentId id = {};\n std::shared_ptr root = {};\n List variables = {};\n PartitioningHandle partitioning = {};\n List tableScanSchedulingOrder = {};\n PartitioningScheme partitioningScheme = {};\n StageExecutionDescriptor stageExecutionDescriptor = {};\n bool outputTableWriterFragment = {};\n std::shared_ptr jsonRepresentation = {};\n};\nvoid to_json(json& j, const PlanFragment& p);\nvoid from_json(const json& j, PlanFragment& p);\n} // namespace facebook::presto::protocol", "struct": true, "fields": [ { - "field_type": "HiveFileSplit", - "field_name": "fileSplit", - "field_text": "HiveFileSplit", + "field_type": "PlanFragmentId", + "field_name": "id", + "field_text": "PlanFragmentId", "_N": 1, "field_local": true }, { - "field_type": "String", - "field_name": "database", - "field_text": "String", + "field_type": "PlanNode", + "field_name": "root", + "field_text": "PlanNode", "_N": 2, - "field_local": true + "field_local": true, + "optional": true }, { - "field_type": "String", - "field_name": "table", - "field_text": "String", + "field_type": "Set", + "field_name": "variables", + "field_text": "List", "_N": 3, "field_local": true }, { - "field_type": "String", - "field_name": "partitionName", - "field_text": "String", + "field_type": "PartitioningHandle", + "field_name": "partitioning", + "field_text": "PartitioningHandle", "_N": 4, "field_local": true }, { - "field_type": "Storage", - "field_name": "storage", - "field_text": "Storage", + "field_type": "List", + "field_name": "tableScanSchedulingOrder", + "field_text": "List", "_N": 5, "field_local": true }, { - "field_type": "List", - "field_name": "partitionKeys", - "field_text": "List", + "field_type": "PartitioningScheme", + "field_name": "partitioningScheme", + "field_text": "PartitioningScheme", "_N": 6, "field_local": true }, { - "field_type": "List", - "field_name": "addresses", - "field_text": "List", + "field_type": "StageExecutionDescriptor", + "field_name": "stageExecutionDescriptor", + "field_text": "StageExecutionDescriptor", "_N": 7, "field_local": true }, - { - "field_type": "OptionalInt", - "field_name": "readBucketNumber", - "field_text": "int", - "optional": true, - "_N": 8, - "field_local": true - }, - { - "field_type": "OptionalInt", - "field_name": "tableBucketNumber", - "field_text": "int", - "optional": true, - "_N": 9, - "field_local": true - }, - { - "field_type": "NodeSelectionStrategy", - "field_name": "nodeSelectionStrategy", - "field_text": "NodeSelectionStrategy", - "_N": 10, - "field_local": true - }, - { - "field_type": "int", - "field_name": "partitionDataColumnCount", - "field_text": "int", - "_N": 11, - "field_local": true - }, - { - "field_type": "TableToPartitionMapping", - "field_name": "tableToPartitionMapping", - "field_text": "TableToPartitionMapping", - "_N": 12, - "field_local": true - }, - { - "field_type": "Optional", - "field_name": "bucketConversion", - "field_text": "BucketConversion", - "optional": true, - "_N": 13, - "field_local": true - }, { "field_type": "boolean", - "field_name": "s3SelectPushdownEnabled", + "field_name": "outputTableWriterFragment", "field_text": "bool", - "_N": 14, + "_N": 8, "field_local": true }, { - "field_type": "CacheQuotaRequirement", - "field_name": "cacheQuota", - "field_text": "CacheQuotaRequirement", - "_N": 15, + "field_type": "StatsAndCosts", + "field_name": "statsAndCosts", + "field_text": "StatsAndCosts", + "_N": 9, "field_local": true }, { - "field_type": "Optional", - "field_name": "encryptionMetadata", - "field_text": "EncryptionInformation", + "field_type": "Optional", + "field_name": "jsonRepresentation", + "field_text": "String", "optional": true, - "_N": 16, - "field_local": true - }, - { - "field_type": "Set", - "field_name": "redundantColumnDomains", - "field_text": "List>", - "_N": 17, - "field_local": true - }, - { - "field_type": "SplitWeight", - "field_name": "splitWeight", - "field_text": "SplitWeight", - "_N": 18, - "field_local": true - } - ], - "subclass": true, - "super_class": "ConnectorSplit", - "json_key": "hive" - }, - { - "class_name": "OutputNode", - "struct": true, - "fields": [ - { - "field_type": "PlanNodeId", - "field_name": "id", - "field_text": "PlanNodeId", - "_N": 1, - "field_local": false - }, - { - "field_type": "PlanNode", - "field_name": "source", - "field_text": "PlanNode", - "_N": 2, - "field_local": true, - "optional": true - }, - { - "field_type": "List", - "field_name": "columnNames", - "field_text": "List", - "_N": 3, - "field_local": true - }, - { - "field_type": "List", - "field_name": "outputVariables", - "field_text": "List", - "_N": 4, + "_N": 10, "field_local": true } - ], - "subclass": true, - "super_class": "PlanNode", - "json_key": "com.facebook.presto.sql.planner.plan.OutputNode" + ] }, { "class_name": "NodeState", diff --git a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml index 65eef61e17894..2bbc4ef198871 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml +++ b/presto-native-execution/presto_cpp/presto_protocol/presto_protocol.yml @@ -50,17 +50,20 @@ AbstractClasses: comparable: true subclasses: - { name: HiveColumnHandle, key: hive } + - { name: TpchColumnHandle, key: tpch } ConnectorPartitioningHandle: super: JsonEncodedSubclass subclasses: - { name: SystemPartitioningHandle, key: $remote } - { name: HivePartitioningHandle, key: hive} + - { name: TpchPartitioningHandle, key: tpch} ConnectorTableHandle: super: JsonEncodedSubclass subclasses: - { name: HiveTableHandle, key: hive } + - { name: TpchTableHandle, key: tpch } ConnectorOutputTableHandle: super: JsonEncodedSubclass @@ -82,6 +85,7 @@ AbstractClasses: super: JsonEncodedSubclass subclasses: - { name: HiveTableLayoutHandle, key: hive } + - { name: TpchTableLayoutHandle, key: tpch } ConnectorMetadataUpdateHandle: super: JsonEncodedSubclass @@ -92,6 +96,7 @@ AbstractClasses: super: JsonEncodedSubclass subclasses: - { name: HiveSplit, key: hive } + - { name: TpchSplit, key: tpch } - { name: RemoteSplit, key: $remote } - { name: EmptySplit, key: $empty } @@ -219,6 +224,11 @@ JavaClasses: - presto-hive/src/main/java/com/facebook/presto/hive/HiveTableHandle.java - presto-hive/src/main/java/com/facebook/presto/hive/HiveTableLayoutHandle.java - presto-hive/src/main/java/com/facebook/presto/hive/HiveTransactionHandle.java + - presto-tpch/src/main/java/com/facebook/presto/tpch/TpchSplit.java + - presto-tpch/src/main/java/com/facebook/presto/tpch/TpchTableHandle.java + - presto-tpch/src/main/java/com/facebook/presto/tpch/TpchTableLayoutHandle.java + - presto-tpch/src/main/java/com/facebook/presto/tpch/TpchColumnHandle.java + - presto-tpch/src/main/java/com/facebook/presto/tpch/TpchPartitioningHandle.java - presto-spi/src/main/java/com/facebook/presto/spi/plan/LimitNode.java - presto-spi/src/main/java/com/facebook/presto/spi/function/LongVariableConstraint.java - presto-common/src/main/java/com/facebook/presto/common/predicate/Marker.java diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ColumnHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ColumnHandle.cpp.inc index 756b3128bbbf1..2d8c6b0957c54 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ColumnHandle.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ColumnHandle.cpp.inc @@ -23,6 +23,11 @@ void to_json(json& j, const std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + throw TypeError(type + " no abstract type ColumnHandle "); } @@ -41,6 +46,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + std::shared_ptr k = std::make_shared(); + j.get_to(*k); + p = std::static_pointer_cast(k); + return; + } + throw TypeError(type + " no abstract type ColumnHandle "); } -} // namespace facebook::presto::protocol \ No newline at end of file +} // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorPartitioningHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorPartitioningHandle.cpp.inc index 06197e1732015..0cd7645e21a07 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorPartitioningHandle.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorPartitioningHandle.cpp.inc @@ -26,6 +26,10 @@ void to_json(json& j, const std::shared_ptr& p) { j = *std::static_pointer_cast(p); return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); } @@ -50,6 +54,12 @@ void from_json(const json& j, std::shared_ptr& p) { p = k; return; } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } throw TypeError(type + " no abstract type ConnectorPartitioningHandle"); } diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorSplit.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorSplit.cpp.inc index e867c1fe6701b..e793613cdfd97 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorSplit.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorSplit.cpp.inc @@ -31,6 +31,11 @@ void to_json(json& j, const std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + throw TypeError(type + " no abstract type ConnectorSplit"); } @@ -61,6 +66,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + throw TypeError(type + " no abstract type ConnectorSplit"); } } // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableHandle.cpp.inc index a0f7faefec1f9..c45faac1a0296 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableHandle.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableHandle.cpp.inc @@ -23,6 +23,11 @@ void to_json(json& j, const std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + throw TypeError(type + " no abstract type ConnectorTableHandle"); } @@ -42,6 +47,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + throw TypeError(type + " no abstract type ConnectorTableHandle"); } } // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableLayoutHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableLayoutHandle.cpp.inc index 49cc6d19aeaec..1ba7ad7746f9f 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableLayoutHandle.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTableLayoutHandle.cpp.inc @@ -23,6 +23,11 @@ void to_json(json& j, const std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } @@ -43,6 +48,13 @@ void from_json(const json& j, std::shared_ptr& p) { return; } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } + throw TypeError(type + " no abstract type ConnectorTableLayoutHandle"); } } // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTransactionHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTransactionHandle.cpp.inc index 5e2cadb64ca22..f8109a7a29b58 100644 --- a/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTransactionHandle.cpp.inc +++ b/presto-native-execution/presto_cpp/presto_protocol/special/ConnectorTransactionHandle.cpp.inc @@ -11,6 +11,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + +// dependency TpchTransactionHandle + namespace facebook::presto::protocol { void to_json(json& j, const std::shared_ptr& p) { if (p == nullptr) { @@ -26,6 +29,10 @@ void to_json(json& j, const std::shared_ptr& p) { j = *std::static_pointer_cast(p); return; } + if (getConnectorKey(type) == "tpch") { + j = *std::static_pointer_cast(p); + return; + } throw TypeError(type + " no abstract type ConnectorTransactionHandle"); } @@ -33,7 +40,12 @@ void to_json(json& j, const std::shared_ptr& p) { void from_json(const json& j, std::shared_ptr& p) { String type; try { - type = p->getSubclassKey(j); + // TPC-H transactionHandle is an array ["tpch","INSTANCE"]. + if (j.is_array()) { + type = j[0]; + } else { + type = p->getSubclassKey(j); + } } catch (json::parse_error& e) { throw ParseError( std::string(e.what()) + @@ -52,6 +64,12 @@ void from_json(const json& j, std::shared_ptr& p) { p = k; return; } + if (getConnectorKey(type) == "tpch") { + auto k = std::make_shared(); + j.get_to(*k); + p = k; + return; + } throw TypeError(type + " no abstract type ConnectorTransactionHandle"); } diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc new file mode 100644 index 0000000000000..b5d7ef49be3fd --- /dev/null +++ b/presto-native-execution/presto_cpp/presto_protocol/special/TpchColumnHandle.hpp.inc @@ -0,0 +1,31 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchColumnHandle is special since we require an implementation of +// operator<(). + +namespace facebook::presto::protocol { +struct TpchColumnHandle : public ColumnHandle { + String columnName = {}; + Type type = {}; + + TpchColumnHandle() noexcept; + + bool operator<(const ColumnHandle& o) const override { + return columnName < dynamic_cast(o).columnName; + } +}; +void to_json(json& j, const TpchColumnHandle& p); +void from_json(const json& j, TpchColumnHandle& p); +} // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.cpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.cpp.inc new file mode 100644 index 0000000000000..9d9915415c05a --- /dev/null +++ b/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.cpp.inc @@ -0,0 +1,30 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. + +namespace facebook::presto::protocol { + +void to_json(json& j, const TpchTransactionHandle& p) { + j = json::array(); + j.push_back(p._type); + j.push_back(p.instance); +} + +void from_json(const json& j, TpchTransactionHandle& p) { + j[0].get_to(p._type); + j[1].get_to(p.instance); +} +} // namespace facebook::presto::protocol diff --git a/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.hpp.inc b/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.hpp.inc new file mode 100644 index 0000000000000..1886cc971cf59 --- /dev/null +++ b/presto-native-execution/presto_cpp/presto_protocol/special/TpchTransactionHandle.hpp.inc @@ -0,0 +1,28 @@ +/* + * Licensed under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +// TpchTransactionHandle is special since +// the corresponding class in Java is an enum. + +namespace facebook::presto::protocol { + +struct TpchTransactionHandle : public ConnectorTransactionHandle { + String instance = {}; +}; + +void to_json(json& j, const TpchTransactionHandle& p); + +void from_json(const json& j, TpchTransactionHandle& p); + +} // namespace facebook::presto::protocol